codope commented on code in PR #6782:
URL: https://github.com/apache/hudi/pull/6782#discussion_r1054076811
##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -188,40 +179,41 @@ protected AbstractHoodieLogRecordReader(FileSystem fs,
String basePath, List<Str
this.path = basePath;
this.useScanV2 = useScanV2;
- // Key fields when populate meta fields is disabled (that is, virtual keys
enabled)
- if (!tableConfig.populateMetaFields()) {
- this.populateMetaFields = false;
- this.simpleKeyGenFields = Option.of(
- Pair.of(tableConfig.getRecordKeyFieldProp(),
tableConfig.getPartitionFieldProp()));
- }
- this.partitionName = partitionName;
- }
+ if (keyFieldOverride.isPresent()) {
+ // TODO elaborate
+ checkState(partitionNameOverride.isPresent());
- protected String getKeyField() {
- if (this.populateMetaFields) {
- return HoodieRecord.RECORD_KEY_METADATA_FIELD;
+ this.populateMetaFields = false;
+ this.recordKeyField = keyFieldOverride.get();
+ this.partitionPathFieldOpt = Option.empty();
+ } else if (tableConfig.populateMetaFields()) {
+ this.populateMetaFields = true;
+ this.recordKeyField = HoodieRecord.RECORD_KEY_METADATA_FIELD;
+ this.partitionPathFieldOpt =
Option.of(HoodieRecord.PARTITION_PATH_METADATA_FIELD);
Review Comment:
makes sense. let's add a comment before
`checkState(partitionNameOverride.isPresent());`
##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataLogRecordReader.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.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();
+ return logRecordScanner.getRecords().values()
+ .stream()
+ .map(record -> (HoodieRecord<HoodieMetadataPayload>) record)
+ .collect(Collectors.toList());
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public List<HoodieRecord<HoodieMetadataPayload>>
getRecordsByKeyPrefixes(List<String> keyPrefixes) {
+ if (keyPrefixes.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ // NOTE: Locking is necessary since we're accessing
[[HoodieMetadataLogRecordReader]]
+ // materialized state, to make sure there's no concurrent access
+ synchronized (this) {
+ logRecordScanner.scanByKeyPrefixes(keyPrefixes);
+ Map<String, HoodieRecord> allRecords = logRecordScanner.getRecords();
+
+ Predicate<String> p = createPrefixMatchingPredicate(keyPrefixes);
+ return allRecords.entrySet()
+ .stream()
+ .filter(r -> r != null && p.test(r.getKey()))
+ .map(r -> (HoodieRecord<HoodieMetadataPayload>) r.getValue())
+ .collect(Collectors.toList());
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public List<HoodieRecord<HoodieMetadataPayload>>
getRecordsByKeys(List<String> keys) {
+ if (keys.isEmpty()) {
+ return Collections.emptyList();
+ }
+
+ // NOTE: Locking is necessary since we're accessing
[[HoodieMetadataLogRecordReader]]
+ // materialized state, to make sure there's no concurrent access
+ synchronized (this) {
+ logRecordScanner.scanByFullKeys(keys);
+ Map<String, HoodieRecord> allRecords = logRecordScanner.getRecords();
+ return keys.stream()
+ .map(key -> (HoodieRecord<HoodieMetadataPayload>)
allRecords.get(key))
+ .filter(Objects::nonNull)
+ .collect(Collectors.toList());
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ logRecordScanner.close();
+ }
+
+ private static Predicate<String> createPrefixMatchingPredicate(List<String>
keyPrefixes) {
+ if (keyPrefixes.size() == 1) {
+ String keyPrefix = keyPrefixes.get(0);
+ return key -> key.startsWith(keyPrefix);
+ }
+
+ return key -> keyPrefixes.stream().anyMatch(key::startsWith);
+ }
+
+ /**
+ * Builder used to build {@code HoodieMetadataMergedLogRecordScanner}.
+ */
+ public static class Builder {
+ private final HoodieMergedLogRecordScanner.Builder scannerBuilder =
+ new HoodieMergedLogRecordScanner.Builder()
+ .withKeyFiledOverride(HoodieMetadataPayload.KEY_FIELD_NAME)
+ .withReadBlocksLazily(true)
+ .withReverseReader(false)
+ .withOperationField(false);
+
+ public Builder withFileSystem(FileSystem fs) {
+ scannerBuilder.withFileSystem(fs);
+ return this;
+ }
+
+ public Builder withBasePath(String basePath) {
+ scannerBuilder.withBasePath(basePath);
+ return this;
+ }
+
+ public Builder withLogFilePaths(List<String> logFilePaths) {
+ scannerBuilder.withLogFilePaths(logFilePaths);
+ return this;
+ }
+
+ public Builder withReaderSchema(Schema schema) {
+ scannerBuilder.withReaderSchema(schema);
+ return this;
+ }
+
+ public Builder withLatestInstantTime(String latestInstantTime) {
+ scannerBuilder.withLatestInstantTime(latestInstantTime);
+ return this;
+ }
+
+ public Builder withBufferSize(int bufferSize) {
+ scannerBuilder.withBufferSize(bufferSize);
+ return this;
+ }
+
+ public Builder withPartition(String partitionName) {
+ scannerBuilder.withPartition(partitionName);
+ return this;
+ }
+
+ public Builder withMaxMemorySizeInBytes(Long maxMemorySizeInBytes) {
+ scannerBuilder.withMaxMemorySizeInBytes(maxMemorySizeInBytes);
+ return this;
+ }
+
+ public Builder withSpillableMapBasePath(String spillableMapBasePath) {
+ scannerBuilder.withSpillableMapBasePath(spillableMapBasePath);
+ return this;
+ }
+
+ public Builder withDiskMapType(ExternalSpillableMap.DiskMapType
diskMapType) {
+ scannerBuilder.withDiskMapType(diskMapType);
+ return this;
+ }
+
+ public Builder withBitCaskDiskMapCompressionEnabled(boolean
isBitCaskDiskMapCompressionEnabled) {
+
scannerBuilder.withBitCaskDiskMapCompressionEnabled(isBitCaskDiskMapCompressionEnabled);
+ return this;
+ }
+
+ public Builder withLogBlockTimestamps(Set<String> validLogBlockTimestamps)
{
+ scannerBuilder.withInstantRange(Option.of(new
ExplicitMatchRange(validLogBlockTimestamps)));
+ return this;
+ }
+
+ public Builder enableFullScan(boolean enableFullScan) {
+ scannerBuilder.withForceFullScan(enableFullScan);
+ return this;
+ }
+
+ public Builder withUseScanV2(boolean useScanV2) {
+ scannerBuilder.withUseScanV2(useScanV2);
+ return this;
+ }
+
+ public HoodieMetadataLogRecordReader build() {
+ return new HoodieMetadataLogRecordReader(scannerBuilder.build());
Review Comment:
Log record scanner depends on the record merger impl. Don't we also need to
specify the record merger? The constructor in the dropped class (which used to
extend HoodieMergedLogRecordScanner) did that:
https://github.com/apache/hudi/blob/fd62a1413e74de686935672aec812aacd5c43a63/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java#L63
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/Iterators.scala:
##########
@@ -261,7 +253,7 @@ object LogFileIterator {
tableState: HoodieTableState,
maxCompactionMemoryInBytes: Long,
hadoopConf: Configuration,
- internalSchema: InternalSchema =
InternalSchema.getEmptyInternalSchema): HoodieMergedLogRecordScanner = {
+ internalSchema: InternalSchema =
InternalSchema.getEmptyInternalSchema): mutable.Map[String, HoodieRecord[_ <:
HoodieRecordPayload[_ <: HoodieRecordPayload[_ <: AnyRef]]]] = {
Review Comment:
This does not compile. We can simply return `mutable.Map[String,
HoodieRecord[_]]`
##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -106,30 +109,85 @@ protected HoodieMergedLogRecordScanner(FileSystem fs,
String basePath, List<Stri
}
}
- protected void performScan() {
+ /**
+ * Scans delta-log files processing blocks
+ */
+ public final void scan() {
+ scan(false);
+ }
+
+ public final void scan(boolean skipProcessingBlocks) {
+ scanInternal(Option.empty(), skipProcessingBlocks);
+ }
+
+ /**
+ * Provides incremental scanning capability where only provided keys will be
looked
+ * up in the delta-log files, scanned and subsequently materialized into the
internal
+ * cache
+ *
+ * @param keys to be looked up
+ */
+ public void scanByFullKeys(List<String> keys) {
+ if (forceFullScan) {
+ return; // no-op
+ }
+
+ List<String> missingKeys = keys.stream()
+ .filter(key -> !records.containsKey(key))
+ .collect(Collectors.toList());
+
+ if (missingKeys.isEmpty()) {
+ // All the required records are already fetched, no-op
+ return;
+ }
+
+ scanInternal(Option.of(KeySpec.fullKeySpec(missingKeys)), false);
+ }
+
+ /**
+ * Provides incremental scanning capability where only keys matching
provided key-prefixes
+ * will be looked up in the delta-log files, scanned and subsequently
materialized into
+ * the internal cache
+ *
+ * @param keyPrefixes to be looked up
+ */
+ public void scanByKeyPrefixes(List<String> keyPrefixes) {
+ // TODO add caching for queried prefixes
Review Comment:
This seems like it deserves a separate PR. All the queued log blocks need to
be cached, isn't it?
--
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]