[HUDI-2656] Generalize HoodieIndex for flexible record data type (#3893)
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -18,16 +18,11 @@
|
||||
|
||||
package org.apache.hudi.table.action.commit;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.client.HoodieJavaWriteClient;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -40,6 +35,13 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
@@ -111,8 +113,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase {
|
||||
List<HoodieRecord> records1 = new ArrayList<>();
|
||||
RawTripTestPayload insertRow1 = new RawTripTestPayload(insertRecordStr1);
|
||||
RawTripTestPayload insertRow2 = new RawTripTestPayload(insertRecordStr2);
|
||||
records1.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records1.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
records1.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records1.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
|
||||
int startInstant = 1;
|
||||
String firstCommitTime = makeNewCommitTime(startInstant++);
|
||||
@@ -140,8 +142,8 @@ public class TestHoodieConcatHandle extends HoodieJavaClientTestBase {
|
||||
insertRow1 = new RawTripTestPayload(insertRecordStr1);
|
||||
insertRow2 = new RawTripTestPayload(insertRecordStr2);
|
||||
// The recordKey of records2 and records1 are the same, but the values of other fields are different
|
||||
records2.add(new HoodieRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records2.add(new HoodieRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
records2.add(new HoodieAvroRecord(new HoodieKey(insertRow1.getRowKey(), insertRow1.getPartitionPath()), insertRow1));
|
||||
records2.add(new HoodieAvroRecord(new HoodieKey(insertRow2.getRowKey(), insertRow2.getPartitionPath()), insertRow2));
|
||||
|
||||
String newCommitTime = makeNewCommitTime(startInstant++);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -42,6 +43,8 @@ import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.table.HoodieJavaCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieJavaTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -49,8 +52,6 @@ import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hudi.testutils.HoodieJavaClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
@@ -142,11 +143,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
writeClient.insert(records, firstCommitTime);
|
||||
@@ -175,12 +176,12 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
RawTripTestPayload updateRowChanges1 = new RawTripTestPayload(updateRecordStr1);
|
||||
HoodieRecord updatedRecord1 = new HoodieRecord(
|
||||
HoodieRecord updatedRecord1 = new HoodieAvroRecord(
|
||||
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
|
||||
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord insertedRecord1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
new HoodieAvroRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
|
||||
|
||||
@@ -254,7 +255,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String recordStr =
|
||||
String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", UUID.randomUUID().toString(), time, i);
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
@@ -280,11 +281,11 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
BaseJavaCommitActionExecutor actionExecutor = new JavaInsertCommitActionExecutor(context, config, table,
|
||||
@@ -382,7 +383,7 @@ public class TestJavaCopyOnWriteActionExecutor extends HoodieJavaClientTestBase
|
||||
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
|
||||
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
records.add(new HoodieAvroRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
|
||||
// Insert new records
|
||||
|
||||
Reference in New Issue
Block a user