Skip to content

Commit

Permalink
Use Hashset and share the key set
Browse files Browse the repository at this point in the history
  • Loading branch information
linliu-code committed Dec 17, 2024
1 parent f2fb700 commit 8f39643
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 9 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,7 @@ public void testRepeatedCleanActionsWithMetadataTableEnabled(final HoodieTableTy
@Test
void testReverseLookupSecondaryKeysInternalWithOnlyBaseFileRecord() {
String recordKey = "recordKey";
List<String> recordKeys = Collections.singletonList(recordKey);
Set<String> recordKeys = Collections.singleton(recordKey);
Map<String, HoodieRecord<HoodieMetadataPayload>> baseFileRecords = new HashMap<>();
HoodieMetadataLogRecordReader logRecordReader = mock(HoodieMetadataLogRecordReader.class);
List<HoodieRecord<HoodieMetadataPayload>> logRecords = new ArrayList<>();
Expand All @@ -391,7 +391,7 @@ void testReverseLookupSecondaryKeysInternalWithOnlyBaseFileRecord() {
@Test
void testReverseLookupSecondaryKeysInternalWithOnlyLogRecords() {
String recordKey = "recordKey";
List<String> recordKeys = Collections.singletonList(recordKey);
Set<String> recordKeys = Collections.singleton(recordKey);
Map<String, HoodieRecord<HoodieMetadataPayload>> baseFileRecords = new HashMap<>();
HoodieMetadataLogRecordReader logRecordReader = mock(HoodieMetadataLogRecordReader.class);
List<HoodieRecord<HoodieMetadataPayload>> logRecords = new ArrayList<>();
Expand Down Expand Up @@ -427,7 +427,7 @@ void testReverseLookupSecondaryKeysInternalWithOnlyLogRecords() {
@Test
void testReverseLookupSecondaryKeysInternal() {
String recordKey = "recordKey";
List<String> recordKeys = Collections.singletonList(recordKey);
Set<String> recordKeys = Collections.singleton(recordKey);
Map<String, HoodieRecord<HoodieMetadataPayload>> baseFileRecords = new HashMap<>();
HoodieMetadataLogRecordReader logRecordReader = mock(HoodieMetadataLogRecordReader.class);
List<HoodieRecord<HoodieMetadataPayload>> logRecords = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,6 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import java.util.stream.Stream;
Expand Down Expand Up @@ -834,11 +833,11 @@ private Map<String, String> reverseLookupSecondaryKeys(String partitionName, Lis
if (baseFileReader == null && logRecordScanner == null) {
return Collections.emptyMap();
}
Set<String> keySet = new TreeSet<>(recordKeys);
Set<String> keySet = new HashSet<>(recordKeys);
Map<String, HoodieRecord<HoodieMetadataPayload>> baseFileRecords =
fetchBaseFileAllRecordsByPayloadForSecIndex(baseFileReader, keySet, partitionName);

return reverseLookupSecondaryKeysInternal(recordKeys, baseFileRecords, logRecordScanner);
return reverseLookupSecondaryKeysInternal(keySet, baseFileRecords, logRecordScanner);
} catch (IOException ioe) {
throw new HoodieIOException("Error merging records from metadata table for " + recordKeys.size() + " key : ", ioe);
} finally {
Expand Down Expand Up @@ -888,10 +887,9 @@ private Map<String, HoodieRecord<HoodieMetadataPayload>> fetchBaseFileAllRecords
}

@VisibleForTesting
public static Map<String, String> reverseLookupSecondaryKeysInternal(List<String> recordKeys,
public static Map<String, String> reverseLookupSecondaryKeysInternal(Set<String> recordKeySet,
Map<String, HoodieRecord<HoodieMetadataPayload>> baseFileRecords,
HoodieMetadataLogRecordReader logRecordScanner) {
Set<String> keySet = new HashSet<>(recordKeys);
Set<String> deletedRecordsFromLogs = new HashSet<>();
Map<String, HoodieRecord<HoodieMetadataPayload>> logRecordsMap = new HashMap<>();
// Note that: we read the log records from the oldest to the latest!!!
Expand All @@ -900,7 +898,7 @@ public static Map<String, String> reverseLookupSecondaryKeysInternal(List<String
String recordKey = SecondaryIndexKeyUtils.getRecordKeyFromSecondaryIndexKey(record.getRecordKey());
HoodieMetadataPayload payload = record.getData();
if (!payload.isDeleted()) { // process only valid records.
if (keySet.contains(recordKey)) {
if (recordKeySet.contains(recordKey)) {
logRecordsMap.put(recordKey, record);
deletedRecordsFromLogs.remove(recordKey);
}
Expand Down

0 comments on commit 8f39643

Please sign in to comment.