You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/07/10 18:20:00 UTC

[GitHub] [spark] revans2 opened a new pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

revans2 opened a new pull request #29067:
URL: https://github.com/apache/spark/pull/29067


   I would love to see this go into the 3.1 release.  It also applies cleanly to the 3.0 branch if someone wants to pull it in there too.
   
   This may look big, but it is mostly refactoring to get all of the code associated with transforming a batch around cacheing into a single location.  I did clean up a few things along the way to separate out the actual transformation from the metrics.
   
   I ran some simple tests manually to see if I could find any performance difference between the old and new code, and if there is any it is within error.


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456533213



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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 {

Review comment:
       I tried to move some of the classes, but I ran into issues with filtering.  `PartitionStatistics` and `ColumnStatisticsSchema` are private to `sql.execution.columnar`. I had to open them up slightly to be able to do the refactoring.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660200667


   **[Test build #126062 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126062/testReport)** for PR 29067 at commit [`cb29e47`](https://github.com/apache/spark/commit/cb29e474fb7ad007734fd0a91a6329e7508e627a).


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666876433


   **[Test build #126819 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126819/testReport)** for PR 29067 at commit [`ea762e5`](https://github.com/apache/spark/commit/ea762e596b4653227835fed27f3093868b6c8d72).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463017864



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().map{ batch =>
         if (enableAccumulatorsForTest) {
           readBatches.add(1)
         }
         numOutputRows += batch.numRows
         batch
       }
-
-      val columnTypes = requestedColumnDataTypes.map {
-        case udt: UserDefinedType[_] => udt.sqlType
-        case other => other
-      }.toArray
-      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
-      if (enableAccumulatorsForTest && columnarIterator.hasNext) {
-        readPartitions.add(1)
+    val rows = serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf)
+    if (enableAccumulatorsForTest) {

Review comment:
       ah I see, makes sense




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666878061






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458066347



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(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 convertForCacheColumnar(input: RDD[ColumnarBatch],

Review comment:
       ditto




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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667634213


   retest this please


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688613392


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay even we forgot to match. It happens.
   Let's just keep the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659577556


   https://issues.apache.org/jira/browse/SPARK-32334  @maropu please add any detailed comments there if you had specific thoughts.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661335997






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658424511






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662720624






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



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


[GitHub] [spark] cloud-fan commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667780468


   github action passes, I'm merging to master, thanks!


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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463383396



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
##########
@@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl
   val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes)
 }
 
-private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {
+private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {

Review comment:
       We can actually remove `private[sql]` since `execution` package itself is meant to be private (as of SPARK-16964)




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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656899609


   @tgravescs please take another look and let me know what you think


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661335028


   **[Test build #126187 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126187/testReport)** for PR 29067 at commit [`46f4021`](https://github.com/apache/spark/commit/46f40216fd246f4e81cbaa6ce930287bf4c92975).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662085767






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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667432114


   retest this please


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688837935


   Oh I understand now.  Sorry I got the title and the description out of sync.  Yes I will update it.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658513850






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



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


[GitHub] [spark] maropu edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659340965


   > Would that alleviate your concern about SPIP @maropu?
   
   Yea, the descison as @HyukjinKwon suggested sounds reasonable to me.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656819397






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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457539751



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,165 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = {
+    val batchSize = cachedPlan.conf.columnBatchSize
+    val useCompression = cachedPlan.conf.useCompression
+    convertForCacheInternal(cachedPlan, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(cachedPlan: SparkPlan,
+      batchSize: Int,
+      useCompression: Boolean): RDD[CachedBatch] = {
+    val output = cachedPlan.output
+    cachedPlan.execute().mapPartitionsInternal { rowIterator =>

Review comment:
       I added some comments.  Please let me know if you want me to remove them after the review.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661097790






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656959838


   **[Test build #125642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125642/testReport)** for PR 29067 at commit [`0361237`](https://github.com/apache/spark/commit/0361237986e10eec828cacb400ee6c948fee30fd).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] maropu edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659340965


   > Would that alleviate your concern about SPIP @maropu?
   
   Yea, the approach that you suggested sounds reasonable to me.


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



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456527948



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some

Review comment:
       Thanks. Actually I just wanted to say the comment is not correct.




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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658821135


   Personally I was thinking we didn't need an entire SPIP since its a localized developer api change which I see as more of a small improvement. If others think it's useful to have a SPIP then we should create one?
   
   Note at my previous job, I had wanted to experiment with changing the caching format as I think caching currently is very expensive.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-664574947






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665690606






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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r455805241



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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 {

Review comment:
       Sure what package would you like me to put them under? `org.apache.spark.sql`? `org.apache.spark.sql.columnar`?




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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666462746


   The test failure looks unrelated.
   ```
   Error in library(SparkR) : there is no package called ‘SparkR’
   Execution halted
   Had test warnings or failures; see logs.
   ```


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r453602842



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##########
@@ -474,12 +484,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
 
   test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") {
     val attribute = AttributeReference("a", IntegerType)()
-    val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil)
-    val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None,
-      LocalRelation(Seq(attribute), Nil))
-    val tableScanExec = InMemoryTableScanExec(Seq(attribute),
-      Seq(In(attribute, Nil)), testRelation)
-    assert(tableScanExec.partitionFilters.isEmpty)
+    val testSerializer = new TestCachedBatchSerializer(false, 1)
+    testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute))

Review comment:
       I removed the assert because the refactoring moved that state into a location that is not easily accessible.  From reading the test and the comments around it the purpose of the test is to verify that the particular case did not result in an exception. The purpose of the assertion was to verify that the internal state was what they expected. I felt that the assert was unnecessary, but if you want me to add it back in I can refactor the code further and do so.  




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662778384






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r459017671



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       or, can we do the check with
   ```
   !plan.supportsColumnar && plan.child.supportsColumnar
   ```




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662778108


   **[Test build #126359 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126359/testReport)** for PR 29067 at commit [`e58783e`](https://github.com/apache/spark/commit/e58783ec28fc839e5559ca8c2888aa54d3f1f5de).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class ColumnarToRowExec(child: SparkPlan) extends ColumnarToRowTransition with CodegenSupport `


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



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463343578



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,96 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertColumnarBatchToCachedBatch(

Review comment:
       When reaching here, this `serializer` always supports columnar input? i.e., `supportsColumnarInput` returns true for the `serializer`.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665828006






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458066130



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(input: RDD[InternalRow],

Review comment:
       code style nit:
   ```
   def func
       para1: T,
       para2: T): R
   ```




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662720624






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



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


[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862232



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,69 +208,55 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
-    }.persist(storageLevel)
-
+    val cached = serializer.convertForCache(cachedPlan)
+        .map { batch =>
+          sizeInBytesStats.add(batch.sizeInBytes)
+          rowCountStats.add(batch.numRows)
+          batch
+        }.persist(storageLevel)
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None

Review comment:
       Can we implement this without double-checked locking? It is not expected in general. Can we do it in initializer?
   
   If we need to use double-checked locking, we need `@volatile` for `ser`. See https://www.cs.umd.edu/~pugh/java/memoryModel/DoubleCheckedLocking.html
   




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667433931






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



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


[GitHub] [spark] kiszk commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660593980


   Can we add additional API for saying whether a given type is cached or not to `CachedBatchSerializer`? It depends on the implementation of `CachedBatchSerializer`. This addresses `These data types may expand in the future. If they do we can add in a new API with a default value that says which data types this serializer supports.`
   
   In the following code, this part can be exposed as an API.
   ```
   relation.schema.fields.forall(f => f.dataType match { 
        case BooleanType | ByteType | ShortType | IntegerType | LongType | 
             FloatType | DoubleType => true 
        case _ => false 
      })
   ```
   
   https://github.com/apache/spark/blob/026b0b926dfd40038f2cee932f38b917eb25b77e/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L72-L80


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661161505






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658172413






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661097790






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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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.
   
   For now, we can just wrap it with `` `...` `` in this case to work around instead of `[[...]]`
   
   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 (via SBT) 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 show 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:
users@infra.apache.org



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456509822



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some

Review comment:
       That was by design.  Calculating min and max is not that complex and other implementations might have more efficient ways to do it, especially if they are also compressing the data they might be able to do it at the same time. I will update the comment to make it clearer.




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



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


[GitHub] [spark] maropu edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659340965






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662678170






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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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.
   
   For now, we can just wrap it with `` `...` `` in this case to work around instead of `[[...]]`
   
   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 show 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:
users@infra.apache.org



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


[GitHub] [spark] maropu commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658506540


   >  We want to produce a transparent replacement for .cache, .persist and the SQL CACHE operator using GPUs for acceleration. Caching data right now is slow on the CPU. .cache is a common enough operation that we would like to support it in our plugin and feel that it is something that we can really accelerate. In theory, we could reuse the datasource v2 API, but it would require a lot more refactoring to make it fit into the cache operator. Possibly refactoring of the data source V2 API as well. If that is what you think we need to do I can work on it, but it will be a much bigger change.
   
   hm, I think its worth documenting the proposal as SPIP. Then, we need to discuss it in the dev mailing list first (I'm currently not sure that other developers want this interface) WDYT? @HyukjinKwon @cloud-fan @dongjoon-hyun 


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661356780






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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661355943


   @cloud-fan I just added in the API changes that you requested.  Please let me know what you think, and if the changes to `QueryExecution` are acceptable. I will work on some tests for the Columnar input path to verify that it works too.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658171648


   **[Test build #125836 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125836/testReport)** for PR 29067 at commit [`641f28f`](https://github.com/apache/spark/commit/641f28f897db6d1bad709118e5701845a1898ae7).


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457527461



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,291 @@
+/*
+ * 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, StructType}
+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 {

Review comment:
       I'd like to understand the API design concerns that lead to this design. IMO the two major concerns should be
   1. the input to the cache builder should be row-based or columnar?
   2. the output of the cache builder should be row-based or columnar?
   
   It seems to me that `convertForCache` is to address 1. `SparkPlan` provides APIs to get row-based or columnar data. But I think exposing `SparkPlan` is overkill: it's a very big class that has many methods. Can we create a trait to abstract the main functionalities?
   
   For 2, the current API looks reasonable.




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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666889187


   https://github.com/apache/spark/pull/29067#issuecomment-666462746, @revans2, there was a bit of unexpected test failures in GitHub Actions, and they are fixed now. The GitHub Actions builds should pass once you sync with the latest master after rebasing or merging the upstream.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r461677607



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def convertForCache(input: RDD[InternalRow],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] = {
+    val batchSize = conf.columnBatchSize
+    val useCompression = conf.useCompression
+    convertForCacheInternal(input, schema, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(input: RDD[InternalRow],
+      output: Seq[Attribute],
+      batchSize: Int,
+      useCompression: Boolean): RDD[CachedBatch] = {
+    // Most of this code originally came from `CachedRDDBuilder.buildBuffers()`

Review comment:
       we should remove it (and other similar comments) before PR merging.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),

Review comment:
       super nit:
   ```
   obj.func(
     para1,
     para2,
     ...)
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       hmm, why do we expect `InputAdapter`? Is it possible to be `ColumnarToRowExec(ColumnarProjectExec(...))`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only used if
+   * `supportsColumnar()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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(

Review comment:
       the name `decompress` sounds like the cache must be compressed, which may not be true. How about `readCachedData` and `readCachedDataColumnar`?

##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##########
@@ -551,4 +637,23 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
       }
     }
   }
+
+  test("Columnar Cache Plugin Plan") {

Review comment:
       Can we move it to a new test suite? Then we can set static conf in that suite to create the spark session, and have a real end-to-end test.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +85,32 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
-        if (enableAccumulatorsForTest) {
-          readBatches.add(1)
-        }
-        numOutputRows += batch.numRows
+    // update SQL metrics
+    val withMetrics = if (enableAccumulatorsForTest) {

Review comment:
       shall we keep the original code style, and put `if (enableAccumulatorsForTest)` inside `filteredCachedBatches().map`? Then we just need a single `map` call.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,189 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def convertForCache(input: RDD[InternalRow],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] = {
+    val batchSize = conf.columnBatchSize
+    val useCompression = conf.useCompression
+    convertForCacheInternal(input, schema, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(input: RDD[InternalRow],
+      output: Seq[Attribute],
+      batchSize: Int,
+      useCompression: Boolean): RDD[CachedBatch] = {
+    // Most of this code originally came from `CachedRDDBuilder.buildBuffers()`
+    input.mapPartitionsInternal { rowIterator =>
+      new Iterator[DefaultCachedBatch] {
+        def next(): DefaultCachedBatch = {
+          val columnBuilders = output.map { attribute =>
+            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
+          }.toArray
+
+          var rowCount = 0
+          var totalSize = 0L
+          while (rowIterator.hasNext && rowCount < batchSize
+              && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
+            val row = rowIterator.next()
+
+            // Added for SPARK-6082. This assertion can be useful for scenarios when something
+            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
+            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
+            // hard to decipher.
+            assert(
+              row.numFields == columnBuilders.length,
+              s"Row column number mismatch, expected ${output.size} columns, " +
+                  s"but got ${row.numFields}." +
+                  s"\nRow content: $row")
+
+            var i = 0
+            totalSize = 0
+            while (i < row.numFields) {
+              columnBuilders(i).appendFrom(row, i)
+              totalSize += columnBuilders(i).columnStats.sizeInBytes
+              i += 1
+            }
+            rowCount += 1
+          }
+
+          val stats = InternalRow.fromSeq(
+            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
+          DefaultCachedBatch(rowCount, columnBuilders.map { builder =>
+            JavaUtils.bufferToArray(builder.build())
+          }, stats)
+        }
+
+        def hasNext: Boolean = rowIterator.hasNext
+      }
+    }
+  }
+
+  override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f =>
+      // This code originally came from `InMemoryTableScanExec.supportsColumnar`
+    f.dataType match {
+        // More types can be supported, but this is to match the original implementation that
+        // only supported primitive types "for ease of review"
+      case BooleanType | ByteType | ShortType | IntegerType | LongType |
+           FloatType | DoubleType => true
+      case _ => false
+    })
+
+  override def vectorTypes(attributes: Seq[Attribute], conf: SQLConf): Option[Seq[String]] =
+    Option(Seq.fill(attributes.length)(
+      if (!conf.offHeapColumnVectorEnabled) {
+        classOf[OnHeapColumnVector].getName
+      } else {
+        classOf[OffHeapColumnVector].getName
+      }
+    ))
+
+  override def decompressColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch] = {
+    // Most of this code originally came from `InMemoryTableScanExec.createAndDecompressColumn`
+    val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled
+    val outputSchema = StructType.fromAttributes(selectedAttributes)
+    val columnIndices =
+      selectedAttributes.map(a => cacheAttributes.map(o => o.exprId).indexOf(a.exprId)).toArray
+
+    def createAndDecompressColumn(cb: CachedBatch): ColumnarBatch = {
+      val cachedColumnarBatch = cb.asInstanceOf[DefaultCachedBatch]
+      val rowCount = cachedColumnarBatch.numRows
+      val taskContext = Option(TaskContext.get())
+      val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) {
+        OnHeapColumnVector.allocateColumns(rowCount, outputSchema)
+      } else {
+        OffHeapColumnVector.allocateColumns(rowCount, outputSchema)
+      }
+      val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]])
+      columnarBatch.setNumRows(rowCount)
+
+      for (i <- selectedAttributes.indices) {
+        ColumnAccessor.decompress(
+          cachedColumnarBatch.buffers(columnIndices(i)),
+          columnarBatch.column(i).asInstanceOf[WritableColumnVector],
+          outputSchema.fields(i).dataType, rowCount)
+      }
+      taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close()))
+      columnarBatch
+    }
+
+    input.map(createAndDecompressColumn)
+  }
+
+  override def decompressToRows(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow] = {
+    // Find the ordinals and data types of the requested columns.
+    val (requestedColumnIndices, requestedColumnDataTypes) =
+      selectedAttributes.map { a =>
+        cacheAttributes.map(_.exprId).indexOf(a.exprId) -> a.dataType
+      }.unzip
+
+    val columnTypes = requestedColumnDataTypes.map {
+      case udt: UserDefinedType[_] => udt.sqlType
+      case other => other
+    }.toArray
+
+    input.mapPartitionsInternal { cachedBatchIterator =>
+      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
+      columnarIterator.initialize(cachedBatchIterator.asInstanceOf[Iterator[DefaultCachedBatch]],
+        columnTypes,
+        requestedColumnIndices.toArray)
+      columnarIterator
+    }
+  }
+
+  override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false

Review comment:
       nit: we should move it to near `convertForCache`

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only used if
+   * `supportsColumnar()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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
+   * generation is advised.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   * Which is repeated for each column in the original data.
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long =
+    Range.apply(4, stats.numFields, 5).map(stats.getLong).sum
+}
+
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
+
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ * The requirement to extend this is that all of the batches produced by your serializer are
+ * instances of [[SimpleMetricsCachedBatch]].
+ * This does not calculate the metrics needed to be stored in the batches. That is up to each
+ * implementation. The metrics required are really just min and max values and those are optional
+ * especially for complex types. Because those metrics are simple and it is likely that compression
+ * will also be done on the data we thought it best to let each implementation decide on the most
+ * efficient way to calculate the metrics, possibly combining them with compression passes that
+ * might also be done across the data.
+ */
+@DeveloperApi
+@Since("3.1.0")
+abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {
+  override def buildFilter(predicates: Seq[Expression],

Review comment:
       super nit:
   ```
   def func(
       para1: T,
       para2: T): R = ...
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala
##########
@@ -49,6 +49,11 @@ class ColumnarRule {
   def postColumnarTransitions: Rule[SparkPlan] = plan => plan
 }
 
+/**
+ * A trait that is used as a tag to indicate a transition from columns to rows.

Review comment:
       maybe explain more about why we need it: help us to know if a query plan can produce columnar data natively.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),

Review comment:
       ditto

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child
+        case _ => plan
+      }
+      case _ => plan
+    }
+    case _ => plan

Review comment:
       shall we allow whole-stage-codegen disabled? I think it's simple to do so
   ```
   case gen: WholeStageCodegenExec => convertToColumnarIfPossible(gen)
   case ... // the real code
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       ah I see, so the case here is simply whole-stage-codegened parquet/orc scan.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       ah I see, so the case here is simply whole-stage-codegened parquet/orc scan or other columnar operators.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       ah I see, so the case here is a simple `WholeStageCodegenExec` with parquet/orc scan or other columnar operators.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       ah I see, so the case here is a simple `WholeStageCodegenExec` with parquet/orc scan or other columnar operators, wrapped by `InputAdapter`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only used if
+   * `supportsColumnar()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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(

Review comment:
       SGTM, it's also consistent with `convertToCache`




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662677438


   **[Test build #126359 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126359/testReport)** for PR 29067 at commit [`e58783e`](https://github.com/apache/spark/commit/e58783ec28fc839e5559ca8c2888aa54d3f1f5de).


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463014789



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,35 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
-
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
-        if (enableAccumulatorsForTest) {
-          readBatches.add(1)
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
+
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().mapPartitionsInternal { iter =>
+        if (enableAccumulatorsForTest && iter.hasNext) {
+          readPartitions.add(1)
         }
-        numOutputRows += batch.numRows
-        batch
-      }
+        new Iterator[CachedBatch] {

Review comment:
       sure




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656910151


   **[Test build #125642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125642/testReport)** for PR 29067 at commit [`0361237`](https://github.com/apache/spark/commit/0361237986e10eec828cacb400ee6c948fee30fd).


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659139846


   I just saw the comment. Thanks for summarizing @revans2.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661335997






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



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


[GitHub] [spark] maropu commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-657188271


   Hi, @revans2 , thanks for the work. I have two questions now;
   > The use case we have right now is in connection with https://github.com/NVIDIA/spark-rapids where we would like to provide compression that is better suited for both the CPU and the GPU.
   
   1.  You cannot use the data source V2 interface for your purpose? (Or, you cannot extend the interface for that?) That's because I think the interface is intended for interconnection between Spark and other systems (or files). The current cache structure is tightly coupled to the Spark internal one, so I'm not sure that we can directly expose it to 3rd-party developers.
   2. If `SPARK_CACHE_SERIALIZER` specified,  does the current approach replace all the existing caching operations with custom ones? Users cannot select which cache structure (default or custom) is used on runtime?
   
   Btw, could you move your design comment above into the PR description? Since it will appear in a commit log, it is important to write the detail of a PR proposal for better traceability.
   


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656824473


   I would love to see this go into the 3.1 release.  It also applies cleanly to the 3.0 branch if someone wants to pull it in there too.
   
   This may look big, but it is mostly refactoring to get all of the code associated with transforming a batch around caching into a single location.  I did clean up a few things along the way to separate out the actual transformation from the metrics.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658424511






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665827301






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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-657080714


   @revans2, would you mind outlining which APIs do you propose in the PR description? Seems like it's not only configurations but classes. It would be great if it describes how to use with a concrete use case to provide a better context.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667250171






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458195360



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       That makes sense.
   
   Then my question is: how do we know a plan can produce columnar data? I see you added quite a bit of changes for it, but I've not figured it out yet.
   
   I expect something like
   ```
   plan.isInstanceOf[WholeStageCodegen] && plan.children.head.isInstanceOf[ColumnarToRow]
   ```
   
   Maybe I missed something. Can you elaborate it a bit more?




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658317535






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661599833


   **[Test build #126204 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126204/testReport)** for PR 29067 at commit [`38ca741`](https://github.com/apache/spark/commit/38ca74124a5d9552fa83bf228258db7bd58595e7).
    * This patch **fails PySpark pip packaging tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class ApplyColumnarRulesAndInsertTransitions(`


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688841873


   Please take a look at the description and let me know if I missed anything?


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661340978






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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457412184



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,69 +208,55 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
-    }.persist(storageLevel)
-
+    val cached = serializer.convertForCache(cachedPlan)
+        .map { batch =>
+          sizeInBytesStats.add(batch.sizeInBytes)
+          rowCountStats.add(batch.numRows)
+          batch
+        }.persist(storageLevel)
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None

Review comment:
       I'll try to get rid of it. I am aware of the issues with double-checked locking. In this case it is a matter of efficiency instead of correctness. The object being created is not going to change in any meaningful way.  If we created a new one each time it would still do the right thing. The goal here was just to remove the overhead of reflection when creating subsequent objects. but I should have at least documented it as such.




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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r462967298



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().map{ batch =>

Review comment:
       can we use `mapPartitionsInternal` like before? It's an optimized internal version and is better than the normal RDD APIs like `map`.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661917144






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662778384






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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658422580


   Thanks @tgravescs those were issues related to metrics.  One that I was over counting, and another that I ended up under counting.  They should be fixed now.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658317535


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662546576


   @cloud-fan I added in a test to validate that the changes I made are working.  I think this PR should be totally ready now.  Please let me know if you have more comments.


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



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


[GitHub] [spark] tgravescs commented on a change in pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r453009530



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -79,46 +77,14 @@ case class InMemoryTableScanExec(
     }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema)
   }
 
-  private val columnIndices =
-    attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray
-
-  private val relationSchema = relation.schema.toArray
-
-  private lazy val columnarBatchSchema = new StructType(columnIndices.map(i => relationSchema(i)))
-
-  private def createAndDecompressColumn(
-      cachedColumnarBatch: CachedBatch,
-      offHeapColumnVectorEnabled: Boolean): ColumnarBatch = {
-    val rowCount = cachedColumnarBatch.numRows
-    val taskContext = Option(TaskContext.get())
-    val columnVectors = if (!offHeapColumnVectorEnabled || taskContext.isEmpty) {
-      OnHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema)
-    } else {
-      OffHeapColumnVector.allocateColumns(rowCount, columnarBatchSchema)
-    }
-    val columnarBatch = new ColumnarBatch(columnVectors.asInstanceOf[Array[ColumnVector]])
-    columnarBatch.setNumRows(rowCount)
-
-    for (i <- attributes.indices) {
-      ColumnAccessor.decompress(
-        cachedColumnarBatch.buffers(columnIndices(i)),
-        columnarBatch.column(i).asInstanceOf[WritableColumnVector],
-        columnarBatchSchema.fields(i).dataType, rowCount)
-    }
-    taskContext.foreach(_.addTaskCompletionListener[Unit](_ => columnarBatch.close()))
-    columnarBatch
-  }
-
   private lazy val columnarInputRDD: RDD[ColumnarBatch] = {
     val numOutputRows = longMetric("numOutputRows")
     val buffers = filteredCachedBatches()
-    val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled
-    buffers
-      .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled))
-      .map { buffer =>
-        numOutputRows += buffer.numRows()
-        buffer
-      }
+    relation.cacheBuilder.serializer.decompressColumnar(buffers, relation.output, attributes, conf)
+        .map { cb =>

Review comment:
       nit, 2 space indention

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code

Review comment:
       nit: the cached batch back into
   also I think adding a , after performant.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+trait CachedBatchSerializer extends Serializable {

Review comment:
       we should mark this with @DeveloperApi
   We should also add in the @Since("3.1.0")
   should add to all the public interfaces you added

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code
+   * generation is advised.
+   * @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 the rows that were stored in the cached batches.
+   */
+  def decompressToRows(input: RDD[CachedBatch],

Review comment:
       put input on next line

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code
+   * generation is advised.
+   * @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 the rows that were stored in the cached batches.
+   */
+  def decompressToRows(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stored some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long = stats.getLong(4)
+}
 
-  @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
 
-  val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
-  val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ */
+trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {
+  override def buildFilter(predicates: Seq[Expression],
+      cachedAttributes: Seq[Attribute]):
+  (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {

Review comment:
       nit indentation off

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code
+   * generation is advised.
+   * @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 the rows that were stored in the cached batches.
+   */
+  def decompressToRows(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stored some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long = stats.getLong(4)
+}
 
-  @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
 
-  val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
-  val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ */
+trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {
+  override def buildFilter(predicates: Seq[Expression],
+      cachedAttributes: Seq[Attribute]):
+  (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
+    val stats = new PartitionStatistics(cachedAttributes)
+    val statsSchema = stats.schema
+
+    def statsFor(a: Attribute): ColumnStatisticsSchema = {
+      stats.forAttribute(a)
+    }
 
-  val cachedName = tableName.map(n => s"In-memory table $n")
-    .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024))
+    // Returned filter predicate should return false iff it is impossible for the input expression
+    // to evaluate to `true` based on statistics collected about this partition batch.
+    @transient lazy val buildFilter: PartialFunction[Expression, Expression] = {
+      case And(lhs: Expression, rhs: Expression)
+        if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) =>
+        (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _)
+
+      case Or(lhs: Expression, rhs: Expression)
+        if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
+        buildFilter(lhs) || buildFilter(rhs)
+
+      case EqualTo(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+      case EqualTo(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+
+      case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+      case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+
+      case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l
+      case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound
+
+      case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l
+      case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) =>
+        l <= statsFor(a).upperBound
+
+      case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound
+      case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l
+
+      case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) =>
+        l <= statsFor(a).upperBound
+      case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l
+
+      case IsNull(a: Attribute) => statsFor(a).nullCount > 0
+      case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0
+
+      case In(a: AttributeReference, list: Seq[Expression])
+        if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty =>
+        list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] &&
+            l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _)
+      // This is an example to explain how it works, imagine that the id column stored as follows:
+      // __________________________________________
+      // | Partition ID | lowerBound | upperBound |
+      // |--------------|------------|------------|
+      // |      p1      |    '1'     |    '9'     |
+      // |      p2      |    '10'    |    '19'    |
+      // |      p3      |    '20'    |    '29'    |
+      // |      p4      |    '30'    |    '39'    |
+      // |      p5      |    '40'    |    '49'    |
+      // |______________|____________|____________|
+      //
+      // A filter: df.filter($"id".startsWith("2")).
+      // In this case it substr lowerBound and upperBound:
+      // ________________________________________________________________________________________
+      // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) |
+      // |--------------|-----------------------------------|-----------------------------------|
+      // |      p1      |    '1'                            |    '9'                            |
+      // |      p2      |    '1'                            |    '1'                            |
+      // |      p3      |    '2'                            |    '2'                            |
+      // |      p4      |    '3'                            |    '3'                            |
+      // |      p5      |    '4'                            |    '4'                            |
+      // |______________|___________________________________|___________________________________|
+      //
+      // We can see that we only need to read p1 and p3.
+      case StartsWith(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound.substr(0, Length(l)) <= l &&
+            l <= statsFor(a).upperBound.substr(0, Length(l))
+    }
 
-  def cachedColumnBuffers: RDD[CachedBatch] = {
-    if (_cachedColumnBuffers == null) {
-      synchronized {
-        if (_cachedColumnBuffers == null) {
-          _cachedColumnBuffers = buildBuffers()
-        }
+    // When we bind the filters we need to do it against the stats schema
+    val partitionFilters: Seq[Expression] = {
+      predicates.flatMap { p =>
+        val filter = buildFilter.lift(p)
+        val boundFilter =
+          filter.map(
+            BindReferences.bindReference(
+              _,
+              statsSchema,
+              allowFailures = true))
+
+        boundFilter.foreach(_ =>
+          filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f")))
+
+        // If the filter can't be resolved then we are missing required statistics.
+        boundFilter.filter(_.resolved)
       }
     }
-    _cachedColumnBuffers
-  }
 
-  def clearCache(blocking: Boolean = false): Unit = {
-    if (_cachedColumnBuffers != null) {
-      synchronized {
-        if (_cachedColumnBuffers != null) {
-          _cachedColumnBuffers.unpersist(blocking)
-          _cachedColumnBuffers = null
+    def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = {
+      val partitionFilter = Predicate.create(
+        partitionFilters.reduceOption(And).getOrElse(Literal(true)),
+        cachedAttributes)
+
+      partitionFilter.initialize(index)
+      val schemaIndex = cachedAttributes.zipWithIndex
+
+      cachedBatchIterator.filter { cb =>
+        val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch]
+        if (!partitionFilter.eval(cachedBatch.stats)) {
+          logDebug {
+            val statsString = schemaIndex.map { case (a, i) =>
+              val value = cachedBatch.stats.get(i, a.dataType)
+              s"${a.name}: $value"
+            }.mkString(", ")
+            s"Skipping partition based on stats $statsString"
+          }
+          false
+        } else {
+          true
         }
       }
     }
+

Review comment:
       remove extra newline

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.

Review comment:
       nit: return RDD... 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code
+   * generation is advised.
+   * @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 the rows that were stored in the cached batches.
+   */
+  def decompressToRows(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stored some simple metrics that can be used for filtering of batches with

Review comment:
       nit s/stored/stores/

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Decompress the cached into back into [[InternalRow]]. If you want this to be performant code
+   * generation is advised.
+   * @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 the rows that were stored in the cached batches.

Review comment:
       return RDD of 

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -119,35 +323,168 @@ case class CachedRDDBuilder(
             rowCount += 1
           }
 
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
           val stats = InternalRow.fromSeq(
             columnBuilders.flatMap(_.columnStats.collectedStatistics))
-          CachedBatch(rowCount, columnBuilders.map { builder =>
+          DefaultCachedBatch(rowCount, columnBuilders.map { builder =>
             JavaUtils.bufferToArray(builder.build())
           }, stats)
         }
 
         def hasNext: Boolean = rowIterator.hasNext
       }
-    }.persist(storageLevel)
+    }
+  }
+
+  override def decompressColumnar(input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch] = {
+

Review comment:
       remove extra newline

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -119,35 +323,168 @@ case class CachedRDDBuilder(
             rowCount += 1
           }
 
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
           val stats = InternalRow.fromSeq(
             columnBuilders.flatMap(_.columnStats.collectedStatistics))
-          CachedBatch(rowCount, columnBuilders.map { builder =>
+          DefaultCachedBatch(rowCount, columnBuilders.map { builder =>
             JavaUtils.bufferToArray(builder.build())
           }, stats)
         }
 
         def hasNext: Boolean = rowIterator.hasNext
       }
-    }.persist(storageLevel)
+    }
+  }
+
+  override def decompressColumnar(input: RDD[CachedBatch],

Review comment:
       input param on next line

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,288 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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.
+ */
+trait CachedBatch {
+  def numRows: Int
+  def sizeInBytes: Long
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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 the batches in the ColumnarBatch format.
+   */
+  def decompressColumnar(input: RDD[CachedBatch],

Review comment:
       nit: put input on the next line




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656900211






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



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


[GitHub] [spark] dongjoon-hyun commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666547312


   Thank you for update, @revans2 .


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665691334






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



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


[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862990



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##########
@@ -42,12 +49,13 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
     data.createOrReplaceTempView(s"testData$dataType")
     val storageLevel = MEMORY_ONLY
     val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan
-    val inMemoryRelation = InMemoryRelation(useCompression = true, 5, storageLevel, plan, None,
+    val inMemoryRelation = InMemoryRelation(new TestCachedBatchSerializer(useCompression = true, 5),
+      storageLevel, plan, None,

Review comment:
       nit: `data.logicalPlan)` can be moved to the end of this line.




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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660250687


   error looks unrelated, kicking it


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667637957


   **[Test build #126933 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126933/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ColumnStatisticsSchema(a: Attribute) extends Serializable `
     * `class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable `


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667433678


   **[Test build #126902 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126902/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667350473


   **[Test build #126897 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126897/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ColumnStatisticsSchema(a: Attribute) extends Serializable `
     * `class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable `


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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458076452



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       This is the part I want to clarify.
   
   If the serializer can't support columnar input, it's easy and we just use the original query plan.
   
   If the serializer supports columnar input, what shall we do?
   1. if the serializer can't support row-based input (is it possible?), we have to add a row -> columnar operator.
   2. If the serializer also supports row-based input, but the query plan is row-based (no parquet scan), does it worth to add a row -> columnar operator?




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661294369


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126185/
   Test FAILed.


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



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


[GitHub] [spark] cloud-fan closed pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29067:
URL: https://github.com/apache/spark/pull/29067


   


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661340246


   **[Test build #126191 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126191/testReport)** for PR 29067 at commit [`10b0ce1`](https://github.com/apache/spark/commit/10b0ce193d2f711ddc5702465c847d0a94cde313).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661355673


   **[Test build #126204 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126204/testReport)** for PR 29067 at commit [`38ca741`](https://github.com/apache/spark/commit/38ca74124a5d9552fa83bf228258db7bd58595e7).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-664693938






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



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


[GitHub] [spark] maropu edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658506540


   >  We want to produce a transparent replacement for .cache, .persist and the SQL CACHE operator using GPUs for acceleration. Caching data right now is slow on the CPU. .cache is a common enough operation that we would like to support it in our plugin and feel that it is something that we can really accelerate. In theory, we could reuse the datasource v2 API, but it would require a lot more refactoring to make it fit into the cache operator. Possibly refactoring of the data source V2 API as well. If that is what you think we need to do I can work on it, but it will be a much bigger change.
   
   Thanks for the explanation, @revans2. hm, I think its worth documenting the proposal as SPIP. Then, we need to discuss it in the dev mailing list first (I'm currently not sure that other developers want this interface) WDYT? @HyukjinKwon @cloud-fan @dongjoon-hyun 


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656960099


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661294364


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661917144






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656818843


   **[Test build #125633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125633/testReport)** for PR 29067 at commit [`4bf995f`](https://github.com/apache/spark/commit/4bf995f35a4115302937eb04ccb99c90d9b3b681).


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



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


[GitHub] [spark] dongjoon-hyun commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688608918


   Hi, @gatorsmile. Is your request just about revising this GitHub PR description, right?
   We cannot change commit logs.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688576511


   @gatorsmile what name were you thinking of?  The patch did "make the SQL cache serialization pluggable" so I am not 100% sure what needs to be corrected.  I am happy to do it, just need to know what to change it to.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463013822



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,35 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
-
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
-        if (enableAccumulatorsForTest) {
-          readBatches.add(1)
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
+
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().mapPartitionsInternal { iter =>
+        if (enableAccumulatorsForTest && iter.hasNext) {
+          readPartitions.add(1)
         }
-        numOutputRows += batch.numRows
-        batch
-      }
+        new Iterator[CachedBatch] {

Review comment:
       I don't have a strong opinion here, but seems it's more common to write
   ```
   iter.map { batch =>
     if (enableAccumulatorsForTest) {
       readBatches.add(1)
     }
     numOutputRows += batch.numRows
     batch
   }
   ```




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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457589738



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,291 @@
+/*
+ * 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, StructType}
+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 {

Review comment:
       > Currently the only way to get around that is by walking the SparkPlan to check if the last stage is a code generation phase with only a child that is doing a columnar to row transition.
   
   I'd like to understand more about the physical path. So we only pass columnar data to the cache builder if the spark plan can produce columnar data natively (the last operator is a columnar to row transition)?
   
   If that's the case, I'd suggest we add 3 methods: `convertForCache(inpur: RDD[InternalRow])`, `convertForCacheColumnar(inpur: RDD[ColumnarBatch])` and `supportsColumnarInput(schema): Boolean`. 




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666649236






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661161505






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661160094


   **[Test build #126191 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126191/testReport)** for PR 29067 at commit [`10b0ce1`](https://github.com/apache/spark/commit/10b0ce193d2f711ddc5702465c847d0a94cde313).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661121310






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



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688613392


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay forgot to match. It happens.
   Let's just keep the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662678170






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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r455803871



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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
+   * generation is advised.
+   * @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 RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {

Review comment:
       `DefaultCachedBatch` is a `case class`, and I didn't think it was acceptable to have a `case class` inherent from a regular `class`. If I am wrong on that just let me know and I'll make it an abstract class.




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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660215714


   The errors from CI all appear to be javadoc errors for code that I didn't touch. Did I miss some change needed when I added in a new package?


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-657090408


   @HyukjinKwon sure, I am really proposing one API and one helper that could let someone reuse some of the existing code.  
   
   1. `CachedBatch` is a trait used to tag data that is intended to be cached. It has a few APIs that lets us keep the compression/serialization of the data separate from the metrics about it.
   2. `CachedBatchSerializer` provides the APIs that must be implemented to cache data.
       * `convertForCache` is an API that runs a cached spark plan and turns its result into an `RDD[CachedBatch]`.  The actual caching is done outside of this API
      * `buildFilter` is an API that takes a set of predicates and builds a filter function that can be used to filter the `RDD[CachedBatch]` returned by `convertForCache`
      * `decompressColumnar` decompresses an `RDD[CachedBatch]` into an `RDD[ColumnarBatch]` This is only used for a limited set of data types.  These data types may expand in the future.  If they do we can add in a new API with a default value that says which data types this serializer supports.
      * `decompressToRows` decompresses an `RDD[CachedBatch]` into an `RDD[InternalRow]` this API, like `decompressColumnar` decompresses the data in `CachedBatch` but turns it into `InternalRow`s, typically using code generation for performance reasons.
   
   There is also an API that lets you reuse the current filtering based on min/max values. `SimpleMetricsCachedBatch` and `SimpleMetricsCachedBatchSerializer`.
   
   These are set up so that a user can explore other types of compression or indexing and their impact on performance.  One could look at adding a bloom filter in addition to the min/max values currently supported for filtering. One could look at adding in compression for some of the data types not currently supported, like arrays or structs if that is something that is an important use case for someone.
   
   The use case we have right now is in connection with https://github.com/NVIDIA/spark-rapids where we would like to provide compression that is better suited for both the CPU and the GPU.
   
   The config right now is a static conf allowing one and only one cache serializer per session. That can change in the future but I didn't want to add in the extra code to track the serializer along with the RDD until there was a use case for it, and theoretically, if something is a clear win it should just be incorporated into the default serializer instead.
   
   
   


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r459060693



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       I switched the patch to do what you requested.




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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r455652745



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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
+   * generation is advised.
+   * @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 RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {

Review comment:
       Would we be able to avoid `trait`? It was actually recommended to avoid as an API when possible, https://github.com/databricks/scala-style-guide#traits-and-abstract-classes

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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 {

Review comment:
       Can we avoid to putting the APIs under `execution` package? it wouldn't be documented and this module is private, `execution/package.scala`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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
+   * generation is advised.
+   * @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 RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   * Which is repeated for each column in the original data.
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long =
+    Range.apply(4, stats.numFields, 5).map(stats.getLong).sum
+}
 
-  @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
 
-  val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
-  val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {
+  override def buildFilter(predicates: Seq[Expression],
+      cachedAttributes: Seq[Attribute]): (Int, Iterator[CachedBatch]) => Iterator[CachedBatch] = {
+    val stats = new PartitionStatistics(cachedAttributes)
+    val statsSchema = stats.schema
+
+    def statsFor(a: Attribute): ColumnStatisticsSchema = {
+      stats.forAttribute(a)
+    }
 
-  val cachedName = tableName.map(n => s"In-memory table $n")
-    .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024))
+    // Returned filter predicate should return false iff it is impossible for the input expression
+    // to evaluate to `true` based on statistics collected about this partition batch.
+    @transient lazy val buildFilter: PartialFunction[Expression, Expression] = {
+      case And(lhs: Expression, rhs: Expression)
+        if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) =>
+        (buildFilter.lift(lhs) ++ buildFilter.lift(rhs)).reduce(_ && _)
+
+      case Or(lhs: Expression, rhs: Expression)
+        if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) =>
+        buildFilter(lhs) || buildFilter(rhs)
+
+      case EqualTo(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+      case EqualTo(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+
+      case EqualNullSafe(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+      case EqualNullSafe(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound
+
+      case LessThan(a: AttributeReference, ExtractableLiteral(l)) => statsFor(a).lowerBound < l
+      case LessThan(ExtractableLiteral(l), a: AttributeReference) => l < statsFor(a).upperBound
+
+      case LessThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound <= l
+      case LessThanOrEqual(ExtractableLiteral(l), a: AttributeReference) =>
+        l <= statsFor(a).upperBound
+
+      case GreaterThan(a: AttributeReference, ExtractableLiteral(l)) => l < statsFor(a).upperBound
+      case GreaterThan(ExtractableLiteral(l), a: AttributeReference) => statsFor(a).lowerBound < l
+
+      case GreaterThanOrEqual(a: AttributeReference, ExtractableLiteral(l)) =>
+        l <= statsFor(a).upperBound
+      case GreaterThanOrEqual(ExtractableLiteral(l), a: AttributeReference) =>
+        statsFor(a).lowerBound <= l
+
+      case IsNull(a: Attribute) => statsFor(a).nullCount > 0
+      case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0
+
+      case In(a: AttributeReference, list: Seq[Expression])
+        if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty =>
+        list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] &&
+            l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _)
+      // This is an example to explain how it works, imagine that the id column stored as follows:
+      // __________________________________________
+      // | Partition ID | lowerBound | upperBound |
+      // |--------------|------------|------------|
+      // |      p1      |    '1'     |    '9'     |
+      // |      p2      |    '10'    |    '19'    |
+      // |      p3      |    '20'    |    '29'    |
+      // |      p4      |    '30'    |    '39'    |
+      // |      p5      |    '40'    |    '49'    |
+      // |______________|____________|____________|
+      //
+      // A filter: df.filter($"id".startsWith("2")).
+      // In this case it substr lowerBound and upperBound:
+      // ________________________________________________________________________________________
+      // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) |
+      // |--------------|-----------------------------------|-----------------------------------|
+      // |      p1      |    '1'                            |    '9'                            |
+      // |      p2      |    '1'                            |    '1'                            |
+      // |      p3      |    '2'                            |    '2'                            |
+      // |      p4      |    '3'                            |    '3'                            |
+      // |      p5      |    '4'                            |    '4'                            |
+      // |______________|___________________________________|___________________________________|
+      //
+      // We can see that we only need to read p1 and p3.
+      case StartsWith(a: AttributeReference, ExtractableLiteral(l)) =>
+        statsFor(a).lowerBound.substr(0, Length(l)) <= l &&
+            l <= statsFor(a).upperBound.substr(0, Length(l))
+    }
 
-  def cachedColumnBuffers: RDD[CachedBatch] = {
-    if (_cachedColumnBuffers == null) {
-      synchronized {
-        if (_cachedColumnBuffers == null) {
-          _cachedColumnBuffers = buildBuffers()
-        }
+    // When we bind the filters we need to do it against the stats schema
+    val partitionFilters: Seq[Expression] = {
+      predicates.flatMap { p =>
+        val filter = buildFilter.lift(p)
+        val boundFilter =
+          filter.map(
+            BindReferences.bindReference(
+              _,
+              statsSchema,
+              allowFailures = true))
+
+        boundFilter.foreach(_ =>
+          filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f")))
+
+        // If the filter can't be resolved then we are missing required statistics.
+        boundFilter.filter(_.resolved)
       }
     }
-    _cachedColumnBuffers
-  }
 
-  def clearCache(blocking: Boolean = false): Unit = {
-    if (_cachedColumnBuffers != null) {
-      synchronized {
-        if (_cachedColumnBuffers != null) {
-          _cachedColumnBuffers.unpersist(blocking)
-          _cachedColumnBuffers = null
+    def ret(index: Int, cachedBatchIterator: Iterator[CachedBatch]): Iterator[CachedBatch] = {
+      val partitionFilter = Predicate.create(
+        partitionFilters.reduceOption(And).getOrElse(Literal(true)),
+        cachedAttributes)
+
+      partitionFilter.initialize(index)
+      val schemaIndex = cachedAttributes.zipWithIndex
+
+      cachedBatchIterator.filter { cb =>
+        val cachedBatch = cb.asInstanceOf[SimpleMetricsCachedBatch]
+        if (!partitionFilter.eval(cachedBatch.stats)) {
+          logDebug {
+            val statsString = schemaIndex.map { case (a, i) =>
+              val value = cachedBatch.stats.get(i, a.dataType)
+              s"${a.name}: $value"
+            }.mkString(", ")
+            s"Skipping partition based on stats $statsString"
+          }
+          false
+        } else {
+          true
         }
       }
     }
+    ret
   }
+}
 
-  def isCachedColumnBuffersLoaded: Boolean = {
-    _cachedColumnBuffers != null
+/**
+ * The default implementation of CachedBatch.
+ *
+ * @param numRows The total number of rows in this batch
+ * @param buffers The buffers for serialized columns
+ * @param stats The stat of columns
+ */
+private[sql]

Review comment:
       Maybe we should remove this since `execution` package is meant to be private as of SPARK-16964




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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463737044



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,96 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertColumnarBatchToCachedBatch(

Review comment:
       Yes and No.  If you go through the front door and create an `InMemoryRelation` using `InMemoryRelation.apply`, but not the one made for testing, then it guarantees that the only time you get a `SparkPlan` that `supportsColumnar` is when the serializer also `supportsColumnarInput`. You could purposely construct an `InMemoryReplaction` that does not match this, but you would have to go out of your way to do it.  I can add in an assertion to an InMemoryRelation constructor to make sure that it is correct if you want me to.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661340978






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662677438


   **[Test build #126359 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126359/testReport)** for PR 29067 at commit [`e58783e`](https://github.com/apache/spark/commit/e58783ec28fc839e5559ca8c2888aa54d3f1f5de).


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



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


[GitHub] [spark] kiszk commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659371202


   I will also review it this weekend.


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458988318



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       I'll switch over to doing `isInstanceOf[ColumnarToRow]` just to get this unblocked.
   
   I don't consider the change I made a hack. It is a relatively small and focused change. It also will result in less implicit coupling between the cache code and the columnar planning code.
   
   I like SPARK-32334 too, but for what I have proposed so far it really only covers transformations to arrow. If you have ideas on how you want it to impact these changes please add some comments there so we can discuss it and get a better idea of how it should all fit together.




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667634347






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662085767






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656900211






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667637986






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



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688613392


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay to forgot matching. It happens.
   Let's just keep the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667440297


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126902/
   Test FAILed.


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



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


[GitHub] [spark] gatorsmile commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688551588


   @revans2 Could you update the PR description? The PR description does not match the actual code 


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



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


[GitHub] [spark] cloud-fan commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666480983


   retest this please


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658513379


   **[Test build #125860 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125860/testReport)** for PR 29067 at commit [`ad11a79`](https://github.com/apache/spark/commit/ad11a794a634488f4b01b7e735179b74c2619774).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456210407



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some

Review comment:
       Looks like the stats calculation is in `DefaultCachedBatchSerializer`, instead of `SimpleMetricsCachedBatchSerializer`?




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



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


[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456862806



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +97,32 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
-
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
-        if (enableAccumulatorsForTest) {
-          readBatches.add(1)
-        }
-        numOutputRows += batch.numRows
+    // update SQL metrics
+    val withMetrics = if (enableAccumulatorsForTest) {
+      filteredCachedBatches().map{ batch =>
+        readBatches.add(1)
         batch
       }
-
-      val columnTypes = requestedColumnDataTypes.map {
-        case udt: UserDefinedType[_] => udt.sqlType
-        case other => other
-      }.toArray
-      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
-      if (enableAccumulatorsForTest && columnarIterator.hasNext) {
-        readPartitions.add(1)
+    } else {
+      filteredCachedBatches()
+    }.map{ batch =>
+      numOutputRows += batch.numRows
+      batch
+    }
+    val ret = serializer.decompressToRows(withMetrics, relOutput, attributes, conf)

Review comment:
       nit: Can we use better name instead of `ret`?




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656862075


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125633/
   Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660200667


   **[Test build #126062 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126062/testReport)** for PR 29067 at commit [`cb29e47`](https://github.com/apache/spark/commit/cb29e474fb7ad007734fd0a91a6329e7508e627a).


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665781333


   The test that failed appears to be unrelated to any of my changes


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



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


[GitHub] [spark] maropu commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659340965


   > Would that alleviate your concern about SPIP @maropu?
   
   Yea, the descison sounds reasonable to me.


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



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


[GitHub] [spark] cloud-fan commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659229267


   Yea IMO this is not a big feature that requires SPIP.
   
   At a high-level, this idea makes sense to me. With the columnar engine extension, the current table cache format may not be the most efficient. I'll review it closely this week.


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661120578


   **[Test build #126187 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126187/testReport)** for PR 29067 at commit [`46f4021`](https://github.com/apache/spark/commit/46f40216fd246f4e81cbaa6ce930287bf4c92975).


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688578327


   I think he just simply meant the naming of APIs such as `decompressColumnar` in the PR description are not matched to the actual APIs added (e.g., after https://github.com/apache/spark/pull/29067#pullrequestreview-457117863).


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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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.
   
   For now, we can just wrap it with `` `...` `` in this case to work around instead of `[[...]]`
   
   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 (via SBT) 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 show all warnings as 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:
users@infra.apache.org



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


[GitHub] [spark] HyukjinKwon removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688613392


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay to forgot matching. It happens.
   Let's just keep the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661101653


   **[Test build #126185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126185/testReport)** for PR 29067 at commit [`f0c7cfe`](https://github.com/apache/spark/commit/f0c7cfe82cd91db8611ea8abc9b91e4df65a35f7).


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



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


[GitHub] [spark] razajafri commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
razajafri commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-664630605


   @revans2 thank you for the quick turn around. Your latest patch has fixed that problem. :+1: 


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458064061



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,291 @@
+/*
+ * 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, StructType}
+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 {

Review comment:
       `SparkPlan` is so internal and I'd like to try harder to avoid leaking it to the developer APIs.
   
   After all, the cache builder only needs an RDD (either contains rows or columnar batches). We should keep the plan execution inside Spark and only exchange data (RDD) with the plugins.




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661916239


   **[Test build #126259 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126259/testReport)** for PR 29067 at commit [`c2d00dc`](https://github.com/apache/spark/commit/c2d00dc9ed77cd5d7e427ccbab8df10e1eefcbf4).


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662548365


   **[Test build #126347 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126347/testReport)** for PR 29067 at commit [`082e883`](https://github.com/apache/spark/commit/082e883cca80d87f2f0be3d38cac6afc0754de97).


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660198312


   I think I have addressed all of the outstanding review comments please take another look and see if the are more changes needed.


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



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463867525



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,96 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertColumnarBatchToCachedBatch(

Review comment:
       I think normally we won't create a `InMemoryRelation` in that way, so sounds good.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667249432


   **[Test build #126897 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126897/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666649236






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458975022



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       BTW if we want to unblock this PR, I'd suggest we do `isInstanceOf[ColumnarToRow]` first, and waiting for SPARK-32334 to extend it. I feel a bit worried about adding hacks to `QueryExecution`, as it's very critical.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656819397






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667634880


   **[Test build #126933 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126933/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661294364






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458974116



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       I like SPARK-32334 , maybe we should do that first.
   
   Say if we have a trait/abstract class for `ColumnarToRow`, then we can safely play `isInstanceOf` here, as your customized version of columnar to row operator should also extend the trait/abstract class.




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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667433678


   **[Test build #126902 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126902/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658172413






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666697587


   **[Test build #126819 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126819/testReport)** for PR 29067 at commit [`ea762e5`](https://github.com/apache/spark/commit/ea762e596b4653227835fed27f3093868b6c8d72).


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658423981


   **[Test build #125860 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125860/testReport)** for PR 29067 at commit [`ad11a79`](https://github.com/apache/spark/commit/ad11a794a634488f4b01b7e735179b74c2619774).


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



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


[GitHub] [spark] viirya commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456206490



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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
+   * generation is advised.
+   * @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 RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   * Which is repeated for each column in the original data.
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long =
+    Range.apply(4, stats.numFields, 5).map(stats.getLong).sum
+}
 
-  @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
 
-  val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
-  val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {

Review comment:
       Similar question, since it is supposed to be exposed, can we avoid `trait` here? 




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656862071


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660201563






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



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


[GitHub] [spark] dongjoon-hyun commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-668376771


   Hi, @revans2 and @cloud-fan .
   
   Unfortunately, this PR causes a flaky UT failure situation. Like the following, this causes failures depending on the test sequence.
   ```
   $ build/sbt "sql/testOnly *.CachedBatchSerializerSuite *.CachedTableSuite"
   ...
   [info] *** 30 TESTS FAILED ***
   [error] Failed: Total 51, Failed 30, Errors 0, Passed 21
   [error] Failed tests:
   [error] 	org.apache.spark.sql.CachedTableSuite
   [error] (sql/test:testOnly) sbt.TestsFailedException: Tests unsuccessful
   ```
   
   I made a PR to fix that. Currently, this causes random Jenkins failures on the other PRs.
   - https://github.com/apache/spark/pull/29344


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r461708141



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       Code generation is not currently supported for Columnar data except to transform it into rows before doing more code generation.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +241,92 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
+  def convertToColumnarIfPossible(plan: SparkPlan): SparkPlan = plan match {
+    case gen: WholeStageCodegenExec => gen.child match {
+      case c2r: ColumnarToRowTransition => c2r.child match {
+        case ia: InputAdapter => ia.child

Review comment:
       Right now yes.  Once we have more operators that support columnar processing it can change.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,336 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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
+
+  /**
+   * Decompress the cached data into a ColumnarBatch. This currently is only used if
+   * `supportsColumnar()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be decompressed.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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(

Review comment:
       How about `convertFromCache`? We are not actually reading data from anywhere but memory, so I am a little reluctant to say it is a read.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +85,32 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
-        if (enableAccumulatorsForTest) {
-          readBatches.add(1)
-        }
-        numOutputRows += batch.numRows
+    // update SQL metrics
+    val withMetrics = if (enableAccumulatorsForTest) {

Review comment:
       Sure I can make the change I did it because I thought it would be more efficient in the common case when we are not testing, but I could be wrong. My knowledge of the JVM JIT + Scala compilation is a little lacking in this area. I know that with Java a `static final boolean` will be inlined during JIT and checks for that value along with dead code paths are entirely removed.
   
   ```
   lazy val enableAccumulatorsForTest: Boolean = ...
   ```
   I believe is translated into a function in Scala so I don't think that the JVM JIT is able to do the same thing, but we are creating a closure for the map so that might allow it to inline it.
   
   Either way it is `O(number of batches)` so it should be much less of an issue compared to a row iterator.

##########
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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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

Review comment:
       Typically yes, but it could be a custom implementation, like what Parquet and Orc do.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667634347






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658317544


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125836/
   Test FAILed.


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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658337723


   @revans can you take a look at the failed test, one is caching test:
   
   [error] Failed tests:
   62375
   [error] 	org.apache.spark.sql.CachedTableSuite
   62376
   [error] 	org.apache.spark.sql.execution.columnar.PartitionBatchPruningSuite


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



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659306235


   Okay, I took a look too. I think the actual API here itself is not very big as it seems. It is a developer API. The idea makes sense to me too.
   
   I guess the main concern is that about leveraging DSv2 (?). I remember we discussed that `ColumnarToRowExec` <> Arrow code paths can merged as well a year ago when we add `ColumnarToRowExec`. It might be great if we have JIRAs and track these with an explicit plan rather than just indefinitely leaving half-baked APIs. With the explicit plans, I think we're good to go without SPIP. Would that alleviate your concern about SPIP @maropu?
   
   


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667634880


   **[Test build #126933 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126933/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667350723


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688614177


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay to forgot to match. It happens.
   Let's just match the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658171648


   **[Test build #125836 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125836/testReport)** for PR 29067 at commit [`641f28f`](https://github.com/apache/spark/commit/641f28f897db6d1bad709118e5701845a1898ae7).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667440292






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



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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r461865047



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks

Review comment:
       `returned` -> `returns`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be converted.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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 convertFromCacheColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+   * generation is advised.
+   * @param input the cached batches that should be converted.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def convertFromCache(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.

Review comment:
       Sorry, but this is technically a little misleading because `ColumnStats` trait doesn't have `upperBound` and `lowerBound`. Since the upperBound and lowerBound needs to hold a value, those are designed to be declared at the derived classes like `BooleanColumnStats`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be converted.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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 convertFromCacheColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+   * generation is advised.
+   * @param input the cached batches that should be converted.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def convertFromCache(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.

Review comment:
       Sorry, but this is technically a little misleading because `ColumnStats` trait itself doesn't have `upperBound` and `lowerBound`. Since the upperBound and lowerBound needs to hold a value, those are designed to be declared at the derived classes like `BooleanColumnStats`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be converted.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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 convertFromCacheColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+   * generation is advised.
+   * @param input the cached batches that should be converted.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def convertFromCache(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.

Review comment:
       Sorry, but this is technically a little misleading because `ColumnStats` trait itself doesn't have `upperBound` and `lowerBound`. Since `upperBound` and `lowerBound` needs to hold a concrete value, those are designed to be declared at the derived classes like `BooleanColumnStats`.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be converted.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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 convertFromCacheColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+   * generation is advised.
+   * @param input the cached batches that should be converted.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def convertFromCache(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   * Which is repeated for each column in the original data.
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long =
+    Range.apply(4, stats.numFields, 5).map(stats.getLong).sum
+}
+
+// Currently, only use statistics from atomic types except binary type only.

Review comment:
       nit. Maybe, `binary type only` -> `binary type`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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()` returned true for the associated schema, but there are other checks
+   * that can force row based output. One of the main advantages of doing columnar output over row
+   * based output is that the code generation is more standard and can be combined with code
+   * generation for downstream operations.
+   * @param input the cached batches that should be converted.
+   * @param cacheAttributes the attributes of the data in the batch.
+   * @param selectedAttributes the fields 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 convertFromCacheColumnar(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[ColumnarBatch]
+
+  /**
+   * Convert the cached batch into `InternalRow`s. If you want this to be performant, code
+   * generation is advised.
+   * @param input the cached batches that should be converted.
+   * @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 rows.
+   * @param conf the configuration for the job.
+   * @return RDD of the rows that were stored in the cached batches.
+   */
+  def convertFromCache(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
+
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ * The metrics are returned by the stats value. For each column in the batch 5 columns of metadata
+ * are needed in the row.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.

Review comment:
       Sorry, but this may be technically a little misleading because `ColumnStats` trait itself doesn't have `upperBound` and `lowerBound`. Since `upperBound` and `lowerBound` needs to hold a concrete value, those are designed to be declared at the derived classes like `BooleanColumnStats`. However, all instances having `ColumnStats` will hold them. Yes.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,186 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch

Review comment:
       nit. Indentation.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,186 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false
+
+  override def convertForCacheColumnar(
+      input: RDD[ColumnarBatch],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] =
+    throw new IllegalStateException("Columnar input is not supported")
+
+  override def convertForCache(input: RDD[InternalRow],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] = {
+    val batchSize = conf.columnBatchSize
+    val useCompression = conf.useCompression
+    convertForCacheInternal(input, schema, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(input: RDD[InternalRow],
+      output: Seq[Attribute],
+      batchSize: Int,
+      useCompression: Boolean): RDD[CachedBatch] = {
+    input.mapPartitionsInternal { rowIterator =>
+      new Iterator[DefaultCachedBatch] {
+        def next(): DefaultCachedBatch = {
+          val columnBuilders = output.map { attribute =>
+            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
+          }.toArray
+
+          var rowCount = 0
+          var totalSize = 0L
+          while (rowIterator.hasNext && rowCount < batchSize
+              && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
+            val row = rowIterator.next()
+
+            // Added for SPARK-6082. This assertion can be useful for scenarios when something
+            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
+            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
+            // hard to decipher.
+            assert(
+              row.numFields == columnBuilders.length,
+              s"Row column number mismatch, expected ${output.size} columns, " +
+                  s"but got ${row.numFields}." +
+                  s"\nRow content: $row")
+
+            var i = 0
+            totalSize = 0
+            while (i < row.numFields) {
+              columnBuilders(i).appendFrom(row, i)
+              totalSize += columnBuilders(i).columnStats.sizeInBytes
+              i += 1
+            }
+            rowCount += 1
+          }
+
+          val stats = InternalRow.fromSeq(
+            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
+          DefaultCachedBatch(rowCount, columnBuilders.map { builder =>
+            JavaUtils.bufferToArray(builder.build())
+          }, stats)
+        }
+
+        def hasNext: Boolean = rowIterator.hasNext
+      }
+    }
+  }
+
+  override def supportsColumnarOutput(schema: StructType): Boolean = schema.fields.forall(f =>
+    f.dataType match {
+        // More types can be supported, but this is to match the original implementation that
+        // only supported primitive types "for ease of review"

Review comment:
       indentation?

##########
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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(
+      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 `decompressColumnar()` be called instead of `decompressToRows()` 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 `decompressToRows()` 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

Review comment:
       Just a question. `The exact java types of the columns` means simply one of `OnHeapColumnVector` and `OffHeapColumnVector` here?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
##########
@@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl
   val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes)
 }
 
-private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {
+private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {

Review comment:
       Do we need this visibility change? For me, this is irrelevant to this PR. Can we remove this change from this PR? We had better make this change when it's inevitable.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
##########
@@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl
   val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes)
 }
 
-private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {
+private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {

Review comment:
       ~Do we need this visibility change? For me, this is irrelevant to this PR. Can we remove this change from this PR? We had better make this change when it's inevitable.~ Oh, got it. Sorry, I was confused.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
##########
@@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl
   val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes)
 }
 
-private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {
+private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {

Review comment:
       ```
   org.apache.spark.sql.columnar
   org.apache.spark.sql.execution.columnar
   ```

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnStats.scala
##########
@@ -32,7 +32,7 @@ private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializabl
   val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes)
 }
 
-private[columnar] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {
+private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable {

Review comment:
       ~Do we need this visibility change? For me, this is irrelevant to this PR. Can we remove this change from this PR? We had better make this change when it's inevitable.~ Oh, got it. Sorry, I was confused between two `columnar` packages.
   
   ```
   org.apache.spark.sql.columnar
   org.apache.spark.sql.execution.columnar
   ```
   

##########
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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(

Review comment:
       Although I know this is designed to add `Columnar` postfix at the end of `convertForCache` function, can we use `convertForColumnarCache` instead? For me, it sounds like more natural.

##########
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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(
+      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 convertForCacheColumnar(

Review comment:
       Although I know this is designed to add `Columnar` postfix at the end of `convertForCache` function, can we use `convertForColumnarCache` instead? For me, it sounds like more natural because we use `supportsColumnarInput` already.




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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688614177


   Just for a bit of more contexts, the commits are regularly digested and shared into the dev mailing list after being summarized. The PRs introducing APIs are usually directly shared into the mailing list with PR links, and likely people visit PRs. Or people take a look for PR descriptions when they track the history for example.
   
   It's best to match the change and PR description. It's okay to forgot to match. It happens.
   Let's just keep the PR description to what the PR proposes since we found it now.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-664693938






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662549137






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r462963211



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().map{ batch =>
         if (enableAccumulatorsForTest) {
           readBatches.add(1)
         }
         numOutputRows += batch.numRows
         batch
       }
-
-      val columnTypes = requestedColumnDataTypes.map {
-        case udt: UserDefinedType[_] => udt.sqlType
-        case other => other
-      }.toArray
-      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
-      if (enableAccumulatorsForTest && columnarIterator.hasNext) {
-        readPartitions.add(1)
+    val rows = serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf)
+    if (enableAccumulatorsForTest) {

Review comment:
       why `if (enableAccumulatorsForTest)` appears twice?




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665690606






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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656822272


   @revans2  can you add back in the PR template (### What changes were proposed in this pull request?, Why are the changes needed?, etc) and fill in the sections. that gets put into the commit history and is nice to have


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656818843


   **[Test build #125633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125633/testReport)** for PR 29067 at commit [`4bf995f`](https://github.com/apache/spark/commit/4bf995f35a4115302937eb04ccb99c90d9b3b681).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667433931






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



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


[GitHub] [spark] kiszk commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
kiszk commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r453257863



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala
##########
@@ -474,12 +484,8 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
 
   test("SPARK-22249: buildFilter should not throw exception when In contains an empty list") {
     val attribute = AttributeReference("a", IntegerType)()
-    val localTableScanExec = LocalTableScanExec(Seq(attribute), Nil)
-    val testRelation = InMemoryRelation(false, 1, MEMORY_ONLY, localTableScanExec, None,
-      LocalRelation(Seq(attribute), Nil))
-    val tableScanExec = InMemoryTableScanExec(Seq(attribute),
-      Seq(In(attribute, Nil)), testRelation)
-    assert(tableScanExec.partitionFilters.isEmpty)
+    val testSerializer = new TestCachedBatchSerializer(false, 1)
+    testSerializer.buildFilter(Seq(In(attribute, Nil)), Seq(attribute))

Review comment:
       Why is `assert` removed from this test?




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661293904


   **[Test build #126185 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126185/testReport)** for PR 29067 at commit [`f0c7cfe`](https://github.com/apache/spark/commit/f0c7cfe82cd91db8611ea8abc9b91e4df65a35f7).
    * This patch **fails PySpark pip packaging tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658316974


   **[Test build #125836 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125836/testReport)** for PR 29067 at commit [`641f28f`](https://github.com/apache/spark/commit/641f28f897db6d1bad709118e5701845a1898ae7).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666697587


   **[Test build #126819 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126819/testReport)** for PR 29067 at commit [`ea762e5`](https://github.com/apache/spark/commit/ea762e596b4653227835fed27f3093868b6c8d72).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667637986


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659444289


   I know that the arrow work was kind of dropped. Please let me know what you want to hand off to me, or if we should have a meeting and figure out what we want to do. I have a few ideas of what I thought it could look like, but I don't think this is the right place to have that discussion.


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457604604



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,291 @@
+/*
+ * 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, StructType}
+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 {

Review comment:
       Columnar to row transitions are inserted as a part of the `ApplyColumnarRulesAndInsertTransitions` rule in `QueryExecution.preparations`.  It looks for `SparkPlan` instances that output columnar data followed by ones that take row based data as input.  The rule inserts a `ColumnarToRowExec` transition between them.  This transition supports code generation so in normal operation it will then be wrapped in a `WholeStageCodeGenExec`. `ApplyColumnarRulesAndInsertTransitions` assumes that the output of any plan must be row based to conform with all of the other code that already makes that assumption.  It should be a minor change to be able to make that optional so `QueryExecution` could provide a new API that would return a `SparkPlan` that might be columnar and might not.
   
   I can put together an prototype of how that would look if you are OK with it.




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-664693363






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457530661



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,165 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+    extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def convertForCache(cachedPlan: SparkPlan): RDD[CachedBatch] = {
+    val batchSize = cachedPlan.conf.columnBatchSize
+    val useCompression = cachedPlan.conf.useCompression
+    convertForCacheInternal(cachedPlan, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(cachedPlan: SparkPlan,
+      batchSize: Int,
+      useCompression: Boolean): RDD[CachedBatch] = {
+    val output = cachedPlan.output
+    cachedPlan.execute().mapPartitionsInternal { rowIterator =>

Review comment:
       it will be very helpful if you can leave some comments for code that are just moved around, to speed up the review.




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660348522






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r462960267



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,344 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be

Review comment:
       We may need a more general word other than `compressing`/`decompressing`




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661600383






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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r462965356



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,186 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+  extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false
+
+  override def convertColumnarBatchToCachedBatch(
+      input: RDD[ColumnarBatch],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] =
+    throw new IllegalStateException("Columnar input is not supported")
+
+  override def convertInternalRowToCachedBatch(input: RDD[InternalRow],

Review comment:
       nit:
   ```
   def f(
       para1: T,
       para2: T,
       ...
   ```




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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666898969


   Looks getting closer to go to me too.


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r462965405



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,33 +19,186 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
-import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.columnar.{CachedBatch, CachedBatchSerializer, SimpleMetricsCachedBatch, SimpleMetricsCachedBatchSerializer}
+import org.apache.spark.sql.execution.{ColumnarToRowTransition, InputAdapter, QueryExecution, SparkPlan, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
-
+import org.apache.spark.util.{LongAccumulator, Utils}
 
 /**
- * CachedBatch is a cached batch of rows.
+ * The default implementation of CachedBatch.
  *
  * @param numRows The total number of rows in this batch
  * @param buffers The buffers for serialized columns
  * @param stats The stat of columns
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+  extends SimpleMetricsCachedBatch
+
+/**
+ * The default implementation of CachedBatchSerializer.
+ */
+class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer {
+  override def supportsColumnarInput(schema: Seq[Attribute]): Boolean = false
+
+  override def convertColumnarBatchToCachedBatch(
+      input: RDD[ColumnarBatch],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] =
+    throw new IllegalStateException("Columnar input is not supported")
+
+  override def convertInternalRowToCachedBatch(input: RDD[InternalRow],
+      schema: Seq[Attribute],
+      storageLevel: StorageLevel,
+      conf: SQLConf): RDD[CachedBatch] = {
+    val batchSize = conf.columnBatchSize
+    val useCompression = conf.useCompression
+    convertForCacheInternal(input, schema, batchSize, useCompression)
+  }
+
+  def convertForCacheInternal(input: RDD[InternalRow],

Review comment:
       ditto




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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659523306


   ok, sorry, I misunderstood, I thought you were referencing different discussion.
   I can file a jira for that and we can discuss if dsv2 can be leveraged. 


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667249432


   **[Test build #126897 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126897/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r457558925



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,291 @@
+/*
+ * 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, StructType}
+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 {

Review comment:
       The major goal was to expose some way to provide custom code for compressing/decompressing the cached data.  Decompression was already really fast and had APIs for columnar output vs non-columnar output so I mostly just kept them in place. 
   
   `convertForCache` does have a goal to be able to let someone process columnar input, but because of how the `SparkPlan` is created that is not a simple task. `QueryExecution.executedPlan` which is the source of the `SparkPlan` will always produce row based output. That was one of the design goals for the columnar execution work. It had to be truly transparent. Currently the only way to get around that is by walking the SparkPlan to check if the last stage is a code generation phase with only a child that is doing a columnar to row transition. If so then it is safe to remove that and process it directly as columnar. If you want me to create a trait for `SparkPlan` that exposes a simpler API just for caching, I can, but it will not be able to expose anything columnar unless we make larger changes. I am fine with making those larger changes, I just want to be sure that the community is willing to accept them conceptually before I start to touch a lot of the code.




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



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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
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.
   
   For now, we can just wrap it with `` `...` `` in this case to work around instead of `[[...]]`
   
   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:
users@infra.apache.org



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458163165



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       I picked row based input as a minimum requirement because the majority of the time the output of the plan will be row based. I also wanted to avoid unnecessary transitions from one data format to another. Each time one tries to change the format of the data, row to column, column to row, or even column of one type to column of another there is a cost involved. As a rule of thumb it is ideal to reduce those transformations when ever possible. So in that respect columnar input support is a performance optimization in the rare case that columnar input is a possibility.
   
   




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



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


[GitHub] [spark] cloud-fan commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688845373


   @revans2 thanks for updating!


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667346251


   The test failure looks unrelated
   ```
   [info] KafkaSourceStressSuite:
   [info] - stress test with multiple topics and partitions *** FAILED *** (1 minute, 1 second)
   [info]   Timed out waiting for stream: The code passed to failAfter did not complete within 30 seconds.
   ```


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661121310






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



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


[GitHub] [spark] maropu edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
maropu edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659340965


   > Would that alleviate your concern about SPIP @maropu?
   
   Yea, the approach that you suggested sounds reasonable to me. Thanks for sum up, @HyukjinKwon .


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661916239


   **[Test build #126259 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126259/testReport)** for PR 29067 at commit [`c2d00dc`](https://github.com/apache/spark/commit/c2d00dc9ed77cd5d7e427ccbab8df10e1eefcbf4).


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667440271


   **[Test build #126902 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126902/testReport)** for PR 29067 at commit [`3f2f527`](https://github.com/apache/spark/commit/3f2f5278c7e8c05e32a5f54ddba1251250dfdca3).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `class ColumnStatisticsSchema(a: Attribute) extends Serializable `
     * `class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable `


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659464261


   @tgravescs, I think he wanted to know the plan and how we will merge these into one codebase with DSv2 in the end. Looks valid concern to me and I guess this is his main concern and why he asked SPIP. I don't have or know the exact plan or I haven't used the new codes by myself before.
   
   Maybe I am remembering wrongly but I thought we addressed the comments from @BryanCutler by leaving these as a future work IIRC. The Arrow one is actually still in the comments:
   
   https://github.com/apache/spark/blob/a4382f7fe1c36a51c64f460c6cb91e93470e0825/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala#L56-L58
   
   I am willing to help this too but would like to ask the plan or at least file the corresponding JIRAs to track as well.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688580536


   But the config is still around sterilization, and despite the name of the APIs compression is optional. If you want me to add compression to the name of the pr I can. 


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458069198



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala
##########
@@ -0,0 +1,325 @@
+/*
+ * 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 for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait CachedBatchSerializer extends Serializable {
+  /**
+   * Can `convertForCacheColumnar()` be called instead of `convertForCache()` 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 convertForCache(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 convertForCacheColumnar(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],

Review comment:
       ditto




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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667350731


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126897/
   Test FAILed.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-657544856


   @maropu 
   
   > You cannot use the data source V2 interface for your purpose?
   
   We want to produce a transparent replacement for `.cache`, `.persist` and the SQL `CACHE` operator using GPUs for acceleration.  Caching data right now is slow on the CPU.  `.cache` is a common enough operation that we would like to support it in our plugin and feel that it is something that we can really accelerate.  In theory, we could reuse the datasource v2 API, but it would require a lot more refactoring to make it fit into the cache operator. Possibly refactoring of the data source V2 API as well. If that is what you think we need to do I can work on it, but it will be a much bigger change.
   
   > The current cache structure is tightly coupled to the Spark internal one, so I'm not sure that we can directly expose it to 3rd-party developers.
   
   Yes, many of the APIs used by this code are internal to Spark and are subject to change at any moment. That is no different than with many other plugin APIs, like the ones we use to enable our GPU accelerated dataframe plugin. Or the callback APIs currently used for metrics. If you would like me to do more to document that these APIs, and the APIs that they depend on, are unstable and can change I am happy to do it.
   
   > If `SPARK_CACHE_SERIALIZER` specified, does the current approach replace all the existing caching operations with custom ones?
   
    `SPARK_CACHE_SERIALIZER` is a static conf so if you specify it, it will be used for all cache operations within that session. But cache is not shared between sessions so I felt that was reasonable because there was no regression in functionality.
   
   > Users cannot select which cache structure (default or custom) is used on runtime?
   
   A user cannot switch between modes within the same session, but each spark session can have a separate setting. The goal of this was to prevent users from changing the setting after caching data. In theory, the code as it is written should be able to handle changing the setting at any point in time, except for the code to load the plugin as a singleton. That use case was not one that I currently am concerned about so I didn't want to add in tests for it nor commit to try and support it. If you really want this use case to be supported I can make the needed changes and test/document it.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660348522






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



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


[GitHub] [spark] cloud-fan commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-688588321


   The `CachedBatchSerializer` has methods like `convertColumnarBatchToCachedBatch`, but the PR description is stale amd still says `decompressColumnar`.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660347743


   **[Test build #126062 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126062/testReport)** for PR 29067 at commit [`cb29e47`](https://github.com/apache/spark/commit/cb29e474fb7ad007734fd0a91a6329e7508e627a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `trait CachedBatch `
     * `trait CachedBatchSerializer extends Serializable `
     * `trait SimpleMetricsCachedBatch extends CachedBatch `
     * `abstract class SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging `


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659141249


   @maropu, per the documentation [Spark Project Improvement Proposals (SPIP)](http://spark.apache.org/improvement-proposals.html), if you feel like it needs an SPIP, it does. I trust your judgement.
   
   I will read it more closely today and provide more feedback. cc @kiszk, @srowen, @viirya, @ueshin too.


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



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


[GitHub] [spark] HyukjinKwon commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659306235


   Okay, I took a look too. I think the actual API here itself is not very big as it seems. It is a developer API. The idea makes sense to me too.
   
   I guess the main concern is that about leveraging DSv2 (?). I remember `ColumnarToRowExec` <> Arrow batches can merged as well around one year ago. It might be great if we have JIRAs and track these with an explicit plan rather than just indefinitely leaving half-baked APIs. With the explicit plans, I think we're good to go without SPIP. Would that alleviate your concern about SPIP @maropu?
   
   


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656910151


   **[Test build #125642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125642/testReport)** for PR 29067 at commit [`0361237`](https://github.com/apache/spark/commit/0361237986e10eec828cacb400ee6c948fee30fd).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-660201563






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



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


[GitHub] [spark] tgravescs commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659430254


   thanks everyone for the feedback. @HyukjinKwon I assume you and @maropu  will file those jiras as it appears you have the most context there?  
   


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665691334






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-666878070






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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658513850






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661355673


   **[Test build #126204 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126204/testReport)** for PR 29067 at commit [`38ca741`](https://github.com/apache/spark/commit/38ca74124a5d9552fa83bf228258db7bd58595e7).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661600383


   Merged build finished. Test FAILed.


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661120578


   **[Test build #126187 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126187/testReport)** for PR 29067 at commit [`46f4021`](https://github.com/apache/spark/commit/46f40216fd246f4e81cbaa6ce930287bf4c92975).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656960105


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/125642/
   Test FAILed.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656862071






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662085100


   **[Test build #126259 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126259/testReport)** for PR 29067 at commit [`c2d00dc`](https://github.com/apache/spark/commit/c2d00dc9ed77cd5d7e427ccbab8df10e1eefcbf4).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-665828006






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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667637988


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126933/
   Test FAILed.


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



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


[GitHub] [spark] HyukjinKwon edited a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-659464261


   @tgravescs, I think he wanted to know the plan and how we will merge these into one codebase with DSv2 in the end. Looks valid concern to me and I guess this is his main concern and why he asked SPIP. I don't have or know the exact plan or I haven't used the new codes by myself before.
   
   Maybe I am remembering wrongly but I thought we addressed the comments from @BryanCutler by leaving these as a future work IIRC. The Arrow one is actually still in the comments:
   
   https://github.com/apache/spark/blob/a4382f7fe1c36a51c64f460c6cb91e93470e0825/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala#L56-L58
   
   I am willing to help this too but would like to ask the plan or at least file the corresponding JIRAs to track since you guys are leading these areas.
   
   +I think we roughly agree on merging the codes into a single code base eventually (?). Let me know if I understood wrongly.


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



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


[GitHub] [spark] revans2 commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661051332


   > Can we add additional API for saying whether a given type is cached or not to `CachedBatchSerializer`?
   
   Sure I'll get on it. 


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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662548365


   **[Test build #126347 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126347/testReport)** for PR 29067 at commit [`082e883`](https://github.com/apache/spark/commit/082e883cca80d87f2f0be3d38cac6afc0754de97).


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662549137






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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661160094


   **[Test build #126191 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126191/testReport)** for PR 29067 at commit [`10b0ce1`](https://github.com/apache/spark/commit/10b0ce193d2f711ddc5702465c847d0a94cde313).


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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-662719849


   **[Test build #126347 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126347/testReport)** for PR 29067 at commit [`082e883`](https://github.com/apache/spark/commit/082e883cca80d87f2f0be3d38cac6afc0754de97).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `case class SingleIntCachedBatch(data: Array[Int]) extends CachedBatch `
     * `class TestSingleIntColumnarCachedBatchSerializer extends CachedBatchSerializer `


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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656960099






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



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


[GitHub] [spark] SparkQA removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661101653


   **[Test build #126185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/126185/testReport)** for PR 29067 at commit [`f0c7cfe`](https://github.com/apache/spark/commit/f0c7cfe82cd91db8611ea8abc9b91e4df65a35f7).


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



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-661600387


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/126204/
   Test FAILed.


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458215059



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       Sorry I didn't explain.  I wanted to avoid reflection if possible, because our GPU plugin can replace `ColumnarToRow` with a version that also pulls data back from the `GPU`, and with (SPARK-32334)[https://issues.apache.org/jira/browse/SPARK-32334] there is the possibility around that more things related to columnar transitions might change. So instead of trying to pick apart the incoming `SparkPlan` to remove the `ColumnarToRow`, I added an API to `QueryExecution` so it can generate a plan without the transition in it.
   
   If you would prefer me to try and pick apart the plan I can do that instead. I just thought this was a cleaner approach.
   
   




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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r456521706



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -19,84 +19,301 @@ package org.apache.spark.sql.execution.columnar
 
 import org.apache.commons.lang3.StringUtils
 
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.internal.Logging
 import org.apache.spark.network.util.JavaUtils
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
+import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan}
 import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
 import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
+import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
+import org.apache.spark.sql.types.{AtomicType, BinaryType, StructType, UserDefinedType}
+import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector}
 import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.{LongAccumulator, Utils}
 
+/**
+ * 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
+}
 
 /**
- * CachedBatch is a cached batch of rows.
- *
- * @param numRows The total number of rows in this batch
- * @param buffers The buffers for serialized columns
- * @param stats The stat of columns
+ * Provides APIs for compressing, filtering, and decompressing SQL data that will be
+ * persisted/cached.
  */
-private[columnar]
-case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)
+@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 what
+   * you need with the added expense of calculating the min and max value for some
+   * data columns, depending on their data type. 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
+   * generation is advised.
+   * @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 RDD of the rows that were stored in the cached batches.
+   */
+  def decompressToRows(
+      input: RDD[CachedBatch],
+      cacheAttributes: Seq[Attribute],
+      selectedAttributes: Seq[Attribute],
+      conf: SQLConf): RDD[InternalRow]
+}
 
-case class CachedRDDBuilder(
-    useCompression: Boolean,
-    batchSize: Int,
-    storageLevel: StorageLevel,
-    @transient cachedPlan: SparkPlan,
-    tableName: Option[String]) {
+/**
+ * A [[CachedBatch]] that stores some simple metrics that can be used for filtering of batches with
+ * the [[SimpleMetricsCachedBatchSerializer]].
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatch extends CachedBatch {
+  /**
+   * Holds the same as ColumnStats.
+   * upperBound (optional), lowerBound (Optional), nullCount: Int, rowCount: Int, sizeInBytes: Long
+   * Which is repeated for each column in the original data.
+   */
+  val stats: InternalRow
+  override def sizeInBytes: Long =
+    Range.apply(4, stats.numFields, 5).map(stats.getLong).sum
+}
 
-  @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null
+// Currently, only use statistics from atomic types except binary type only.
+private object ExtractableLiteral {
+  def unapply(expr: Expression): Option[Literal] = expr match {
+    case lit: Literal => lit.dataType match {
+      case BinaryType => None
+      case _: AtomicType => Some(lit)
+      case _ => None
+    }
+    case _ => None
+  }
+}
 
-  val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
-  val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator
+/**
+ * Provides basic filtering for [[CachedBatchSerializer]] implementations.
+ */
+@DeveloperApi
+@Since("3.1.0")
+trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging {

Review comment:
       That I can make an abstract class




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-656861989


   **[Test build #125633 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125633/testReport)** for PR 29067 at commit [`4bf995f`](https://github.com/apache/spark/commit/4bf995f35a4115302937eb04ccb99c90d9b3b681).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `    .doc(\"The name of a class that implements \" +`
     * `trait CachedBatch `
     * `trait CachedBatchSerializer extends Serializable `
     * `trait SimpleMetricsCachedBatch extends CachedBatch `
     * `trait SimpleMetricsCachedBatchSerializer extends CachedBatchSerializer with Logging `
     * `case class DefaultCachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow)`
     * `class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer `
     * `case class CachedRDDBuilder(`


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



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


[GitHub] [spark] cloud-fan commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r458195360



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
##########
@@ -85,77 +232,81 @@ case class CachedRDDBuilder(
   }
 
   private def buildBuffers(): RDD[CachedBatch] = {
-    val output = cachedPlan.output
-    val cached = cachedPlan.execute().mapPartitionsInternal { rowIterator =>
-      new Iterator[CachedBatch] {
-        def next(): CachedBatch = {
-          val columnBuilders = output.map { attribute =>
-            ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression)
-          }.toArray
-
-          var rowCount = 0
-          var totalSize = 0L
-          while (rowIterator.hasNext && rowCount < batchSize
-            && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE) {
-            val row = rowIterator.next()
-
-            // Added for SPARK-6082. This assertion can be useful for scenarios when something
-            // like Hive TRANSFORM is used. The external data generation script used in TRANSFORM
-            // may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
-            // hard to decipher.
-            assert(
-              row.numFields == columnBuilders.length,
-              s"Row column number mismatch, expected ${output.size} columns, " +
-                s"but got ${row.numFields}." +
-                s"\nRow content: $row")
-
-            var i = 0
-            totalSize = 0
-            while (i < row.numFields) {
-              columnBuilders(i).appendFrom(row, i)
-              totalSize += columnBuilders(i).columnStats.sizeInBytes
-              i += 1
-            }
-            rowCount += 1
-          }
-
-          sizeInBytesStats.add(totalSize)
-          rowCountStats.add(rowCount)
-
-          val stats = InternalRow.fromSeq(
-            columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq)
-          CachedBatch(rowCount, columnBuilders.map { builder =>
-            JavaUtils.bufferToArray(builder.build())
-          }, stats)
-        }
-
-        def hasNext: Boolean = rowIterator.hasNext
-      }
+    val cb = if (cachedPlan.supportsColumnar) {
+      serializer.convertForCacheColumnar(cachedPlan.executeColumnar(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    } else {
+      serializer.convertForCache(cachedPlan.execute(),
+        cachedPlan.output,
+        storageLevel,
+        cachedPlan.conf)
+    }
+    val cached = cb.map { batch =>
+      sizeInBytesStats.add(batch.sizeInBytes)
+      rowCountStats.add(batch.numRows)
+      batch
     }.persist(storageLevel)
-
     cached.setName(cachedName)
     cached
   }
 }
 
 object InMemoryRelation {
 
+  private[this] var ser: Option[CachedBatchSerializer] = None
+  private[this] def getSerializer(sqlConf: SQLConf): CachedBatchSerializer = synchronized {
+    if (ser.isEmpty) {
+      val serName = sqlConf.getConf(StaticSQLConf.SPARK_CACHE_SERIALIZER)
+      val serClass = Utils.classForName(serName)
+      val instance = serClass.getConstructor().newInstance().asInstanceOf[CachedBatchSerializer]
+      ser = Some(instance)
+    }
+    ser.get
+  }
+
   def apply(
-      useCompression: Boolean,
-      batchSize: Int,
+      storageLevel: StorageLevel,
+      qe: QueryExecution,
+      tableName: Option[String]): InMemoryRelation = {
+    val optimizedPlan = qe.optimizedPlan
+    val serializer = getSerializer(optimizedPlan.conf)
+    val child = if (serializer.supportsColumnarInput(optimizedPlan.output)) {

Review comment:
       That makes sense.
   
   Then my question is: how do we know a plan can produce columnar data? I see you added quite a bit of changes for it, but I've figured it out yet.
   
   I expect something like
   ```
   plan.isInstanceOf[WholeStageCodegen] && plan.children.head.isInstanceOf[ColumnarToRow]
   ```
   
   Maybe I missed something. Can you elaborate it a bit more?




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



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


[GitHub] [spark] SparkQA commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-658423981


   **[Test build #125860 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/125860/testReport)** for PR 29067 at commit [`ad11a79`](https://github.com/apache/spark/commit/ad11a794a634488f4b01b7e735179b74c2619774).


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



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


[GitHub] [spark] revans2 commented on a change in pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
revans2 commented on a change in pull request #29067:
URL: https://github.com/apache/spark/pull/29067#discussion_r463004283



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala
##########
@@ -130,34 +87,29 @@ case class InMemoryTableScanExec(
     val numOutputRows = longMetric("numOutputRows")
     // Using these variables here to avoid serialization of entire objects (if referenced
     // directly) within the map Partitions closure.
-    val relOutput: AttributeSeq = relation.output
-
-    filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator =>
-      // Find the ordinals and data types of the requested columns.
-      val (requestedColumnIndices, requestedColumnDataTypes) =
-        attributes.map { a =>
-          relOutput.indexOf(a.exprId) -> a.dataType
-        }.unzip
+    val relOutput = relation.output
+    val serializer = relation.cacheBuilder.serializer
 
-      // update SQL metrics
-      val withMetrics = cachedBatchIterator.map { batch =>
+    // update SQL metrics
+    val withMetrics =
+      filteredCachedBatches().map{ batch =>
         if (enableAccumulatorsForTest) {
           readBatches.add(1)
         }
         numOutputRows += batch.numRows
         batch
       }
-
-      val columnTypes = requestedColumnDataTypes.map {
-        case udt: UserDefinedType[_] => udt.sqlType
-        case other => other
-      }.toArray
-      val columnarIterator = GenerateColumnAccessor.generate(columnTypes)
-      columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray)
-      if (enableAccumulatorsForTest && columnarIterator.hasNext) {
-        readPartitions.add(1)
+    val rows = serializer.convertCachedBatchToInternalRow(withMetrics, relOutput, attributes, conf)
+    if (enableAccumulatorsForTest) {

Review comment:
       The original code has it appear three times just like this code does.
   
   1) Zero out the metrics
   https://github.com/apache/spark/blob/743772095273b464f845efefb3eb59284b06b9be/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L125
   
   2) check if it should update `readBatches`
   https://github.com/apache/spark/blob/743772095273b464f845efefb3eb59284b06b9be/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L144
   
   3) check if it should update `readPartitions`
   https://github.com/apache/spark/blob/743772095273b464f845efefb3eb59284b06b9be/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L157
   
   I cannot get rid of those three.
   
   I did rearrange things so there is only one `mapPartitionsInternal` call (which is what I think you are really asking for), but to do it I had to move the `readPartitions` check from being after deserialization to before it. The issue is that the high level API we have for deserialization takes an `RDD` and returns an `RDD` so I cannot  do what the original code did because the API they used took an Iterator and returned an Iterator.
   




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



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


[GitHub] [spark] AmplabJenkins commented on pull request #29067: [SPARK-32274][SQL] Make SQL cache serialization pluggable

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29067:
URL: https://github.com/apache/spark/pull/29067#issuecomment-667350723






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



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