aokolnychyi commented on code in PR #8123: URL: https://github.com/apache/iceberg/pull/8123#discussion_r1270143466
########## spark/v3.4/spark/src/main/java/org/apache/iceberg/SparkDistributedDataBatchScan.java: ########## @@ -0,0 +1,332 @@ +/* + * 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; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.ClosingIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.SparkSession; + +public class SparkDistributedDataBatchScan extends DistributedDataBatchScan { + + private final SparkSession spark; + private final JavaSparkContext sparkContext; + private final SparkReadConf readConf; + + private Broadcast<Table> tableBroadcast = null; + + public SparkDistributedDataBatchScan(SparkSession spark, Table table, SparkReadConf readConf) { + this(spark, table, readConf, table.schema(), TableScanContext.empty()); + } + + private SparkDistributedDataBatchScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Schema schema, + TableScanContext context) { + super(table, schema, context); + this.spark = spark; + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.readConf = readConf; + } + + @Override + protected BatchScan newRefinedScan( + Table newTable, Schema newSchema, TableScanContext newContext) { + return new SparkDistributedDataBatchScan(spark, newTable, readConf, newSchema, newContext); + } + + @Override + protected int remoteParallelism() { + return sparkContext.defaultParallelism(); + } + + @Override + protected PlanningMode dataPlanningMode() { + return readConf.dataPlanningMode(); + } + + @Override + protected List<DataFile> planDataRemotely(List<ManifestFile> manifests) { Review Comment: My initial though was to query the `data_files` metadata table. However, that faced a number of issues. - Need to translate a predicate on the main table into a predicate on the metadata table. - Need to implement Iceberg to Spark filter conversion. - Hard to populate scan metrics. It was also expensive as `DataFile` is wrapped as `GenericInternalRow`, then converted to `UnsafeRow`, then converted to public `Row`, then serialized and collected to the driver, deserialized on the driver and then wrapped into `DataFile` again. Not to mention the required complexity. Right now, I am using `RDD` so there is only one round of serialization (either Java or Kryo) and no conversion. The current approach performs exceptionally well for selective queries. The cost to serialize the entire content of a 10 MB manifest is around 0.2-0.3 s and is not an issue. The full table scan performance depends on how quickly the driver can fetch the result from other nodes. The most critical part is the size of serialized data. **Option 1**: Java serialization of `DataFile` and `DeleteFile`. **Option 2**: Kryo serialization of `DataFile` and `DeleteFile`. **Option 3**: Converting `DataFile` and `DeleteFile` to `UnsafeRow` and using Java/Kryo serialization on top. Option 2 (serializing files with Kryo) produced around 15% smaller chunks compared to Option 1 (serializing files with Java) and did not require any extra logic to convert to and from `UnsafeRow`. Option 3 (serializing `UnsafeRow`) does not require `Kryo` to be efficient but requires way conversion and gave only 3-5% size reduction in size compared to Option 2. I was surprised how well Kryo worked on top of files but I still debate whether conversion to `UnsafeRow` makes sense to not require the user to enable Kryo. That said, using `UnsafeRow` will yield only marginally smaller chunks. -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
