Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-3682] testReaderFilterRowKeys fails in TestHoodieOrcReaderWriter #5790

Merged
merged 1 commit into from
Jun 13, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,15 @@
import org.mockito.Mockito;

import java.io.IOException;
import java.util.function.Supplier;

import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
import static org.apache.hudi.io.storage.HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.when;

public class TestHoodieOrcReaderWriter extends TestHoodieReaderWriterBase {

Expand All @@ -59,6 +61,9 @@ protected HoodieFileWriter<GenericRecord> createWriter(
int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_SIZE.defaultValue());
HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter);
TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
Supplier<Integer> partitionSupplier = Mockito.mock(Supplier.class);
when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier);
when(partitionSupplier.get()).thenReturn(10);
String instantTime = "000";
return new HoodieOrcWriter<>(instantTime, getFilePath(), config, avroSchema, mockTaskContextSupplier);
}
Expand Down Expand Up @@ -92,9 +97,4 @@ protected void verifySchema(Configuration conf, String schemaPath) throws IOExce
orcReader.getSchema().toString());
}
}

@Override
public void testReaderFilterRowKeys() {
// TODO(HUDI-3682): fix filterRowKeys test for ORC due to a bug in ORC logic
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.hudi.io.storage;

import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieKey;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
Expand Down Expand Up @@ -157,7 +158,7 @@ public void testWriteReadWithEvolvedSchema() throws Exception {

@Test
public void testReaderFilterRowKeys() throws Exception {
writeFileWithSimpleSchema();
writeFileWithSchemaWithMeta();
Configuration conf = new Configuration();
verifyMetadata(conf);
verifyFilterRowKeys(createReader(conf));
Expand All @@ -177,6 +178,21 @@ protected void writeFileWithSimpleSchema() throws Exception {
writer.close();
}

protected void writeFileWithSchemaWithMeta() throws Exception {
Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchemaWithMetaFields.avsc");
HoodieFileWriter<GenericRecord> writer = createWriter(avroSchema, true);
for (int i = 0; i < NUM_RECORDS; i++) {
GenericRecord record = new GenericData.Record(avroSchema);
String key = "key" + String.format("%02d", i);
record.put("_row_key", key);
record.put("time", Integer.toString(i));
record.put("number", i);
writer.writeAvroWithMetadata(new HoodieKey((String) record.get("_row_key"),
Integer.toString((Integer) record.get("number"))), record);
}
writer.close();
}

protected void verifySimpleRecords(Iterator<GenericRecord> iterator) {
int index = 0;
while (iterator.hasNext()) {
Expand Down