ConeyLiu commented on code in PR #8123: URL: https://github.com/apache/iceberg/pull/8123#discussion_r1271259796
########## 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: > For instance, UnsafeRow implements both Externalizable and KryoSerializable to customize both. Do we still need the customized serializer if we use `UnsafeRow`? From my understanding, the object to/from UnsafeRow should already implement serializing/deserializing. Or maybe something I misunderstood. > I am inclined to try out optimizing Java serialization for GenericDataFile first but I can be convinced otherwise. This is indeed a good direction to try. Maybe benefits for other engines as well? > Another idea can be to have an extra data structure per a group of data files so that we would serialize each partition info only once vs serializing the same partition tuple for many data files. That said, it is only applicable to full table scans and I am not sure is worth the complexity. Is the reason for this because the biggest overhead of serialization and deserialization comes from partition data? -- 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]
