[HUDI-1951] Add bucket hash index, compatible with the hive bucket (#3173)
* [HUDI-2154] Add index key field to HoodieKey * [HUDI-2157] Add the bucket index and its read/write implemention of Spark engine. * revert HUDI-2154 add index key field to HoodieKey * fix all comments and introduce a new tricky way to get index key at runtime support double insert for bucket index * revert spark read optimizer based on bucket index * add the storage layout * index tag, hash function and add ut * fix ut * address partial comments * Code review feedback * add layout config and docs * fix ut * rename hoodie.layout and rebase master Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -36,6 +36,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieLayoutConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -44,6 +45,7 @@ import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner;
|
||||
import org.apache.hudi.testutils.Assertions;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
|
||||
@@ -89,7 +91,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
{IndexType.SIMPLE, true},
|
||||
{IndexType.GLOBAL_SIMPLE, true},
|
||||
{IndexType.SIMPLE, false},
|
||||
{IndexType.GLOBAL_SIMPLE, false}
|
||||
{IndexType.GLOBAL_SIMPLE, false},
|
||||
{IndexType.BUCKET, false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
@@ -112,11 +115,16 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
initFileSystem();
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, populateMetaFields ? new Properties()
|
||||
: getPropertiesForKeyGen());
|
||||
HoodieIndexConfig.Builder indexBuilder = HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.fromProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
|
||||
.withIndexType(indexType);
|
||||
config = getConfigBuilder()
|
||||
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
|
||||
.withRollbackUsingMarkers(rollbackUsingMarkers)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build();
|
||||
.withIndexConfig(indexBuilder
|
||||
.build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build())
|
||||
.withLayoutConfig(HoodieLayoutConfig.newBuilder().fromProperties(indexBuilder.build().getProps())
|
||||
.withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build()).build();
|
||||
writeClient = getHoodieWriteClient(config);
|
||||
this.index = writeClient.getIndex();
|
||||
}
|
||||
@@ -239,7 +247,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
@@ -249,7 +257,6 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords);
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0);
|
||||
List<String> taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect();
|
||||
|
||||
@@ -474,7 +481,6 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
@@ -32,6 +32,7 @@ import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.bucket.HoodieBucketIndex;
|
||||
import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
|
||||
import org.apache.hudi.index.inmemory.HoodieInMemoryHashIndex;
|
||||
import org.apache.hudi.index.simple.HoodieSimpleIndex;
|
||||
@@ -60,7 +61,7 @@ public class TestHoodieIndexConfigs {
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE"})
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE", "BUCKET"})
|
||||
public void testCreateIndex(IndexType indexType) throws Exception {
|
||||
HoodieWriteConfig config;
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
@@ -93,6 +94,11 @@ public class TestHoodieIndexConfigs {
|
||||
.build();
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof SparkHoodieHBaseIndex);
|
||||
break;
|
||||
case BUCKET:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.BUCKET).build()).build();
|
||||
assertTrue(SparkHoodieIndexFactory.createIndex(config) instanceof HoodieBucketIndex);
|
||||
break;
|
||||
default:
|
||||
// no -op. just for checkstyle errors
|
||||
}
|
||||
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bucket;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.keygen.KeyGenUtils;
|
||||
import org.apache.hudi.testutils.KeyGeneratorTestUtilities;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
public class TestBucketIdentifier {
|
||||
|
||||
@Test
|
||||
public void testBucketFileId() {
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
String bucketId = BucketIdentifier.bucketIdStr(i);
|
||||
String fileId = BucketIdentifier.newBucketFileIdPrefix(bucketId);
|
||||
assert BucketIdentifier.bucketIdFromFileId(fileId) == i;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBucketIdWithSimpleRecordKey() {
|
||||
String recordKeyField = "_row_key";
|
||||
String indexKeyField = "_row_key";
|
||||
GenericRecord record = KeyGeneratorTestUtilities.getRecord();
|
||||
HoodieRecord hoodieRecord = new HoodieRecord(
|
||||
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null);
|
||||
int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8);
|
||||
assert bucketId == BucketIdentifier.getBucketId(
|
||||
Arrays.asList(record.get(indexKeyField).toString()), 8);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBucketIdWithComplexRecordKey() {
|
||||
List<String> recordKeyField = Arrays.asList("_row_key","ts_ms");
|
||||
String indexKeyField = "_row_key";
|
||||
GenericRecord record = KeyGeneratorTestUtilities.getRecord();
|
||||
HoodieRecord hoodieRecord = new HoodieRecord(
|
||||
new HoodieKey(KeyGenUtils.getRecordKey(record, recordKeyField), ""), null);
|
||||
int bucketId = BucketIdentifier.getBucketId(hoodieRecord, indexKeyField, 8);
|
||||
assert bucketId == BucketIdentifier.getBucketId(
|
||||
Arrays.asList(record.get(indexKeyField).toString()), 8);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,147 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bucket;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.data.HoodieJavaRDD;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieBucketIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestHoodieBucketIndex.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBucketIndex.class, "/exampleSchema.avsc", true);
|
||||
private static final int NUM_BUCKET = 8;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initFileSystem();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBucketIndexValidityCheck() {
|
||||
Properties props = new Properties();
|
||||
props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "_row_key");
|
||||
assertThrows(HoodieIndexException.class, () -> {
|
||||
HoodieIndexConfig.newBuilder().fromProperties(props)
|
||||
.withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build();
|
||||
});
|
||||
props.setProperty(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key(), "uuid");
|
||||
HoodieIndexConfig.newBuilder().fromProperties(props)
|
||||
.withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("8").build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocation() throws Exception {
|
||||
String rowKey1 = UUID.randomUUID().toString();
|
||||
String rowKey2 = UUID.randomUUID().toString();
|
||||
String rowKey3 = UUID.randomUUID().toString();
|
||||
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 = new HoodieRecord(
|
||||
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 = new HoodieRecord(
|
||||
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 = new HoodieRecord(
|
||||
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 = new HoodieRecord(
|
||||
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
HoodieWriteConfig config = makeConfig();
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieBucketIndex bucketIndex = new HoodieBucketIndex(config);
|
||||
HoodieData<HoodieRecord> taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context, table);
|
||||
assertFalse(taggedRecordRDD.collectAsList().stream().anyMatch(r -> r.isCurrentLocationKnown()));
|
||||
|
||||
HoodieSparkWriteableTestTable testTable = HoodieSparkWriteableTestTable.of(table, SCHEMA);
|
||||
testTable.addCommit("001").withInserts("2016/01/31", getRecordFileId(record1), record1);
|
||||
testTable.addCommit("002").withInserts("2016/01/31", getRecordFileId(record2), record2);
|
||||
testTable.addCommit("003").withInserts("2016/01/31", getRecordFileId(record3), record3);
|
||||
taggedRecordRDD = bucketIndex.tagLocation(HoodieJavaRDD.of(recordRDD), context,
|
||||
HoodieSparkTable.create(config, context, metaClient));
|
||||
assertFalse(taggedRecordRDD.collectAsList().stream().filter(r -> r.isCurrentLocationKnown())
|
||||
.filter(r -> BucketIdentifier.bucketIdFromFileId(r.getCurrentLocation().getFileId())
|
||||
!= getRecordBucketId(r)).findAny().isPresent());
|
||||
assertTrue(taggedRecordRDD.collectAsList().stream().filter(r -> r.getPartitionPath().equals("2015/01/31")
|
||||
&& !r.isCurrentLocationKnown()).count() == 1L);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeConfig() {
|
||||
Properties props = new Properties();
|
||||
props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString())
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().fromProperties(props)
|
||||
.withIndexType(HoodieIndex.IndexType.BUCKET)
|
||||
.withIndexKeyField("_row_key")
|
||||
.withBucketNum(String.valueOf(NUM_BUCKET)).build()).build();
|
||||
}
|
||||
|
||||
private String getRecordFileId(HoodieRecord record) {
|
||||
return BucketIdentifier.bucketIdStr(
|
||||
BucketIdentifier.getBucketId(record, "_row_key", NUM_BUCKET));
|
||||
}
|
||||
|
||||
private int getRecordBucketId(HoodieRecord record) {
|
||||
return BucketIdentifier
|
||||
.getBucketId(record, "_row_key", NUM_BUCKET);
|
||||
}
|
||||
}
|
||||
@@ -33,14 +33,19 @@ import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieLayoutConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.storage.HoodieStorageLayout;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
@@ -58,6 +63,8 @@ import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
@@ -67,7 +74,9 @@ import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
|
||||
@@ -83,6 +92,13 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.avsc");
|
||||
private static final Stream<Arguments> indexType() {
|
||||
HoodieIndex.IndexType[] data = new HoodieIndex.IndexType[] {
|
||||
HoodieIndex.IndexType.BLOOM,
|
||||
HoodieIndex.IndexType.BUCKET
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMakeNewPath() {
|
||||
@@ -118,11 +134,29 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
.withRemoteServerPort(timelineServicePort).build());
|
||||
}
|
||||
|
||||
private Properties makeIndexConfig(HoodieIndex.IndexType indexType) {
|
||||
Properties props = new Properties();
|
||||
HoodieIndexConfig.Builder indexConfig = HoodieIndexConfig.newBuilder()
|
||||
.withIndexType(indexType);
|
||||
props.putAll(indexConfig.build().getProps());
|
||||
if (indexType.equals(HoodieIndex.IndexType.BUCKET)) {
|
||||
props.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
|
||||
indexConfig.fromProperties(props).withIndexKeyField("_row_key").withBucketNum("1");
|
||||
props.putAll(indexConfig.build().getProps());
|
||||
props.putAll(HoodieLayoutConfig.newBuilder().fromProperties(props)
|
||||
.withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name())
|
||||
.withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build().getProps());
|
||||
}
|
||||
return props;
|
||||
}
|
||||
|
||||
// TODO (weiy): Add testcases for crossing file writing.
|
||||
@Test
|
||||
public void testUpdateRecords() throws Exception {
|
||||
@ParameterizedTest
|
||||
@MethodSource("indexType")
|
||||
public void testUpdateRecords(HoodieIndex.IndexType indexType) throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withProps(makeIndexConfig(indexType)).build();
|
||||
String firstCommitTime = makeNewCommitTime();
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(firstCommitTime);
|
||||
@@ -168,7 +202,6 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
GenericRecord newRecord;
|
||||
int index = 0;
|
||||
for (GenericRecord record : fileRecords) {
|
||||
//System.out.println("Got :" + record.get("_row_key").toString() + ", Exp :" + records.get(index).getRecordKey());
|
||||
assertEquals(records.get(index).getRecordKey(), record.get("_row_key").toString());
|
||||
index++;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user