the-other-tim-brown commented on code in PR #18042:
URL: https://github.com/apache/hudi/pull/18042#discussion_r2743885514
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderOnSpark.scala:
##########
@@ -110,7 +110,8 @@ class TestHoodieFileGroupReaderOnSpark extends
TestHoodieFileGroupReaderBase[Int
val parquetReader = sparkAdapter.createParquetFileReader(vectorized =
false, spark.sessionState.conf, Map.empty,
storageConf.unwrapAs(classOf[Configuration]))
val dataSchema =
HoodieSchemaConversionUtils.convertHoodieSchemaToStructType(schema)
val orcReader = sparkAdapter.createOrcFileReader(vectorized = false,
spark.sessionState.conf, Map.empty,
storageConf.unwrapAs(classOf[Configuration]), dataSchema)
- val multiFormatReader = new
MultipleColumnarFileFormatReader(parquetReader, orcReader)
+ val lanceReader = sparkAdapter.createLanceFileReader(vectorized = false,
spark.sessionState.conf, Map.empty,
storageConf.unwrapAs(classOf[Configuration]))
Review Comment:
Should this also have a check like `HoodieSparkUtils.gteqSpark3_4`?
##########
hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java:
##########
@@ -76,25 +76,35 @@ public ClosableIterator<HoodieKey>
getHoodieKeyIterator(HoodieStorage storage,
Option<BaseKeyGenerator> keyGeneratorOpt,
Option<String>
partitionPath) {
try {
+ HoodieSchema keySchema = getKeyIteratorSchema(storage, filePath,
keyGeneratorOpt, partitionPath);
HoodieFileReader reader = HoodieIOFactory.getIOFactory(storage)
.getReaderFactory(HoodieRecord.HoodieRecordType.SPARK)
.getFileReader(new HoodieReaderConfig(), filePath,
HoodieFileFormat.LANCE);
- ClosableIterator<String> keyIterator = reader.getRecordKeyIterator();
+ ClosableIterator<HoodieRecord> recordIterator =
reader.getRecordIterator(keySchema);
Review Comment:
You can use `CloseableMappingIterator` to simplify the setup here
##########
hudi-common/src/main/java/org/apache/hudi/common/util/LanceUtils.java:
##########
@@ -76,25 +76,35 @@ public ClosableIterator<HoodieKey>
getHoodieKeyIterator(HoodieStorage storage,
Option<BaseKeyGenerator> keyGeneratorOpt,
Option<String>
partitionPath) {
try {
+ HoodieSchema keySchema = getKeyIteratorSchema(storage, filePath,
keyGeneratorOpt, partitionPath);
HoodieFileReader reader = HoodieIOFactory.getIOFactory(storage)
.getReaderFactory(HoodieRecord.HoodieRecordType.SPARK)
.getFileReader(new HoodieReaderConfig(), filePath,
HoodieFileFormat.LANCE);
- ClosableIterator<String> keyIterator = reader.getRecordKeyIterator();
+ ClosableIterator<HoodieRecord> recordIterator =
reader.getRecordIterator(keySchema);
+
return new ClosableIterator<HoodieKey>() {
@Override
public void close() {
- keyIterator.close();
+ recordIterator.close();
}
@Override
public boolean hasNext() {
- return keyIterator.hasNext();
+ return recordIterator.hasNext();
}
@Override
public HoodieKey next() {
- String key = keyIterator.next();
- return new HoodieKey(key, partitionPath.orElse(null));
+ HoodieRecord record = recordIterator.next();
+ String recordKey;
+ if (keyGeneratorOpt.isPresent()) {
+ // With keyGenerator, extracts user-defined key fields by name
+ recordKey = record.getRecordKey(keySchema, keyGeneratorOpt);
+ } else {
+ // Without keyGenerator, read record key field by name
Review Comment:
You'll want to construct the full key if the partition path is set. Check
out `HoodieKeyIterator` for an example
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]