viirya commented on a change in pull request #29067: URL: https://github.com/apache/spark/pull/29067#discussion_r463340106
########## File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala ########## @@ -0,0 +1,343 @@ +/* + * 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.spark.sql.columnar + +import org.apache.spark.annotation.{DeveloperApi, Since} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BindReferences, EqualNullSafe, EqualTo, Expression, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, Length, LessThan, LessThanOrEqual, Literal, Or, Predicate, StartsWith} +import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, PartitionStatistics} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.storage.StorageLevel + +/** + * Basic interface that all cached batches of data must support. This is primarily to allow + * for metrics to be handled outside of the encoding and decoding steps in a standard way. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatch { + def numRows: Int + def sizeInBytes: Long +} + +/** + * Provides APIs that handle transformations of SQL data associated with the cache/persist APIs. + */ +@DeveloperApi +@Since("3.1.0") +trait CachedBatchSerializer extends Serializable { + /** + * Can `convertColumnarBatchToCachedBatch()` be called instead of + * `convertInternalRowToCachedBatch()` for this given schema? True if it can and false if it + * cannot. Columnar input is only supported if the plan could produce columnar output. Currently + * this is mostly supported by input formats like parquet and orc, but more operations are likely + * to be supported soon. + * @param schema the schema of the data being stored. + * @return True if columnar input can be supported, else false. + */ + def supportsColumnarInput(schema: Seq[Attribute]): Boolean + + /** + * Convert an `RDD[InternalRow]` into an `RDD[CachedBatch]` in preparation for caching the data. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertInternalRowToCachedBatch( + input: RDD[InternalRow], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Convert an `RDD[ColumnarBatch]` into an `RDD[CachedBatch]` in preparation for caching the data. + * This will only be called if `supportsColumnarInput()` returned true for the given schema and + * the plan up to this point would could produce columnar output without modifying it. + * @param input the input `RDD` to be converted. + * @param schema the schema of the data being stored. + * @param storageLevel where the data will be stored. + * @param conf the config for the query. + * @return The data converted into a format more suitable for caching. + */ + def convertColumnarBatchToCachedBatch( + input: RDD[ColumnarBatch], + schema: Seq[Attribute], + storageLevel: StorageLevel, + conf: SQLConf): RDD[CachedBatch] + + /** + * Builds a function that can be used to filter batches prior to being decompressed. + * In most cases extending [[SimpleMetricsCachedBatchSerializer]] will provide the filter logic + * necessary. You will need to provide metrics for this to work. [[SimpleMetricsCachedBatch]] + * provides the APIs to hold those metrics and explains the metrics used, really just min and max. + * Note that this is intended to skip batches that are not needed, and the actual filtering of + * individual rows is handled later. + * @param predicates the set of expressions to use for filtering. + * @param cachedAttributes the schema/attributes of the data that is cached. This can be helpful + * if you don't store it with the data. + * @return a function that takes the partition id and the iterator of batches in the partition. + * It returns an iterator of batches that should be decompressed. + */ + def buildFilter( + predicates: Seq[Expression], + cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] + + /** + * Can `convertCachedBatchToColumnarBatch()` be called instead of + * `convertCachedBatchToInternalRow()` for this given schema? True if it can and false if it + * cannot. Columnar output is typically preferred because it is more efficient. Note that + * `convertCachedBatchToInternalRow()` must always be supported as there are other checks that + * can force row based output. + * @param schema the schema of the data being checked. + * @return true if columnar output should be used for this schema, else false. + */ + def supportsColumnarOutput(schema: StructType): Boolean + + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + * @param attributes the attributes to be output. + * @param conf the config for the query that will read the data. + */ + def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] = None + + /** + * Convert the cached data into a ColumnarBatch. This currently is only used if + * `supportsColumnar()` returns true for the associated schema, but there are other checks Review comment: supportsColumnar -> supportsColumnarOutput? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
