[HUDI-995] Use Transformations, Assertions and SchemaTestUtil (#1884)
- Consolidate transform functions for tests in Transformations.java - Consolidate assertion functions for tests in Assertions.java - Make use of SchemaTestUtil for loading schema from resource
This commit is contained in:
@@ -83,6 +83,8 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_S
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.NULL_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
|
||||
import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet;
|
||||
import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -634,7 +636,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
Set<String> keys1 = recordsToRecordKeySet(inserts1);
|
||||
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
|
||||
@@ -651,7 +653,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime2 = "002";
|
||||
client.startCommitWithTime(commitTime2);
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
Set<String> keys2 = recordsToRecordKeySet(inserts2);
|
||||
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
|
||||
insertsAndUpdates2.addAll(inserts2);
|
||||
insertsAndUpdates2.addAll(dataGen.generateUpdates(commitTime2, inserts1));
|
||||
@@ -678,7 +680,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime3 = "003";
|
||||
client.startCommitWithTime(commitTime3);
|
||||
List<HoodieRecord> insertsAndUpdates3 = dataGen.generateInserts(commitTime3, 200);
|
||||
Set<String> keys3 = HoodieClientTestUtils.getRecordKeys(insertsAndUpdates3);
|
||||
Set<String> keys3 = recordsToRecordKeySet(insertsAndUpdates3);
|
||||
List<HoodieRecord> updates3 = dataGen.generateUpdates(commitTime3, inserts2);
|
||||
insertsAndUpdates3.addAll(updates3);
|
||||
|
||||
@@ -745,7 +747,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
Set<String> keys1 = recordsToRecordKeySet(inserts1);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
|
||||
|
||||
@@ -762,7 +764,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime2 = "002";
|
||||
client.startCommitWithTime(commitTime2);
|
||||
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
|
||||
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
|
||||
Set<String> keys2 = recordsToRecordKeySet(inserts2);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
|
||||
statuses = client.insert(insertRecordsRDD2, commitTime2).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
@@ -826,7 +828,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String commitTime1 = "001";
|
||||
client.startCommitWithTime(commitTime1);
|
||||
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
|
||||
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
|
||||
Set<String> keys1 = recordsToRecordKeySet(inserts1);
|
||||
List<String> keysSoFar = new ArrayList<>(keys1);
|
||||
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
|
||||
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
|
||||
@@ -858,8 +860,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
client.startCommitWithTime(commitTime6);
|
||||
|
||||
List<HoodieRecord> dummyInserts3 = dataGen.generateInserts(commitTime6, 20);
|
||||
List<HoodieKey> hoodieKeysToDelete3 = HoodieClientTestUtils
|
||||
.getKeysToDelete(HoodieClientTestUtils.getHoodieKeys(dummyInserts3), 20);
|
||||
List<HoodieKey> hoodieKeysToDelete3 = randomSelectAsHoodieKeys(dummyInserts3, 20);
|
||||
JavaRDD<HoodieKey> deleteKeys3 = jsc.parallelize(hoodieKeysToDelete3, 1);
|
||||
statuses = client.delete(deleteKeys3, commitTime6).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
@@ -884,7 +885,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
throws IOException {
|
||||
client.startCommitWithTime(instantTime);
|
||||
List<HoodieRecord> inserts = dataGen.generateInserts(instantTime, sizeToInsertAndUpdate);
|
||||
Set<String> keys = HoodieClientTestUtils.getRecordKeys(inserts);
|
||||
Set<String> keys = recordsToRecordKeySet(inserts);
|
||||
List<HoodieRecord> insertsAndUpdates = new ArrayList<>();
|
||||
insertsAndUpdates.addAll(inserts);
|
||||
insertsAndUpdates.addAll(dataGen.generateUpdates(instantTime, inserts));
|
||||
@@ -908,8 +909,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
String existingFile, String instantTime, int exepctedRecords, List<String> keys) {
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieKey> hoodieKeysToDelete = HoodieClientTestUtils
|
||||
.getKeysToDelete(HoodieClientTestUtils.getHoodieKeys(previousRecords), sizeToDelete);
|
||||
List<HoodieKey> hoodieKeysToDelete = randomSelectAsHoodieKeys(previousRecords, sizeToDelete);
|
||||
JavaRDD<HoodieKey> deleteKeys = jsc.parallelize(hoodieKeysToDelete, 1);
|
||||
List<WriteStatus> statuses = client.delete(deleteKeys, instantTime).collect();
|
||||
|
||||
@@ -958,8 +958,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
client.startCommitWithTime(commitTime1);
|
||||
|
||||
List<HoodieRecord> dummyInserts = dataGen.generateInserts(commitTime1, 20);
|
||||
List<HoodieKey> hoodieKeysToDelete = HoodieClientTestUtils
|
||||
.getKeysToDelete(HoodieClientTestUtils.getHoodieKeys(dummyInserts), 20);
|
||||
List<HoodieKey> hoodieKeysToDelete = randomSelectAsHoodieKeys(dummyInserts, 20);
|
||||
JavaRDD<HoodieKey> deleteKeys = jsc.parallelize(hoodieKeysToDelete, 1);
|
||||
assertThrows(HoodieIOException.class, () -> {
|
||||
client.delete(deleteKeys, commitTime1).collect();
|
||||
|
||||
@@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
@@ -33,6 +32,7 @@ import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -46,6 +46,7 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
@@ -135,8 +136,8 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
}).collect().size());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream(schema));
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String name) {
|
||||
Schema schema = getSchemaFromResource(getClass(), name);
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -31,7 +30,6 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
@@ -64,6 +62,7 @@ import java.util.UUID;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
@@ -71,18 +70,15 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.txt", true);
|
||||
private final Random random = new Random();
|
||||
private IndexType indexType;
|
||||
private HoodieIndex index;
|
||||
private HoodieWriteConfig config;
|
||||
private Schema schema;
|
||||
|
||||
private void setUp(IndexType indexType) throws Exception {
|
||||
this.indexType = indexType;
|
||||
initResources();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.build()).withAutoCommit(false).build();
|
||||
@@ -284,11 +280,11 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String filename1 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), SCHEMA, null, true);
|
||||
String filename2 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), SCHEMA, null, true);
|
||||
String filename3 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), SCHEMA, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -380,11 +376,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
|
||||
HoodieClientTestUtils
|
||||
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), schema, null, false);
|
||||
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), SCHEMA, null, false);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
@@ -27,7 +26,6 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
@@ -64,6 +62,7 @@ import java.util.stream.Stream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
@@ -73,9 +72,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private String schemaStr;
|
||||
private Schema schema;
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.txt", true);
|
||||
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
|
||||
|
||||
public static Stream<Arguments> configParams() {
|
||||
@@ -90,8 +87,6 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
initPath();
|
||||
initFileSystem();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@@ -141,13 +136,13 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet", new ArrayList<>(),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet", new ArrayList<>(),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet", Collections.singletonList(record1),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "4_0_20150312101010.parquet",
|
||||
Arrays.asList(record2, record3, record4), schema, null, false);
|
||||
Arrays.asList(record2, record3, record4), SCHEMA, null, false);
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -245,7 +240,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record3.getRecordKey());
|
||||
String filename = HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1, record2),
|
||||
schema, filter, true);
|
||||
SCHEMA, filter, true);
|
||||
|
||||
// The bloom filter contains 3 records
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
@@ -333,11 +328,11 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String filename1 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), SCHEMA, null, true);
|
||||
String filename2 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), SCHEMA, null, true);
|
||||
String filename3 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), SCHEMA, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -405,11 +400,11 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String filename1 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), SCHEMA, null, true);
|
||||
String filename2 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record2), SCHEMA, null, true);
|
||||
String filename3 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), schema, null, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Collections.singletonList(record4), SCHEMA, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -455,7 +450,7 @@ public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record2.getRecordKey());
|
||||
String filename =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), schema, filter, true);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(record1), SCHEMA, filter, true);
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
@@ -26,7 +25,6 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -55,6 +53,7 @@ import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
@@ -64,18 +63,12 @@ import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private Schema schema;
|
||||
|
||||
public TestHoodieGlobalBloomIndex() {
|
||||
}
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.txt", true);
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@@ -119,13 +112,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet", new ArrayList<>(),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet", new ArrayList<>(),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet", Collections.singletonList(record1),
|
||||
schema, null, false);
|
||||
SCHEMA, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", "4_0_20150312101010.parquet",
|
||||
Arrays.asList(record2, record3, record4), schema, null, false);
|
||||
Arrays.asList(record2, record3, record4), SCHEMA, null, false);
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
@@ -250,13 +243,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
|
||||
|
||||
String filename0 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1), schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1), SCHEMA, null, false);
|
||||
String filename1 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(), schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(), SCHEMA, null, false);
|
||||
String filename2 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2), schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2), SCHEMA, null, false);
|
||||
String filename3 =
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4), schema, null, false);
|
||||
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4), SCHEMA, null, false);
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
@@ -342,7 +335,7 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
HoodieClientTestUtils
|
||||
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), schema, null, false);
|
||||
.writeParquetFile(basePath, "2016/01/31", Collections.singletonList(originalRecord), SCHEMA, null, false);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
|
||||
|
||||
@@ -42,6 +42,7 @@ import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
@@ -1431,7 +1432,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
actionExecutor.getUpsertPartitioner(new WorkloadProfile(deleteRDD));
|
||||
final List<List<WriteStatus>> deleteStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}).map(Transformations::flatten).collect();
|
||||
|
||||
// Verify there are errors because records are from multiple partitions (but handleUpdate is invoked for
|
||||
// specific partition)
|
||||
|
||||
@@ -28,7 +28,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -40,9 +40,9 @@ import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -68,6 +68,7 @@ import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateExpectedPartitionNumRecords;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateTestRecordsForBulkInsert;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
@@ -78,9 +79,10 @@ import static org.mockito.Mockito.when;
|
||||
public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.txt");
|
||||
|
||||
@Test
|
||||
public void testMakeNewPath() throws Exception {
|
||||
public void testMakeNewPath() {
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String partitionPath = "2016/05/04";
|
||||
|
||||
@@ -102,14 +104,13 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
|
||||
private HoodieWriteConfig makeHoodieClientConfig() {
|
||||
return makeHoodieClientConfigBuilder().build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
|
||||
}
|
||||
|
||||
// TODO (weiy): Add testcases for crossing file writing.
|
||||
@@ -288,7 +289,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
firstCommitTime, jsc.parallelize(records));
|
||||
List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
Map<String, String> allWriteStatusMergedMetadataMap =
|
||||
MetadataMergeWriteStatus.mergeMetadataForWriteStatuses(writeStatuses);
|
||||
@@ -331,7 +332,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
instantTime, jsc.parallelize(recs2));
|
||||
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
// TODO: check the actual files and make sure 11 records, total were written.
|
||||
assertEquals(2, returnedStatuses.size());
|
||||
@@ -352,7 +353,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
instantTime, jsc.parallelize(recs3));
|
||||
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator());
|
||||
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
assertEquals(3, returnedStatuses.size());
|
||||
expectedPartitionNumRecords.clear();
|
||||
@@ -384,7 +385,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
instantTime, jsc.parallelize(records));
|
||||
jsc.parallelize(Arrays.asList(1))
|
||||
.map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()))
|
||||
.map(x -> HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
.map(Transformations::flatten).collect();
|
||||
|
||||
// Check the updated file
|
||||
int counts = 0;
|
||||
@@ -410,7 +411,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
instantTime, jsc.parallelize(inserts));
|
||||
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}).map(Transformations::flatten).collect();
|
||||
|
||||
WriteStatus writeStatus = ws.get(0).get(0);
|
||||
String fileId = writeStatus.getFileId();
|
||||
@@ -423,7 +424,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
instantTime, jsc.parallelize(updates));
|
||||
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
|
||||
}).map(x -> (List<WriteStatus>) HoodieClientTestUtils.collectStatuses(x)).collect();
|
||||
}).map(Transformations::flatten).collect();
|
||||
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
|
||||
}
|
||||
|
||||
|
||||
@@ -26,7 +26,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
@@ -37,6 +36,7 @@ import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -53,6 +53,7 @@ import java.util.Map;
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.generateFakeHoodieWriteStat;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.table.action.commit.UpsertPartitioner.averageBytesPerRecord;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
@@ -63,6 +64,7 @@ import static org.mockito.Mockito.when;
|
||||
public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestUpsertPartitioner.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestUpsertPartitioner.class, "/exampleSchema.txt");
|
||||
|
||||
private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
|
||||
String testPartitionPath, boolean autoSplitInserts) throws Exception {
|
||||
@@ -273,9 +275,8 @@ public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
assertEquals(200.0 / 2400, insertBuckets.get(0).weight, 0.01, "First insert bucket should have weight 0.5");
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||
// Prepare the AvroParquetIO
|
||||
String schemaStr = FileIOUtils.readAsUTFString(getClass().getResourceAsStream("/exampleSchema.txt"));
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,7 +22,6 @@ import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
@@ -30,7 +29,6 @@ import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -65,12 +63,8 @@ import java.io.RandomAccessFile;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@@ -80,46 +74,8 @@ import java.util.stream.Collectors;
|
||||
public class HoodieClientTestUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieClientTestUtils.class);
|
||||
private static final Random RANDOM = new Random();
|
||||
public static final String DEFAULT_WRITE_TOKEN = "1-0-1";
|
||||
|
||||
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
while (statusListItr.hasNext()) {
|
||||
statuses.addAll(statusListItr.next());
|
||||
}
|
||||
return statuses;
|
||||
}
|
||||
|
||||
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
|
||||
Set<String> keys = new HashSet<>();
|
||||
for (HoodieRecord rec : hoodieRecords) {
|
||||
keys.add(rec.getRecordKey());
|
||||
}
|
||||
return keys;
|
||||
}
|
||||
|
||||
public static List<HoodieKey> getHoodieKeys(List<HoodieRecord> hoodieRecords) {
|
||||
List<HoodieKey> keys = new ArrayList<>();
|
||||
for (HoodieRecord rec : hoodieRecords) {
|
||||
keys.add(rec.getKey());
|
||||
}
|
||||
return keys;
|
||||
}
|
||||
|
||||
public static List<HoodieKey> getKeysToDelete(List<HoodieKey> keys, int size) {
|
||||
List<HoodieKey> toReturn = new ArrayList<>();
|
||||
int counter = 0;
|
||||
while (counter < size) {
|
||||
int index = RANDOM.nextInt(keys.size());
|
||||
if (!toReturn.contains(keys.get(index))) {
|
||||
toReturn.add(keys.get(index));
|
||||
counter++;
|
||||
}
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
private static void fakeMetaFile(String basePath, String instantTime, String suffix) throws IOException {
|
||||
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
|
||||
new File(parentPath).mkdirs();
|
||||
|
||||
Reference in New Issue
Block a user