alexeykudinkin commented on code in PR #6782: URL: https://github.com/apache/hudi/pull/6782#discussion_r1072896769
########## hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java: ########## @@ -671,11 +658,188 @@ public void testBasicAppendAndScanMultipleFiles(ExternalSpillableMap.DiskMapType ((HoodieAvroRecord) record).getData().getInsertValue(schema).get()); } - assertEquals(scannedRecords.size(), allRecords.stream().mapToLong(Collection::size).sum(), + assertEquals(sort(genRecords), sort(scannedRecords), "Scanner records count should be the same as appended records"); scanner.close(); } + @ParameterizedTest + @MethodSource("testArguments") + public void testBasicAppendAndPartialScanning(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled, + boolean readBlocksLazily, + boolean useScanV2) + throws IOException, URISyntaxException, InterruptedException { + // Generate 3 delta-log files w/ random records + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + List<IndexedRecord> genRecords = SchemaTestUtil.generateHoodieTestRecords(0, 300); + + Set<HoodieLogFile> logFiles = writeLogFiles(partitionPath, schema, genRecords, 3); + + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + + // scan all log blocks (across multiple log files) + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths( + logFiles.stream() + .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList())) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(1024L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(BUFFER_SIZE) + .withSpillableMapBasePath(spillableBasePath) + .withDiskMapType(diskMapType) + .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanV2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) + .withForceFullScan(false) + .build(); + + List<String> sampledRecordKeys = Arrays.asList( + "b190b1fb-392b-4ceb-932d-a72c906127c2", + "409e9ad3-5def-45e7-9180-ef579c1c220b", + "e6b31f1c-60a8-4577-acf5-7e8ea318b08b", + "0c477a9e-e602-4642-8e96-1cfd357b4ba0", + "ea076c17-32ae-4659-8caf-6ad538b4dd8d", + "7a943e09-3856-4874-83a1-8ee93e158f94", + "9cbff584-d8a4-4b05-868b-dc917d6cf841", + "bda0b0d8-0c56-43b0-89f9-e090d924586b", + "ee118fb3-69cb-4705-a8c4-88a18e8aa1b7", + "cb1fbe4d-06c3-4c9c-aea7-2665ffa8b205" + ); + + List<IndexedRecord> sampledRecords = genRecords.stream() + .filter(r -> sampledRecordKeys.contains(((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())) + .collect(Collectors.toList()); + + // + // Step 1: Scan by a list of keys + // + + scanner.scanByFullKeys(sampledRecordKeys); + + List<HoodieRecord> scannedHoodieRecords = new ArrayList<>(); + List<IndexedRecord> scannedAvroRecords = new ArrayList<>(); + for (HoodieRecord record : scanner) { + scannedHoodieRecords.add(record); + scannedAvroRecords.add((IndexedRecord) + ((HoodieAvroRecord) record).getData().getInsertValue(schema).get()); + } + + assertEquals(sort(sampledRecords), sort(scannedAvroRecords)); + + // + // Step 2: Scan by the same list of keys (no new scanning should be performed, + // in this case, and same _objects_ have to be returned) + // + + scanner.scanByFullKeys(sampledRecordKeys); + + List<HoodieRecord> newScannedHoodieRecords = new ArrayList<>(); + for (HoodieRecord record : scanner) { + newScannedHoodieRecords.add(record); + } + + assertEquals(scannedHoodieRecords.size(), newScannedHoodieRecords.size()); + + for (int i = 0; i < scannedHoodieRecords.size(); ++i) { + assertSame(scannedHoodieRecords.get(i), newScannedHoodieRecords.get(i), "Objects have to be identical"); + } + + scanner.close(); + } + + @ParameterizedTest + @MethodSource("testArguments") + public void testBasicAppendAndPartialScanningByKeyPrefixes(ExternalSpillableMap.DiskMapType diskMapType, + boolean isCompressionEnabled, + boolean readBlocksLazily, + boolean useScanV2) + throws IOException, URISyntaxException, InterruptedException { + // Generate 3 delta-log files w/ random records + Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema()); + List<IndexedRecord> genRecords = SchemaTestUtil.generateHoodieTestRecords(0, 300); + + Set<HoodieLogFile> logFiles = writeLogFiles(partitionPath, schema, genRecords, 3); + + FileCreateUtils.createDeltaCommit(basePath, "100", fs); + + // scan all log blocks (across multiple log files) + HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder() + .withFileSystem(fs) + .withBasePath(basePath) + .withLogFilePaths( + logFiles.stream() + .map(logFile -> logFile.getPath().toString()).collect(Collectors.toList())) + .withReaderSchema(schema) + .withLatestInstantTime("100") + .withMaxMemorySizeInBytes(1024L) + .withReadBlocksLazily(readBlocksLazily) + .withReverseReader(false) + .withBufferSize(BUFFER_SIZE) + .withSpillableMapBasePath(spillableBasePath) + .withDiskMapType(diskMapType) + .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled) + .withUseScanV2(useScanV2) + .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName())) + .withForceFullScan(false) + .build(); + + + List<String> sampledRecordKeys = Arrays.asList( + "00509b14-3d1a-4283-9a8c-c72b971a9d06", + "006b2f57-9bf7-4634-910c-c91542ea61e5", + "007fc45d-7ce2-45be-8765-0b9082412518", + "00826e50-73b4-4cb0-9d5a-375554d5e0f7" + ); + + List<IndexedRecord> sampledRecords = genRecords.stream() + .filter(r -> sampledRecordKeys.contains(((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString())) + .collect(Collectors.toList()); + + List<String> sampledKeyPrefixes = Collections.singletonList("00"); + + // + // Step 1: Scan by a list of keys + // + + scanner.scanByKeyPrefixes(sampledKeyPrefixes); + + List<HoodieRecord> scannedHoodieRecords = new ArrayList<>(); + List<IndexedRecord> scannedAvroRecords = new ArrayList<>(); + for (HoodieRecord record : scanner) { + scannedHoodieRecords.add(record); + scannedAvroRecords.add((IndexedRecord) + ((HoodieAvroRecord) record).getData().getInsertValue(schema).get()); + } + + //assertEquals(sort(sampledRecords), sort(scannedAvroRecords)); Review Comment: Need to address ########## hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java: ########## @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.metadata; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hudi.common.model.HoodieAvroRecordMerger; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; +import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ExternalSpillableMap; + +import javax.annotation.concurrent.ThreadSafe; +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is + * useful in limiting memory usage when only a small subset of updates records are to be read. + */ +@ThreadSafe +public class HoodieMetadataLogRecordReader implements Closeable { + + private final HoodieMergedLogRecordScanner logRecordScanner; + + private HoodieMetadataLogRecordReader(HoodieMergedLogRecordScanner logRecordScanner) { + this.logRecordScanner = logRecordScanner; + } + + /** + * Returns the builder for {@code HoodieMetadataMergedLogRecordScanner}. + */ + public static HoodieMetadataLogRecordReader.Builder newBuilder() { + return new HoodieMetadataLogRecordReader.Builder(); + } + + @SuppressWarnings("unchecked") + public List<HoodieRecord<HoodieMetadataPayload>> getRecords() { + // NOTE: Locking is necessary since we're accessing [[HoodieMetadataLogRecordReader]] + // materialized state, to make sure there's no concurrent access + synchronized (this) { + logRecordScanner.scan(); Review Comment: Correct -- 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: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org