Skip to content
Merged
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
21 changes: 19 additions & 2 deletions spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.OptionalLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
Expand Down Expand Up @@ -280,8 +281,12 @@ public void pruneColumns(StructType newRequestedSchema) {

@Override
public Statistics estimateStatistics() {
if (!(table instanceof LegacyHiveTable) &&
(filterExpressions == null || filterExpressions == Expressions.alwaysTrue())) {
if (table instanceof LegacyHiveTable) {
// We currently don't have reliable stats for Hive tables
return EMPTY_STATS;
}

if (filterExpressions == null || filterExpressions == Expressions.alwaysTrue()) {
long totalRecords = PropertyUtil.propertyAsLong(table.currentSnapshot().summary(),
SnapshotSummary.TOTAL_RECORDS_PROP, Long.MAX_VALUE);
return new Stats(SparkSchemaUtil.estimateSize(lazyType(), totalRecords), totalRecords);
Expand All @@ -300,6 +305,18 @@ public Statistics estimateStatistics() {
return new Stats(sizeInBytes, numRows);
}

private static final Statistics EMPTY_STATS = new Statistics() {
@Override
public OptionalLong sizeInBytes() {
return OptionalLong.empty();
}

@Override
public OptionalLong numRows() {
return OptionalLong.empty();
}
};

@Override
public boolean enableBatchRead() {
if (readUsingBatch == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.io.Serializable;
import java.util.Collection;
import java.util.List;
import java.util.OptionalLong;
import java.util.stream.Collectors;
import org.apache.iceberg.CombinedScanTask;
import org.apache.iceberg.FileFormat;
Expand Down Expand Up @@ -184,8 +185,12 @@ public PartitionReaderFactory createReaderFactory() {

@Override
public Statistics estimateStatistics() {
if (!(table instanceof LegacyHiveTable) &&
(filterExpressions == null || filterExpressions.isEmpty())) {
if (table instanceof LegacyHiveTable) {
// We currently don't have reliable stats for Hive tables
return EMPTY_STATS;
}

if (filterExpressions == null || filterExpressions.isEmpty()) {
LOG.debug("using table metadata to estimate table statistics");
long totalRecords = PropertyUtil.propertyAsLong(table.currentSnapshot().summary(),
SnapshotSummary.TOTAL_RECORDS_PROP, Long.MAX_VALUE);
Expand All @@ -208,6 +213,18 @@ public Statistics estimateStatistics() {
return new Stats(sizeInBytes, numRows);
}

private static final Statistics EMPTY_STATS = new Statistics() {
@Override
public OptionalLong sizeInBytes() {
return OptionalLong.empty();
}

@Override
public OptionalLong numRows() {
return OptionalLong.empty();
}
};

private List<CombinedScanTask> tasks() {
if (tasks == null) {
TableScan scan = table
Expand Down