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
60 changes: 60 additions & 0 deletions orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,60 @@
/*
* 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.iceberg.orc;

import java.io.IOException;
import java.util.Collections;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.hadoop.HadoopInputFile;
import org.apache.iceberg.io.InputFile;
import org.apache.orc.OrcFile;
import org.apache.orc.Reader;

public class OrcMetrics {

private OrcMetrics() {}

public static Metrics fromInputFile(InputFile file) {
final Configuration config = (file instanceof HadoopInputFile)
? ((HadoopInputFile)file).getConf()
: new Configuration();
return fromInputFile(file, config);
}

public static Metrics fromInputFile(InputFile file, Configuration config) {
try {
final Reader orcReader = OrcFile.createReader(new Path(file.location()),
OrcFile.readerOptions(config));

// TODO: implement rest of the methods for ORC metrics
return new Metrics(orcReader.getNumberOfRows(),
null,
null,
Collections.emptyMap(),
null,
null);
} catch (IOException ioe) {
throw new RuntimeIOException(ioe, "Failed to read footer of file: %s", file);
}
}
}
29 changes: 29 additions & 0 deletions spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,9 @@ import org.apache.spark.sql.{DataFrame, SparkSession}
import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition
import scala.collection.JavaConverters._

import org.apache.iceberg.hadoop.HadoopInputFile
import org.apache.iceberg.orc.OrcMetrics

object SparkTableUtil {
/**
* Returns a DataFrame with a row for each partition in the table.
Expand Down Expand Up @@ -73,6 +76,8 @@ object SparkTableUtil {
listAvroPartition(partition, uri)
} else if (format.contains("parquet")) {
listParquetPartition(partition, uri)
} else if (format.contains("orc")) {
listOrcPartition(partition, uri)
} else {
throw new UnsupportedOperationException(s"Unknown partition format: $format")
}
Expand Down Expand Up @@ -248,5 +253,29 @@ object SparkTableUtil {
bytesMapToArray(metrics.upperBounds))
}
}

private def listOrcPartition(
partitionPath: Map[String, String],
partitionUri: String): Seq[SparkDataFile] = {
val conf = new Configuration()
val partition = new Path(partitionUri)
val fs = partition.getFileSystem(conf)

fs.listStatus(partition, HiddenPathFilter).filter(_.isFile).map { stat =>
val metrics = OrcMetrics.fromInputFile(HadoopInputFile.fromPath(stat.getPath, conf))

SparkDataFile(
stat.getPath.toString,
partitionPath, "orc", stat.getLen,
stat.getBlockSize,
metrics.recordCount,
mapToArray(metrics.columnSizes),
mapToArray(metrics.valueCounts),
mapToArray(metrics.nullValueCounts),
bytesMapToArray(metrics.lowerBounds()),
bytesMapToArray(metrics.upperBounds())
)
}
}
}