-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-32274][SQL] Make SQL cache serialization pluggable #29067
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 7 commits
4bf995f
427fc68
0361237
641f28f
f989572
ad11a79
cb29e47
c7c89bb
f0c7cfe
46f4021
10b0ce1
38ca741
c2d00dc
082e883
e58783e
05a430d
e0f5398
c9b1488
4ec28bb
7840a4c
5723aa9
f8ca4af
ea762e5
ce66042
3f2f527
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||
|---|---|---|---|---|
| @@ -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 { | ||||
cloud-fan marked this conversation as resolved.
Show resolved
Hide resolved
|
||||
| /** | ||||
| * 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 | ||||
|
||||
| .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/sql/catalyst"))) |
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.
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit. Maybe, binary type only -> binary type?
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
super nit:
def func(
para1: T,
para2: T): R = ...
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Attri | |
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} | ||
|
|
||
| private[columnar] class ColumnStatisticsSchema(a: Attribute) extends Serializable { | ||
| private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { | ||
| val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() | ||
| val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() | ||
| val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() | ||
|
|
@@ -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 { | ||
|
||
| val (forAttribute: AttributeMap[ColumnStatisticsSchema], schema: Seq[AttributeReference]) = { | ||
| val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) | ||
| (AttributeMap(allStats), allStats.flatMap(_._2.schema)) | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We may need a more general word other than
compressing/decompressing