From f85cd9b16d2bd0c49fb6a05a4be627dffb4c2065 Mon Sep 17 00:00:00 2001 From: Sivabalan Narayanan Date: Tue, 7 Jun 2022 08:19:52 -0400 Subject: [PATCH] [HUDI-4200] Fixing sorting of keys fetched from metadata table (#5773) - Key fetched from metadata table especially from base file reader is not sorted. and hence may result in throwing NPE (key prefix search) or unnecessary seeks to starting of Hfile (full key look ups). Fixing the same in this patch. This is not an issue with log blocks, since sorting is taking care within HoodieHfileDataBlock. - Commit where the sorting was mistakenly reverted [HUDI-3760] Adding capability to fetch Metadata Records by prefix #5208 --- .../storage/TestHoodieHFileReaderWriter.java | 14 ++++++++++---- .../hudi/io/storage/HoodieHFileReader.java | 8 +++----- .../metadata/HoodieBackedTableMetadata.java | 19 +++++++++++++------ .../functional/TestColumnStatsIndex.scala | 2 +- 4 files changed, 27 insertions(+), 16 deletions(-) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java index baede154c..a45b8a9aa 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java @@ -50,6 +50,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -316,15 +317,20 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase { assertEquals(expectedKey50and0s, recordsByPrefix); // filter for "key1" and "key0" : entries from 'key10 to key19' and 'key00 to key09' should be matched. - List expectedKey1sand0s = expectedKey1s; - expectedKey1sand0s.addAll(allRecords.stream() - .filter(entry -> (entry.get("_row_key").toString()).contains("key0")) - .collect(Collectors.toList())); + List expectedKey1sand0s = allRecords.stream() + .filter(entry -> (entry.get("_row_key").toString()).contains("key1") || (entry.get("_row_key").toString()).contains("key0")) + .collect(Collectors.toList()); iterator = hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema); recordsByPrefix = StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false) .collect(Collectors.toList()); + Collections.sort(recordsByPrefix, new Comparator() { + @Override + public int compare(GenericRecord o1, GenericRecord o2) { + return o1.get("_row_key").toString().compareTo(o2.get("_row_key").toString()); + } + }); assertEquals(expectedKey1sand0s, recordsByPrefix); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java index 0bf31d2a2..3e5b3ff6a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java @@ -259,11 +259,9 @@ public class HoodieHFileReader implements HoodieFileRea return Collections.emptyIterator(); } } else if (val == -1) { - // If scanner is aleady on the top of hfile. avoid trigger seekTo again. - Option headerCell = Option.fromJavaOptional(scanner.getReader().getFirstKey()); - if (headerCell.isPresent() && !headerCell.get().equals(scanner.getCell())) { - scanner.seekTo(); - } + // Whenever val == -1 HFile reader will place the pointer right before the first record. We have to advance it to the first record + // of the file to validate whether it matches our search criteria + scanner.seekTo(); } class KeyPrefixIterator implements Iterator { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java index e8937b39d..e96889f04 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java @@ -144,6 +144,10 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { @Override public HoodieData> getRecordsByKeyPrefixes(List keyPrefixes, String partitionName) { + // Sort the columns so that keys are looked up in order + List sortedkeyPrefixes = new ArrayList<>(keyPrefixes); + Collections.sort(sortedkeyPrefixes); + // NOTE: Since we partition records to a particular file-group by full key, we will have // to scan all file-groups for all key-prefixes as each of these might contain some // records matching the key-prefix @@ -171,17 +175,17 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { boolean fullKeys = false; Map>> logRecords = - readLogRecords(logRecordScanner, keyPrefixes, fullKeys, timings); + readLogRecords(logRecordScanner, sortedkeyPrefixes, fullKeys, timings); List>>> mergedRecords = - readFromBaseAndMergeWithLogRecords(baseFileReader, keyPrefixes, fullKeys, logRecords, timings, partitionName); + readFromBaseAndMergeWithLogRecords(baseFileReader, sortedkeyPrefixes, fullKeys, logRecords, timings, partitionName); LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms", - keyPrefixes.size(), timings)); + sortedkeyPrefixes.size(), timings)); return mergedRecords.iterator(); } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + keyPrefixes.size() + " key : ", ioe); + throw new HoodieIOException("Error merging records from metadata table for " + sortedkeyPrefixes.size() + " key : ", ioe); } finally { closeReader(readers); } @@ -194,7 +198,10 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { @Override public List>>> getRecordsByKeys(List keys, String partitionName) { - Map, List> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, keys); + // Sort the columns so that keys are looked up in order + List sortedKeys = new ArrayList<>(keys); + Collections.sort(sortedKeys); + Map, List> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, sortedKeys); List>>> result = new ArrayList<>(); AtomicInteger fileSlicesKeysCount = new AtomicInteger(); partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> { @@ -219,7 +226,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata { fileSliceKeys.size(), timings)); fileSlicesKeysCount.addAndGet(fileSliceKeys.size()); } catch (IOException ioe) { - throw new HoodieIOException("Error merging records from metadata table for " + keys.size() + " key : ", ioe); + throw new HoodieIOException("Error merging records from metadata table for " + sortedKeys.size() + " key : ", ioe); } finally { if (!reuse) { close(Pair.of(partitionFileSlicePair.getLeft(), partitionFileSlicePair.getRight().getFileId())); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala index 75d3ce0b7..b982b1851 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -250,7 +250,7 @@ class TestColumnStatsIndex extends HoodieClientTestBase with ColumnStatsIndexSup { // We have to include "c1", since we sort the expected outputs by this column - val requestedColumns = Seq("c1", "c4") + val requestedColumns = Seq("c4", "c1") val partialColStatsDF = readColumnStatsIndex(spark, basePath, metadataConfig, requestedColumns) val partialTransposedColStatsDF = transposeColumnStatsIndex(spark, partialColStatsDF, requestedColumns, sourceTableSchema)