Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,8 @@ public class DeltaConversionSource implements ConversionSource<Long> {
@Builder.Default
private final DeltaTableExtractor tableExtractor = DeltaTableExtractor.builder().build();

@Builder.Default private final boolean skipColumnStats = false;

private Optional<DeltaIncrementalChangesState> deltaIncrementalChangesState = Optional.empty();

private final SparkSession sparkSession;
Expand Down Expand Up @@ -123,7 +125,7 @@ public TableChange getTableChangeForCommit(Long versionNumber) {
fileFormat,
tableAtVersion.getPartitioningFields(),
tableAtVersion.getReadSchema().getAllFields(),
true,
!skipColumnStats,
DeltaPartitionExtractor.getInstance(),
DeltaStatsExtractor.getInstance());
addedFiles.put(dataFile.getPhysicalPath(), dataFile);
Expand Down Expand Up @@ -223,7 +225,8 @@ private void resetState(long versionToStartFrom) {
}

private List<PartitionFileGroup> getInternalDataFiles(Snapshot snapshot, InternalSchema schema) {
try (DataFileIterator fileIterator = dataFileExtractor.iterator(snapshot, schema)) {
try (DataFileIterator fileIterator =
dataFileExtractor.iterator(snapshot, schema, !skipColumnStats)) {
List<InternalDataFile> dataFiles = new ArrayList<>();
fileIterator.forEachRemaining(dataFiles::add);
return PartitionFileGroup.fromFiles(dataFiles);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,12 +31,15 @@ public class DeltaConversionSourceProvider extends ConversionSourceProvider<Long
public DeltaConversionSource getConversionSourceInstance(SourceTable sourceTable) {
SparkSession sparkSession = DeltaConversionUtils.buildSparkSession(hadoopConf);
DeltaTable deltaTable = DeltaTable.forPath(sparkSession, sourceTable.getBasePath());
boolean skipColumnStats =
DeltaSourceConfig.getSkipColumnStats(sourceTable.getAdditionalProperties(), hadoopConf);
return DeltaConversionSource.builder()
.sparkSession(sparkSession)
.tableName(sourceTable.getName())
.basePath(sourceTable.getBasePath())
.deltaTable(deltaTable)
.deltaLog(deltaTable.deltaLog())
.skipColumnStats(skipColumnStats)
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,11 @@ public DataFileIterator iterator(Snapshot deltaSnapshot, InternalSchema schema)
return new DeltaDataFileIterator(deltaSnapshot, schema, true);
}

public DataFileIterator iterator(
Snapshot deltaSnapshot, InternalSchema schema, boolean includeColumnStats) {
return new DeltaDataFileIterator(deltaSnapshot, schema, includeColumnStats);
}

public class DeltaDataFileIterator implements DataFileIterator {
private final FileFormat fileFormat;
private final List<InternalField> fields;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,12 @@ private Stream<AddFile> createAddFileAction(

private String getColumnStats(
InternalSchema schema, long recordCount, List<ColumnStat> columnStats) {
// In skip-column-stats mode source files may not have row count/column stats.
// Return null only when row count is unknown (negative sentinel).
// Explicit rowCount=0 should be persisted as numRecords=0.
if (recordCount < 0 && (columnStats == null || columnStats.isEmpty())) {
return null;
}
try {
return deltaStatsExtractor.convertStatsToDeltaFormat(schema, recordCount, columnStats);
} catch (JsonProcessingException e) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.xtable.delta;

import java.util.Properties;

import org.apache.hadoop.conf.Configuration;

/** Configuration keys for Delta source format. */
public final class DeltaSourceConfig {
public static final String SKIP_COLUMN_STATS_CONFIG = "xtable.source.skip_column_stats";

private DeltaSourceConfig() {}

public static boolean getSkipColumnStats(Properties properties, Configuration configuration) {
if (properties != null) {
String propertyValue = properties.getProperty(SKIP_COLUMN_STATS_CONFIG);
if (propertyValue != null) {
return Boolean.parseBoolean(propertyValue);
}
}
if (configuration == null) {
return false;
}
String configValue = configuration.get(SKIP_COLUMN_STATS_CONFIG);
return configValue != null && Boolean.parseBoolean(configValue);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,13 @@ public class HudiConversionSource implements ConversionSource<HoodieInstant> {
public HudiConversionSource(
HoodieTableMetaClient metaClient,
PathBasedPartitionSpecExtractor sourcePartitionSpecExtractor) {
this(metaClient, sourcePartitionSpecExtractor, false);
}

public HudiConversionSource(
HoodieTableMetaClient metaClient,
PathBasedPartitionSpecExtractor sourcePartitionSpecExtractor,
boolean skipColumnStats) {
this.metaClient = metaClient;
this.tableExtractor =
new HudiTableExtractor(new HudiSchemaExtractor(), sourcePartitionSpecExtractor);
Expand All @@ -68,7 +75,8 @@ public HudiConversionSource(
metaClient,
new PathBasedPartitionValuesExtractor(
sourcePartitionSpecExtractor.getPathToPartitionFieldFormat()),
new HudiFileStatsExtractor(metaClient));
new HudiFileStatsExtractor(metaClient),
skipColumnStats);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,9 @@ public HudiConversionSource getConversionSourceInstance(SourceTable sourceTable)
final PathBasedPartitionSpecExtractor sourcePartitionSpecExtractor =
HudiSourceConfig.fromProperties(sourceTable.getAdditionalProperties())
.loadSourcePartitionSpecExtractor();
boolean skipColumnStats =
HudiSourceConfig.getSkipColumnStats(sourceTable.getAdditionalProperties(), hadoopConf);

return new HudiConversionSource(metaClient, sourcePartitionSpecExtractor);
return new HudiConversionSource(metaClient, sourcePartitionSpecExtractor, skipColumnStats);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ public class HudiDataFileExtractor implements AutoCloseable {
private final HoodieEngineContext engineContext;
private final PathBasedPartitionValuesExtractor partitionValuesExtractor;
private final HudiFileStatsExtractor fileStatsExtractor;
private final boolean skipColumnStats;
private final HoodieMetadataConfig metadataConfig;
private final FileSystemViewManager fileSystemViewManager;
private final Path basePath;
Expand All @@ -85,6 +86,14 @@ public HudiDataFileExtractor(
HoodieTableMetaClient metaClient,
PathBasedPartitionValuesExtractor hudiPartitionValuesExtractor,
HudiFileStatsExtractor hudiFileStatsExtractor) {
this(metaClient, hudiPartitionValuesExtractor, hudiFileStatsExtractor, false);
}

public HudiDataFileExtractor(
HoodieTableMetaClient metaClient,
PathBasedPartitionValuesExtractor hudiPartitionValuesExtractor,
HudiFileStatsExtractor hudiFileStatsExtractor,
boolean skipColumnStats) {
this.engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf());
metadataConfig =
HoodieMetadataConfig.newBuilder()
Expand All @@ -107,6 +116,7 @@ public HudiDataFileExtractor(
this.metaClient = metaClient;
this.partitionValuesExtractor = hudiPartitionValuesExtractor;
this.fileStatsExtractor = hudiFileStatsExtractor;
this.skipColumnStats = skipColumnStats;
}

public List<PartitionFileGroup> getFilesCurrentState(InternalTable table) {
Expand All @@ -132,11 +142,15 @@ public InternalFilesDiff getDiffForCommit(
getAddedAndRemovedPartitionInfo(
visibleTimeline, instant, fsView, hoodieInstantForDiff, table.getPartitioningFields());

Stream<InternalDataFile> filesAddedWithoutStats = allInfo.getAdded().stream();
List<InternalDataFile> filesAdded =
fileStatsExtractor
.addStatsToFiles(tableMetadata, filesAddedWithoutStats, table.getReadSchema())
.collect(Collectors.toList());
skipColumnStats
? fileStatsExtractor
.addRecordCountToFiles(
tableMetadata, allInfo.getAdded().stream(), table.getReadSchema())
.collect(Collectors.toList())
: fileStatsExtractor
.addStatsToFiles(tableMetadata, allInfo.getAdded().stream(), table.getReadSchema())
.collect(Collectors.toList());
List<InternalDataFile> filesRemoved = allInfo.getRemoved();

return InternalFilesDiff.builder().filesAdded(filesAdded).filesRemoved(filesRemoved).build();
Expand Down Expand Up @@ -359,6 +373,12 @@ private List<PartitionFileGroup> getInternalDataFilesForPartitions(
.getLatestBaseFiles(partitionPath)
.map(baseFile -> buildFileWithoutStats(partitionValues, baseFile));
});
if (skipColumnStats) {
Stream<InternalDataFile> filesWithRecordCount =
fileStatsExtractor.addRecordCountToFiles(
tableMetadata, filesWithoutStats, table.getReadSchema());
return PartitionFileGroup.fromFiles(filesWithRecordCount);
}
Stream<InternalDataFile> files =
fileStatsExtractor.addStatsToFiles(tableMetadata, filesWithoutStats, table.getReadSchema());
return PartitionFileGroup.fromFiles(files);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,30 @@ public Stream<InternalDataFile> addStatsToFiles(
: computeColumnStatsFromParquetFooters(files, nameFieldMap);
}

/**
* Adds record count information only.
*
* <p>This avoids materializing full min/max/null column stats in memory while still writing
* correct file row counts (for example Delta `numRecords`).
*/
public Stream<InternalDataFile> addRecordCountToFiles(
HoodieTableMetadata metadataTable, Stream<InternalDataFile> files, InternalSchema schema) {
boolean useMetadataTableColStats =
metadataTable != null
&& metaClient
.getTableConfig()
.isMetadataPartitionAvailable(MetadataPartitionType.COLUMN_STATS);
final Map<String, InternalField> nameFieldMap =
schema.getAllFields().stream()
.collect(
Collectors.toMap(
field -> getFieldNameForStats(field, useMetadataTableColStats),
Function.identity()));
return useMetadataTableColStats
? computeRecordCountFromMetadataTable(metadataTable, files, nameFieldMap)
: computeRecordCountFromParquetFooters(files);
}

private Stream<InternalDataFile> computeColumnStatsFromParquetFooters(
Stream<InternalDataFile> files, Map<String, InternalField> nameFieldMap) {
return files.map(
Expand All @@ -115,6 +139,16 @@ private Stream<InternalDataFile> computeColumnStatsFromParquetFooters(
});
}

private Stream<InternalDataFile> computeRecordCountFromParquetFooters(
Stream<InternalDataFile> files) {
return files.map(
file ->
file.toBuilder()
.recordCount(
UTILS.getRowCount(metaClient.getHadoopConf(), new Path(file.getPhysicalPath())))
.build());
}

private Pair<String, String> getPartitionAndFileName(String path) {
Path filePath = new CachingPath(path);
String partitionPath = HudiPathUtils.getPartitionPath(metaClient.getBasePathV2(), filePath);
Expand Down Expand Up @@ -167,6 +201,37 @@ private Stream<InternalDataFile> computeColumnStatsFromMetadataTable(
});
}

private Stream<InternalDataFile> computeRecordCountFromMetadataTable(
HoodieTableMetadata metadataTable,
Stream<InternalDataFile> files,
Map<String, InternalField> nameFieldMap) {
if (nameFieldMap.isEmpty()) {
return files.map(file -> file.toBuilder().recordCount(0L).build());
}
Map<Pair<String, String>, InternalDataFile> filePathsToDataFile =
files.collect(
Collectors.toMap(
file -> getPartitionAndFileName(file.getPhysicalPath()), Function.identity()));
if (filePathsToDataFile.isEmpty()) {
return Stream.empty();
}
// Query a single column to fetch per-file valueCount, which is sufficient for row count.
String anyField = nameFieldMap.keySet().iterator().next();
Map<Pair<String, String>, HoodieMetadataColumnStats> statsByFile =
metadataTable.getColumnStats(new ArrayList<>(filePathsToDataFile.keySet()), anyField);
return filePathsToDataFile.entrySet().stream()
.map(
entry -> {
HoodieMetadataColumnStats stats = statsByFile.get(entry.getKey());
long recordCount =
stats != null
? stats.getValueCount()
: UTILS.getRowCount(
metaClient.getHadoopConf(), new Path(entry.getValue().getPhysicalPath()));
return entry.getValue().toBuilder().recordCount(recordCount).build();
});
}

private Optional<Long> getMaxFromColumnStats(List<ColumnStat> columnStats) {
return columnStats.stream()
.filter(entry -> entry.getField().getParentPath() == null)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@

import lombok.Value;

import org.apache.hadoop.conf.Configuration;

import com.google.common.base.Preconditions;

import org.apache.xtable.model.schema.PartitionFieldSpec;
Expand All @@ -39,6 +41,7 @@ public class HudiSourceConfig {
"xtable.hudi.source.partition_spec_extractor_class";
public static final String PARTITION_FIELD_SPEC_CONFIG =
"xtable.hudi.source.partition_field_spec_config";
public static final String SKIP_COLUMN_STATS_CONFIG = "xtable.source.skip_column_stats";

String partitionSpecExtractorClass;
List<PartitionFieldSpec> partitionFieldSpecs;
Expand Down Expand Up @@ -84,4 +87,23 @@ public PathBasedPartitionSpecExtractor loadSourcePartitionSpecExtractor() {
return ReflectionUtils.createInstanceOfClass(
partitionSpecExtractorClass, this.getPartitionFieldSpecs());
}

public static boolean getSkipColumnStats(Properties properties, Configuration configuration) {
String propertyValue = getPropertyOrNull(properties, SKIP_COLUMN_STATS_CONFIG);
if (propertyValue != null) {
return Boolean.parseBoolean(propertyValue);
}
if (configuration == null) {
return false;
}
String configValue = configuration.get(SKIP_COLUMN_STATS_CONFIG);
return configValue != null && Boolean.parseBoolean(configValue);
}

private static String getPropertyOrNull(Properties properties, String key) {
if (properties == null) {
return null;
}
return properties.getProperty(key);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,8 @@ public class IcebergConversionSource implements ConversionSource<Snapshot> {
private final IcebergDataFileExtractor dataFileExtractor =
IcebergDataFileExtractor.builder().build();

@Builder.Default private final boolean skipColumnStats = false;

private Table initSourceTable() {
IcebergTableManager tableManager = IcebergTableManager.of(hadoopConf);
String[] namespace = sourceTableConfig.getNamespace();
Expand Down Expand Up @@ -147,6 +149,9 @@ public InternalTable getTable(Snapshot snapshot) {
public InternalTable getCurrentTable() {
Table iceTable = getSourceTable();
Snapshot currentSnapshot = iceTable.currentSnapshot();
if (currentSnapshot == null) {
throw new ReadException("Unable to read latest snapshot from Iceberg source table");
}
return getTable(currentSnapshot);
}

Expand All @@ -166,11 +171,12 @@ public InternalSnapshot getCurrentSnapshot() {
.sourceIdentifier("0")
.build();
}

InternalTable irTable = getTable(currentSnapshot);

TableScan scan =
iceTable.newScan().useSnapshot(currentSnapshot.snapshotId()).includeColumnStats();
TableScan scan = iceTable.newScan().useSnapshot(currentSnapshot.snapshotId());
if (!skipColumnStats) {
scan = scan.includeColumnStats();
}
PartitionSpec partitionSpec = iceTable.spec();
List<PartitionFileGroup> partitionedDataFiles;
try (CloseableIterable<FileScanTask> files = scan.planFiles()) {
Expand All @@ -197,7 +203,8 @@ private InternalDataFile fromIceberg(
DataFile file, PartitionSpec partitionSpec, InternalTable internalTable) {
List<PartitionValue> partitionValues =
partitionConverter.toXTable(internalTable, file.partition(), partitionSpec);
return dataFileExtractor.fromIceberg(file, partitionValues, internalTable.getReadSchema());
return dataFileExtractor.fromIceberg(
file, partitionValues, internalTable.getReadSchema(), !skipColumnStats);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,13 @@
public class IcebergConversionSourceProvider extends ConversionSourceProvider<Snapshot> {
@Override
public IcebergConversionSource getConversionSourceInstance(SourceTable sourceTableConfig) {
boolean skipColumnStats =
IcebergSourceConfig.getSkipColumnStats(
sourceTableConfig.getAdditionalProperties(), hadoopConf);
return IcebergConversionSource.builder()
.sourceTableConfig(sourceTableConfig)
.hadoopConf(hadoopConf)
.skipColumnStats(skipColumnStats)
.build();
}
}
Loading
Loading