HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456847406



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,279 @@
+/*
+ * 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.SparkPlan
+import org.apache.spark.sql.execution.columnar.{ColumnStatisticsSchema, 
PartitionStatistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{AtomicType, BinaryType}
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+/**
+ * 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 for compressing, filtering, and decompressing SQL data that 
will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Run the given plan and convert its output to a implementation of 
[[CachedBatch]].
+   * @param cachedPlan the plan to run.
+   * @return the RDD containing the batches of data to cache.
+   */
+  def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch]
+
+  /**
+   * Builds a function that can be used to filter which batches are loaded.
+   * 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 loaded.
+   */
+  def buildFilter(predicates: Seq[Expression],
+      cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => 
Iterator[CachedBatch]
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only 
used for basic types
+   * BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | 
DoubleType
+   * That may change in the future.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the field that should be loaded from the data, 
and the order they
+   *                           should appear in the output batch.
+   * @param conf the configuration for the job.
+   * @return an RDD of the input cached batches transformed into the 
ColumnarBatch format.
+   */
+  def decompressColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached batch into [[InternalRow]]. If you want this to be 
performant, code

Review comment:
        I think the error seems from here:
   
   ```
   [warn] 
/home/runner/work/spark/spark/sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala:90:
 Could not find any member to link for "InternalRow".
   [warn]   /**
   [warn]   ^
   
   [error] 
/home/runner/work/spark/spark/sql/core/target/java/org/apache/spark/sql/columnar/CachedBatchSerializer.java:40:
 error: reference not found
   [error]    * Decompress the cached batch into {@link InternalRow}. If you 
want this to be performant, code
   [error]                                              ^
   ```
   
   `InternalRow` is not documented and the corresponding Javadoc is not 
generated via Unidoc: 
https://github.com/apache/spark/blob/cf22d947fb8f37aa4d394b6633d6f08dbbf6dc1c/project/SparkBuild.scala#L843
   
   It is currently not very easy to navigate the logs from GitHub Actions. It 
is being investigated at SPARK-32253. Maybe we should have a way to report the 
test results nicely.
   
   Another related issue here is that, Unidoc shows all warnings as false 
positive errors when there is any actual error. That's why we see the errors 
such as:
   
   ```
   [info] Constructing Javadoc information...
   [error] 
/home/runner/work/spark/spark/mllib/target/java/org/apache/spark/mllib/util/MLlibTestSparkContext.java:10:
 error: illegal combination of modifiers: public and protected
   [error]   protected  class testImplicits {
   [error]              ^
   [error] 
/home/runner/work/spark/spark/mllib/target/java/org/apache/spark/mllib/util/MLlibTestSparkContext.java:67:
 error: illegal combination of modifiers: public and protected
   [error]   protected  class testImplicits$ extends 
org.apache.spark.sql.SQLImplicits {
   [error]              ^
   ```
   
   which are actually warnings IIRC. I investigated at SPARK-20840 which I 
tried hard to fix it but failed.
   _If I am remembering right_, Javadoc is being executed in a separate process 
inside Unidoc and there's no way to only show errors because warnings are also 
redirected to stderr. So, if the process is terminated with a non-zero exit 
code, it just redirects all warnings and errors.
   
   




----------------------------------------------------------------
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:
us...@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to