[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:
@@ -32,6 +32,7 @@ import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
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;
|
||||
@@ -510,7 +511,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
try {
|
||||
String key = gr.get("_row_key").toString();
|
||||
String pPath = p.getKey();
|
||||
return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath,
|
||||
return new HoodieAvroRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath,
|
||||
HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.common.bootstrap.FileStatusUtils;
|
||||
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieAvroRecord;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
@@ -425,7 +426,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
|
||||
try {
|
||||
String key = gr.get("_row_key").toString();
|
||||
String pPath = p.getKey();
|
||||
return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath,
|
||||
return new HoodieAvroRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath,
|
||||
HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
|
||||
Reference in New Issue
Block a user