From 6a34b1582d280f3817ab2d4676fbbf9c1982488d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 7 Mar 2016 09:37:37 -0500 Subject: [PATCH 01/47] improve the doc for "spark.memory.offHeap.size" --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 9b9583d9165e..832cfa743b85 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1073,7 +1073,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.size 0 - The absolute amount of memory in bytes which can be used for off-heap allocation. + The absolute amount of memory (in terms by bytes) which can be used for off-heap allocation. This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true. From 3a2be8cf4dcc3b93715a6e1f42bd41146cd31fa0 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 7 Mar 2016 14:00:16 -0500 Subject: [PATCH 02/47] fix --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 832cfa743b85..9b9583d9165e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1073,7 +1073,7 @@ Apart from these, the following properties are also available, and may be useful spark.memory.offHeap.size 0 - The absolute amount of memory (in terms by bytes) which can be used for off-heap allocation. + The absolute amount of memory in bytes which can be used for off-heap allocation. This setting has no impact on heap memory usage, so if your executors' total memory consumption must fit within some hard limit then be sure to shrink your JVM heap size accordingly. This must be set to a positive value when spark.memory.offHeap.enabled=true. From eb535d61e04682eaf179a11f5695456d2ead5552 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 27 Oct 2017 10:36:08 -0700 Subject: [PATCH 03/47] add configuration for partition_metadata --- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4eda9f337953..1c5f07973ace 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -136,7 +136,7 @@ object SQLConf { val IN_MEMORY_PARTITION_PRUNING = buildConf("spark.sql.inMemoryColumnarStorage.partitionPruning") .internal() - .doc("When true, enable partition pruning for in-memory columnar tables.") + .doc("When true, enable partition batch pruning for in-memory columnar tables.") .booleanConf .createWithDefault(true) @@ -144,6 +144,12 @@ object SQLConf { buildConf("spark.sql.columnVector.offheap.enabled") .internal() .doc("When true, use OffHeapColumnVector in ColumnarBatch.") + + val IN_MEMORY_PARTITION_METADATA = + buildConf("spark.sql.inMemoryColumnarStorage.partitionMetadata") + .internal() + .doc("When true, spark sql will collect partition level stats for in-memory columnar" + + " tables and do coarse-grained pruning") .booleanConf .createWithDefault(false) From f910b2786bfcabd4d056424751e5e7abe1c28d91 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 27 Oct 2017 12:49:33 -0700 Subject: [PATCH 04/47] framework of CachedColumnarRDD --- .../columnar/CachedColumnarRDD.scala | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala new file mode 100644 index 000000000000..6ddcf519873c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -0,0 +1,62 @@ +/* + * 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.execution.columnar + +import scala.reflect.ClassTag + +import org.apache.spark.{Dependency, Partition, SparkContext, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow + +private[columnar] class CachedColumnarRDDPartition( + partitionIndex: Int, + columnnStats: Array[InternalRow]) extends Partition { + + override def index: Int = partitionIndex + + def columnStats: Array[InternalRow] = columnnStats +} + +private[columnar] class CachedColumnarRDD[T: ClassTag]( + @transient private var _sc: SparkContext, + @transient private var deps: Seq[Dependency[_]], + dataRDD: RDD[T], + partitionStats: Array[Array[InternalRow]]) extends RDD[T](_sc, deps) { + + /** + * :: DeveloperApi :: + * Implemented by subclasses to compute a given partition. + */ + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + Iterator() + } + + /** + * Implemented by subclasses to return the set of partitions in this RDD. This method will only + * be called once, so it is safe to implement a time-consuming computation in it. + * + * The partitions in this array must satisfy the following property: + * `rdd.partitions.zipWithIndex.forall { case (partition, index) => partition.index == index }` + */ + override protected def getPartitions: Array[Partition] = { + partitionStats.zipWithIndex.map { + case (statsRow, index) => + new CachedColumnarRDDPartition(index, statsRow) + } + } +} From 846b032548b72348ddc8f4107359d721d816b86a Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 27 Oct 2017 15:53:38 -0700 Subject: [PATCH 05/47] code framework --- .../columnar/CachedColumnarRDD.scala | 14 +- .../execution/columnar/InMemoryRelation.scala | 196 ++++++++++++++---- .../columnar/InMemoryTableScanExec.scala | 59 ++++-- 3 files changed, 196 insertions(+), 73 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 6ddcf519873c..5af2e26563ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -25,25 +25,25 @@ import org.apache.spark.sql.catalyst.InternalRow private[columnar] class CachedColumnarRDDPartition( partitionIndex: Int, - columnnStats: Array[InternalRow]) extends Partition { + columnnStats: InternalRow) extends Partition { override def index: Int = partitionIndex - def columnStats: Array[InternalRow] = columnnStats + def columnStats: InternalRow = columnnStats } -private[columnar] class CachedColumnarRDD[T: ClassTag]( +private[columnar] class CachedColumnarRDD( @transient private var _sc: SparkContext, @transient private var deps: Seq[Dependency[_]], - dataRDD: RDD[T], - partitionStats: Array[Array[InternalRow]]) extends RDD[T](_sc, deps) { + dataRDD: RDD[CachedBatch], + partitionStats: Array[InternalRow]) extends RDD[CachedBatch](_sc, deps) { /** * :: DeveloperApi :: * Implemented by subclasses to compute a given partition. */ - override def compute(split: Partition, context: TaskContext): Iterator[T] = { - Iterator() + override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { + dataRDD.iterator(split, context) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index a1c62a729900..248ef9008d57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -50,7 +50,138 @@ object InMemoryRelation { * @param stats The stat of columns */ private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: Option[InternalRow]) + +private[columnar] class CachedPartitionIterator( + rowIterator: Iterator[InternalRow], + output: Seq[Attribute], + batchSize: Int, + useCompression: Boolean, + batchStats: LongAccumulator) extends Iterator[(CachedBatch, InternalRow)] { + + def next(): (CachedBatch, InternalRow) = { + val columnBuilders = output.map { attribute => + ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) + }.toArray + + var rowCount = 0 + var totalSize = 0L + while (rowIterator.hasNext) { + 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 + } + + batchStats.add(totalSize) + val stats = InternalRow.fromSeq(columnBuilders.flatMap(_.columnStats.collectedStatistics)) + (CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, None), stats) + /* + 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 + } + + batchStats.add(totalSize) + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics)) + CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, stats) + */ + } + + def hasNext: Boolean = rowIterator.hasNext +} + +private[columnar] class CachedBatchIterator( + rowIterator: Iterator[InternalRow], + output: Seq[Attribute], + batchSize: Int, + useCompression: Boolean, + batchStats: LongAccumulator) extends 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 + } + + batchStats.add(totalSize) + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics)) + CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build()) + }, Some(stats)) + } + + def hasNext: Boolean = rowIterator.hasNext +} case class InMemoryRelation( output: Seq[Attribute], @@ -69,6 +200,8 @@ case class InMemoryRelation( @transient val partitionStatistics = new PartitionStatistics(output) + private val usePartitionLevelMetadata = conf.inMemoryPartitionMetadata + override def computeStats(): Statistics = { if (batchStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, no useful statistics information @@ -87,56 +220,29 @@ case class InMemoryRelation( private def buildBuffers(): Unit = { val output = child.output - val cached = child.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 - } - - batchStats.add(totalSize) - - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics)) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - def hasNext: Boolean = rowIterator.hasNext + // TODO: + val batchedRDD = child.execute().mapPartitionsInternal { rowIterator => + if (!usePartitionLevelMetadata) { + new CachedBatchIterator(rowIterator, output, batchSize, useCompression, batchStats) + } else { + new CachedPartitionIterator(rowIterator, output, batchSize, useCompression, batchStats) } - }.persist(storageLevel) + } + + val cached = if (!usePartitionLevelMetadata) { + batchedRDD.persist(storageLevel) + } else { + val r = batchedRDD.map(_.asInstanceOf[(CachedBatch, InternalRow)]) + val partitionLevelStats = r.map(_._2).collect() + new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD.dependencies, r.map(_._1), + partitionLevelStats) + } cached.setName( tableName.map(n => s"In-memory table $n") .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnBuffers = cached + _cachedColumnBuffers = cached.asInstanceOf[RDD[CachedBatch]] } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 3e73393b1285..a4f6732e3f72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.execution.vectorized._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -200,30 +201,46 @@ case class InMemoryTableScanExec( val schemaIndex = schema.zipWithIndex val buffers = relation.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - 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" + if (sqlContext.conf.inMemoryPartitionMetadata) { + val cachedColumnarRDD = buffers.asInstanceOf[CachedColumnarRDD] + val partitions = cachedColumnarRDD.partitions.map(_.asInstanceOf[CachedColumnarRDDPartition]) + buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(index) + if (!partitionFilter.eval(partitions(index).columnStats)) { + Iterator() + } else { + cachedBatchIterator + } + } + } else { + buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(index) + + // Do partition batch pruning if enabled + if (inMemoryPartitionPruningEnabled) { + cachedBatchIterator.filter { cachedBatch => + if (!partitionFilter.eval(cachedBatch.stats.get)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true } - false - } else { - true } + } else { + cachedBatchIterator } - } else { - cachedBatchIterator } } } From f511b6f6af1d751e20acec784d7c82b16f951591 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 30 Oct 2017 13:41:18 -0700 Subject: [PATCH 06/47] remove cachedcolumnarbatchRDD --- .../apache/spark/sql/internal/SQLConf.scala | 4 + .../columnar/CachedColumnarRDD.scala | 62 --------------- .../execution/columnar/InMemoryRelation.scala | 71 ++++++----------- .../columnar/InMemoryTableScanExec.scala | 79 ++++++++++--------- 4 files changed, 68 insertions(+), 148 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1c5f07973ace..806c151481cd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -144,6 +144,8 @@ object SQLConf { buildConf("spark.sql.columnVector.offheap.enabled") .internal() .doc("When true, use OffHeapColumnVector in ColumnarBatch.") + .booleanConf + .createWithDefault(false) val IN_MEMORY_PARTITION_METADATA = buildConf("spark.sql.inMemoryColumnarStorage.partitionMetadata") @@ -1225,6 +1227,8 @@ class SQLConf extends Serializable with Logging { def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED) + def inMemoryPartitionMetadata: Boolean = getConf(IN_MEMORY_PARTITION_METADATA) + def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD) def broadcastTimeout: Long = getConf(BROADCAST_TIMEOUT) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala deleted file mode 100644 index 5af2e26563ff..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.execution.columnar - -import scala.reflect.ClassTag - -import org.apache.spark.{Dependency, Partition, SparkContext, TaskContext} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow - -private[columnar] class CachedColumnarRDDPartition( - partitionIndex: Int, - columnnStats: InternalRow) extends Partition { - - override def index: Int = partitionIndex - - def columnStats: InternalRow = columnnStats -} - -private[columnar] class CachedColumnarRDD( - @transient private var _sc: SparkContext, - @transient private var deps: Seq[Dependency[_]], - dataRDD: RDD[CachedBatch], - partitionStats: Array[InternalRow]) extends RDD[CachedBatch](_sc, deps) { - - /** - * :: DeveloperApi :: - * Implemented by subclasses to compute a given partition. - */ - override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { - dataRDD.iterator(split, context) - } - - /** - * Implemented by subclasses to return the set of partitions in this RDD. This method will only - * be called once, so it is safe to implement a time-consuming computation in it. - * - * The partitions in this array must satisfy the following property: - * `rdd.partitions.zipWithIndex.forall { case (partition, index) => partition.index == index }` - */ - override protected def getPartitions: Array[Partition] = { - partitionStats.zipWithIndex.map { - case (statsRow, index) => - new CachedColumnarRDDPartition(index, statsRow) - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 248ef9008d57..2f5cddd3d8df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -57,9 +57,13 @@ private[columnar] class CachedPartitionIterator( output: Seq[Attribute], batchSize: Int, useCompression: Boolean, - batchStats: LongAccumulator) extends Iterator[(CachedBatch, InternalRow)] { + batchStats: LongAccumulator) extends Iterator[AnyRef] { - def next(): (CachedBatch, InternalRow) = { + private var partitionStats: InternalRow = _ + + private var fetchingFirstElement = true + + private def buildCachedBatch(): Option[CachedBatch] = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -88,45 +92,24 @@ private[columnar] class CachedPartitionIterator( } rowCount += 1 } - + partitionStats = InternalRow.fromSeq(columnBuilders.flatMap(_.columnStats.collectedStatistics)) batchStats.add(totalSize) - val stats = InternalRow.fromSeq(columnBuilders.flatMap(_.columnStats.collectedStatistics)) - (CachedBatch(rowCount, columnBuilders.map { builder => + Some(CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) - }, None), stats) - /* - 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") + }, Some(partitionStats))) + } - var i = 0 - totalSize = 0 - while (i < row.numFields) { - columnBuilders(i).appendFrom(row, i) - totalSize += columnBuilders(i).columnStats.sizeInBytes - i += 1 + def next(): AnyRef = { + if (partitionStats == null) { + buildCachedBatch().get + } else { + if (fetchingFirstElement) { + fetchingFirstElement = false + partitionStats + } else { + buildCachedBatch() } - rowCount += 1 } - - batchStats.add(totalSize) - - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics)) - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - */ } def hasNext: Boolean = rowIterator.hasNext @@ -175,6 +158,7 @@ private[columnar] class CachedBatchIterator( val stats = InternalRow.fromSeq( columnBuilders.flatMap(_.columnStats.collectedStatistics)) + CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, Some(stats)) @@ -190,7 +174,7 @@ case class InMemoryRelation( storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( - @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, + @transient var _cachedColumnBuffers: RDD[AnyRef] = null, val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { @@ -230,19 +214,12 @@ case class InMemoryRelation( } } - val cached = if (!usePartitionLevelMetadata) { - batchedRDD.persist(storageLevel) - } else { - val r = batchedRDD.map(_.asInstanceOf[(CachedBatch, InternalRow)]) - val partitionLevelStats = r.map(_._2).collect() - new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD.dependencies, r.map(_._1), - partitionLevelStats) - } + val cached = batchedRDD.persist(storageLevel) cached.setName( tableName.map(n => s"In-memory table $n") .getOrElse(StringUtils.abbreviate(child.toString, 1024))) - _cachedColumnBuffers = cached.asInstanceOf[RDD[CachedBatch]] + _cachedColumnBuffers = cached } def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { @@ -263,7 +240,7 @@ case class InMemoryRelation( batchStats).asInstanceOf[this.type] } - def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers + def cachedColumnBuffers: RDD[AnyRef] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, batchStats) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index a4f6732e3f72..f63aa2f8d093 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -22,11 +22,11 @@ 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._ +import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => GenPredicate, _} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} import org.apache.spark.sql.execution.vectorized._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -194,52 +194,53 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning + private def filterCachedBatchesInternal( + cachedBatchIterator: Iterator[CachedBatch], + partitionStatsSchema: Seq[AttributeReference], + partitionFilter: GenPredicate): Iterator[CachedBatch] = { + val schemaIndex = partitionStatsSchema.zipWithIndex + cachedBatchIterator.filter { cachedBatch => + if (!partitionFilter.eval(cachedBatch.stats.get)) { + logDebug { + val statsString = schemaIndex.map { case (a, i) => + val value = cachedBatch.stats.get.get(i, a.dataType) + s"${a.name}: $value" + }.mkString(", ") + s"Skipping partition based on stats $statsString" + } + false + } else { + true + } + } + } + private def filteredCachedBatches(): RDD[CachedBatch] = { // Using these variables here to avoid serialization of entire objects (if referenced directly) // within the map Partitions closure. val schema = relation.partitionStatistics.schema - val schemaIndex = schema.zipWithIndex val buffers = relation.cachedColumnBuffers - if (sqlContext.conf.inMemoryPartitionMetadata) { - val cachedColumnarRDD = buffers.asInstanceOf[CachedColumnarRDD] - val partitions = cachedColumnarRDD.partitions.map(_.asInstanceOf[CachedColumnarRDDPartition]) - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - if (!partitionFilter.eval(partitions(index).columnStats)) { - Iterator() - } else { - cachedBatchIterator - } - } - } else { - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - // Do partition batch pruning if enabled - if (inMemoryPartitionPruningEnabled) { - cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats.get)) { - logDebug { - val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get.get(i, a.dataType) - s"${a.name}: $value" - }.mkString(", ") - s"Skipping partition based on stats $statsString" - } - false + buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => + + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(index) + val (iterForPartitionCase, iterForDefault) = cachedBatchIterator.duplicate + if (!iterForDefault.hasNext) { + Iterator[CachedBatch]() + } else { + iterForPartitionCase.next() match { + case partitionStats: InternalRow => + if (!partitionFilter.eval(partitionStats)) { + Iterator[CachedBatch]() } else { - true + filterCachedBatchesInternal(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), + schema, partitionFilter) } - } - } else { - cachedBatchIterator + case _: CachedBatch => + iterForDefault.map(_.asInstanceOf[CachedBatch]) } } } From 50f2612f56641f2193eac49945b69d7f3b47765f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 30 Oct 2017 17:01:53 -0700 Subject: [PATCH 07/47] temp --- .../columnar/CachedColumnarRDD.scala | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala new file mode 100644 index 000000000000..81488558d389 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -0,0 +1,40 @@ +/* + * 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.execution.columnar + +import org.apache.spark._ +import org.apache.spark.rdd.RDD + +class CachedColumnarRDD( + @transient private var _sc: SparkContext, + private var dataRDD: RDD[CachedBatch]) + extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { + + override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { + null + } + + override protected def getPartitions = { + null + } + + override private[spark] def getOrCompute(partition: Partition, context: TaskContext): + Iterator[AnyRef] = { + null + } +} From 2b945c9a187240e17b1b4d8f9b9825413ec5a5fc Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 1 Nov 2017 16:01:49 -0700 Subject: [PATCH 08/47] 'CachedColumnarRDD' --- .../org/apache/spark/storage/BlockId.scala | 10 ++- .../columnar/CachedColumnarRDD.scala | 70 +++++++++++++++++-- 2 files changed, 74 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 7ac2c71c18eb..e26a3c6aff03 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -31,7 +31,7 @@ import org.apache.spark.annotation.DeveloperApi * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ @DeveloperApi -sealed abstract class BlockId { +abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -49,6 +49,11 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { override def name: String = "rdd_" + rddId + "_" + splitIndex } +@DeveloperApi +case class RDDPartitionMetadataBlockId(rddId: Int, splitIndex: Int) extends BlockId { + override def name: String = "rdd_" + rddId + "_" + splitIndex + ".metadata" +} + // Format of the shuffle block ids (including data and index) should be kept in sync with // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getBlockData(). @DeveloperApi @@ -103,6 +108,7 @@ class UnrecognizedBlockId(name: String) @DeveloperApi object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r + val PARTITION_METADATA = "rdd_([0-9]+)_([0-9]+).metadata".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r @@ -116,6 +122,8 @@ object BlockId { def apply(name: String): BlockId = name match { case RDD(rddId, splitIndex) => RDDBlockId(rddId.toInt, splitIndex.toInt) + case PARTITION_METADATA(rddId, splitIndex) => + RDDPartitionMetadataBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case SHUFFLE_DATA(shuffleId, mapId, reduceId) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 81488558d389..f97a8313d4bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId} class CachedColumnarRDD( @transient private var _sc: SparkContext, @@ -26,15 +28,73 @@ class CachedColumnarRDD( extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { - null + val cachedBatch = dataRDD.iterator(split, context).next() + // put metadata to blockmanager + SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), + cachedBatch.stats.get, dataRDD.getStorageLevel) + Iterator(cachedBatch) } - override protected def getPartitions = { - null + override protected def getPartitions: Array[Partition] = dataRDD.partitions + + private def fetchOrComputeCachedBatch(partition: Partition, context: TaskContext): + Iterator[CachedBatch] = { + // metadata block can be evicted by BlockManagers but we may still keep CachedBatch in memory + // so that we still need to try to fetch it from Cache + val blockId = RDDBlockId(id, partition.index) + SparkEnv.get.blockManager.getOrElseUpdate(blockId, getStorageLevel, elementClassTag, () => { + computeOrReadCheckpoint(partition, context) + }) match { + case Left(blockResult) => + val existingMetrics = context.taskMetrics().inputMetrics + existingMetrics.incBytesRead(blockResult.bytes) + new InterruptibleIterator[CachedBatch](context, + blockResult.data.asInstanceOf[Iterator[CachedBatch]]) { + override def next(): CachedBatch = { + existingMetrics.incRecordsRead(1) + delegate.next() + } + } + case Right(iter) => + new InterruptibleIterator(context, iter.asInstanceOf[Iterator[CachedBatch]]) + } } - override private[spark] def getOrCompute(partition: Partition, context: TaskContext): + override private[spark] def getOrCompute(split: Partition, context: TaskContext): Iterator[AnyRef] = { - null + val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) + // if metadata block is not contained + val metadataBlockOpt = SparkEnv.get.blockManager.get[InternalRow](metadataBlockId) + if (metadataBlockOpt.isDefined) { + val metadataBlock = metadataBlockOpt.get + new InterruptibleIterator[AnyRef](context, new Iterator[AnyRef] { + + var fetchingFirstElement = true + + var delegate: Iterator[CachedBatch] = _ + + override def hasNext: Boolean = { + if (fetchingFirstElement) { + true + } else { + delegate = fetchOrComputeCachedBatch(split, context) + delegate.hasNext + } + } + + override def next(): AnyRef = { + if (fetchingFirstElement) { + fetchingFirstElement = false + val mb = metadataBlock.data.next() + mb.asInstanceOf[InternalRow] + } else { + delegate.next() + } + } + }) + } else { + fetchOrComputeCachedBatch(split, context) + } + } } From 89f0a982309f1c589d4b0b31ed32cbd9a1494f32 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 1 Nov 2017 16:11:52 -0700 Subject: [PATCH 09/47] change types --- .../execution/columnar/InMemoryRelation.scala | 37 +++++++------------ .../columnar/InMemoryTableScanExec.scala | 4 +- 2 files changed, 15 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 2f5cddd3d8df..16082ae4e444 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -50,20 +50,16 @@ object InMemoryRelation { * @param stats The stat of columns */ private[columnar] -case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: Option[InternalRow]) +case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) private[columnar] class CachedPartitionIterator( rowIterator: Iterator[InternalRow], output: Seq[Attribute], batchSize: Int, useCompression: Boolean, - batchStats: LongAccumulator) extends Iterator[AnyRef] { - - private var partitionStats: InternalRow = _ - - private var fetchingFirstElement = true + batchStats: LongAccumulator) extends Iterator[CachedBatch] { - private def buildCachedBatch(): Option[CachedBatch] = { + private def buildCachedBatch(): CachedBatch = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -92,24 +88,17 @@ private[columnar] class CachedPartitionIterator( } rowCount += 1 } - partitionStats = InternalRow.fromSeq(columnBuilders.flatMap(_.columnStats.collectedStatistics)) batchStats.add(totalSize) - Some(CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, Some(partitionStats))) + + val stats = InternalRow.fromSeq( + columnBuilders.flatMap(_.columnStats.collectedStatistics)) + + CachedBatch(rowCount, columnBuilders.map { builder => + JavaUtils.bufferToArray(builder.build())}, stats) } - def next(): AnyRef = { - if (partitionStats == null) { - buildCachedBatch().get - } else { - if (fetchingFirstElement) { - fetchingFirstElement = false - partitionStats - } else { - buildCachedBatch() - } - } + def next(): CachedBatch = { + buildCachedBatch() } def hasNext: Boolean = rowIterator.hasNext @@ -161,7 +150,7 @@ private[columnar] class CachedBatchIterator( CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) - }, Some(stats)) + }, stats) } def hasNext: Boolean = rowIterator.hasNext @@ -214,7 +203,7 @@ case class InMemoryRelation( } } - val cached = batchedRDD.persist(storageLevel) + val cached = new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD).persist(storageLevel) cached.setName( tableName.map(n => s"In-memory table $n") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f63aa2f8d093..f2609d205f91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -200,10 +200,10 @@ case class InMemoryTableScanExec( partitionFilter: GenPredicate): Iterator[CachedBatch] = { val schemaIndex = partitionStatsSchema.zipWithIndex cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter.eval(cachedBatch.stats.get)) { + if (!partitionFilter.eval(cachedBatch.stats)) { logDebug { val statsString = schemaIndex.map { case (a, i) => - val value = cachedBatch.stats.get.get(i, a.dataType) + val value = cachedBatch.stats.get(i, a.dataType) s"${a.name}: $value" }.mkString(", ") s"Skipping partition based on stats $statsString" From 5aa1808b69e95818e82c6ece86dce1f51d94f4ca Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 1 Nov 2017 16:16:58 -0700 Subject: [PATCH 10/47] fix compilation error --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index f97a8313d4bd..523a44b1248d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -31,7 +31,7 @@ class CachedColumnarRDD( val cachedBatch = dataRDD.iterator(split, context).next() // put metadata to blockmanager SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), - cachedBatch.stats.get, dataRDD.getStorageLevel) + cachedBatch.stats, dataRDD.getStorageLevel) Iterator(cachedBatch) } From a5adc5609968e67c0c184d29359507f067030190 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 10:13:15 -0700 Subject: [PATCH 11/47] update --- .../columnar/CachedColumnarRDD.scala | 26 ++++++++++++------- .../execution/columnar/InMemoryRelation.scala | 11 +++----- .../columnar/InMemoryTableScanExec.scala | 7 ++--- 3 files changed, 25 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 523a44b1248d..cce25d1beed1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -24,15 +24,24 @@ import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId} class CachedColumnarRDD( @transient private var _sc: SparkContext, - private var dataRDD: RDD[CachedBatch]) + private var dataRDD: RDD[CachedBatch], + containsPartitionMetadata: Boolean) extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { - val cachedBatch = dataRDD.iterator(split, context).next() - // put metadata to blockmanager - SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), - cachedBatch.stats, dataRDD.getStorageLevel) - Iterator(cachedBatch) + if (containsPartitionMetadata) { + val parentIterator = dataRDD.iterator(split, context) + if (!parentIterator.hasNext) { + Iterator() + } else { + val cachedBatch = parentIterator.next() + SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), + cachedBatch.stats, dataRDD.getStorageLevel) + Iterator(cachedBatch) + } + } else { + firstParent.iterator(split, context) + } } override protected def getPartitions: Array[Partition] = dataRDD.partitions @@ -63,15 +72,14 @@ class CachedColumnarRDD( override private[spark] def getOrCompute(split: Partition, context: TaskContext): Iterator[AnyRef] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) - // if metadata block is not contained val metadataBlockOpt = SparkEnv.get.blockManager.get[InternalRow](metadataBlockId) if (metadataBlockOpt.isDefined) { val metadataBlock = metadataBlockOpt.get new InterruptibleIterator[AnyRef](context, new Iterator[AnyRef] { - var fetchingFirstElement = true + private var fetchingFirstElement = true - var delegate: Iterator[CachedBatch] = _ + private var delegate: Iterator[CachedBatch] = _ override def hasNext: Boolean = { if (fetchingFirstElement) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 16082ae4e444..871d7f13f24c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -59,7 +59,7 @@ private[columnar] class CachedPartitionIterator( useCompression: Boolean, batchStats: LongAccumulator) extends Iterator[CachedBatch] { - private def buildCachedBatch(): CachedBatch = { + def next(): CachedBatch = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -97,10 +97,6 @@ private[columnar] class CachedPartitionIterator( JavaUtils.bufferToArray(builder.build())}, stats) } - def next(): CachedBatch = { - buildCachedBatch() - } - def hasNext: Boolean = rowIterator.hasNext } @@ -111,7 +107,7 @@ private[columnar] class CachedBatchIterator( useCompression: Boolean, batchStats: LongAccumulator) extends Iterator[CachedBatch] { - def next(): CachedBatch = { + def next(): CachedBatch = { val columnBuilders = output.map { attribute => ColumnBuilder(attribute.dataType, batchSize, attribute.name, useCompression) }.toArray @@ -203,7 +199,8 @@ case class InMemoryRelation( } } - val cached = new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD).persist(storageLevel) + val cached = new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD, + usePartitionLevelMetadata).persist(storageLevel) cached.setName( tableName.map(n => s"In-memory table $n") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f2609d205f91..b13d79dafa52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -194,7 +194,7 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning - private def filterCachedBatchesInternal( + private def doFilterCachedBatches( cachedBatchIterator: Iterator[CachedBatch], partitionStatsSchema: Seq[AttributeReference], partitionFilter: GenPredicate): Iterator[CachedBatch] = { @@ -236,11 +236,12 @@ case class InMemoryTableScanExec( if (!partitionFilter.eval(partitionStats)) { Iterator[CachedBatch]() } else { - filterCachedBatchesInternal(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), + doFilterCachedBatches(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), schema, partitionFilter) } case _: CachedBatch => - iterForDefault.map(_.asInstanceOf[CachedBatch]) + doFilterCachedBatches(iterForDefault.map(_.asInstanceOf[CachedBatch]), schema, + partitionFilter) } } } From 00b1642523c7008a8e0065b41d3d36ff1b13ea66 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 10:21:23 -0700 Subject: [PATCH 12/47] fix storage level --- .../sql/execution/columnar/CachedColumnarRDD.scala | 10 ++++++---- .../sql/execution/columnar/InMemoryRelation.scala | 2 +- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index cce25d1beed1..bb9ec494a7fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId} +import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId, StorageLevel} class CachedColumnarRDD( @transient private var _sc: SparkContext, private var dataRDD: RDD[CachedBatch], - containsPartitionMetadata: Boolean) + containsPartitionMetadata: Boolean, + expectedStorageLevel: StorageLevel) extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { @@ -36,7 +37,7 @@ class CachedColumnarRDD( } else { val cachedBatch = parentIterator.next() SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), - cachedBatch.stats, dataRDD.getStorageLevel) + cachedBatch.stats, expectedStorageLevel) Iterator(cachedBatch) } } else { @@ -51,7 +52,8 @@ class CachedColumnarRDD( // metadata block can be evicted by BlockManagers but we may still keep CachedBatch in memory // so that we still need to try to fetch it from Cache val blockId = RDDBlockId(id, partition.index) - SparkEnv.get.blockManager.getOrElseUpdate(blockId, getStorageLevel, elementClassTag, () => { + SparkEnv.get.blockManager.getOrElseUpdate(blockId, expectedStorageLevel, elementClassTag, + () => { computeOrReadCheckpoint(partition, context) }) match { case Left(blockResult) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 871d7f13f24c..973117613ac2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -200,7 +200,7 @@ case class InMemoryRelation( } val cached = new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD, - usePartitionLevelMetadata).persist(storageLevel) + usePartitionLevelMetadata, storageLevel).persist(storageLevel) cached.setName( tableName.map(n => s"In-memory table $n") From 311fe5ae184e28c8f17cb3bf12e188eb4a7b5075 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 10:42:25 -0700 Subject: [PATCH 13/47] fix getOrCompute --- .../columnar/CachedColumnarRDD.scala | 31 +++---------------- 1 file changed, 4 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index bb9ec494a7fa..b63cc2e1d8ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -47,47 +47,24 @@ class CachedColumnarRDD( override protected def getPartitions: Array[Partition] = dataRDD.partitions - private def fetchOrComputeCachedBatch(partition: Partition, context: TaskContext): - Iterator[CachedBatch] = { - // metadata block can be evicted by BlockManagers but we may still keep CachedBatch in memory - // so that we still need to try to fetch it from Cache - val blockId = RDDBlockId(id, partition.index) - SparkEnv.get.blockManager.getOrElseUpdate(blockId, expectedStorageLevel, elementClassTag, - () => { - computeOrReadCheckpoint(partition, context) - }) match { - case Left(blockResult) => - val existingMetrics = context.taskMetrics().inputMetrics - existingMetrics.incBytesRead(blockResult.bytes) - new InterruptibleIterator[CachedBatch](context, - blockResult.data.asInstanceOf[Iterator[CachedBatch]]) { - override def next(): CachedBatch = { - existingMetrics.incRecordsRead(1) - delegate.next() - } - } - case Right(iter) => - new InterruptibleIterator(context, iter.asInstanceOf[Iterator[CachedBatch]]) - } - } - override private[spark] def getOrCompute(split: Partition, context: TaskContext): Iterator[AnyRef] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) val metadataBlockOpt = SparkEnv.get.blockManager.get[InternalRow](metadataBlockId) + val superGetOrCompute: (Partition, TaskContext) => Iterator[AnyRef] = super.getOrCompute if (metadataBlockOpt.isDefined) { val metadataBlock = metadataBlockOpt.get new InterruptibleIterator[AnyRef](context, new Iterator[AnyRef] { private var fetchingFirstElement = true - private var delegate: Iterator[CachedBatch] = _ + private var delegate: Iterator[AnyRef] = _ override def hasNext: Boolean = { if (fetchingFirstElement) { true } else { - delegate = fetchOrComputeCachedBatch(split, context) + delegate = superGetOrCompute(split, context) delegate.hasNext } } @@ -103,7 +80,7 @@ class CachedColumnarRDD( } }) } else { - fetchOrComputeCachedBatch(split, context) + superGetOrCompute(split, context) } } From 6411b823682f745223e60417ee91b8ee067a1dbc Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 10:55:36 -0700 Subject: [PATCH 14/47] evaluate with partition metadata --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index b63cc2e1d8ac..0d01be24d1ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -31,6 +31,7 @@ class CachedColumnarRDD( override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { if (containsPartitionMetadata) { + println("evaluate with partition metadata") val parentIterator = dataRDD.iterator(split, context) if (!parentIterator.hasNext) { Iterator() From 41f6ad23f901cf2ba9aed91d138007f9f2c14e7c Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 11:21:19 -0700 Subject: [PATCH 15/47] fix getOrCompute --- .../execution/columnar/CachedColumnarRDD.scala | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 0d01be24d1ce..706edcaf4d4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -31,7 +31,6 @@ class CachedColumnarRDD( override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { if (containsPartitionMetadata) { - println("evaluate with partition metadata") val parentIterator = dataRDD.iterator(split, context) if (!parentIterator.hasNext) { Iterator() @@ -51,10 +50,8 @@ class CachedColumnarRDD( override private[spark] def getOrCompute(split: Partition, context: TaskContext): Iterator[AnyRef] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) - val metadataBlockOpt = SparkEnv.get.blockManager.get[InternalRow](metadataBlockId) val superGetOrCompute: (Partition, TaskContext) => Iterator[AnyRef] = super.getOrCompute - if (metadataBlockOpt.isDefined) { - val metadataBlock = metadataBlockOpt.get + SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => new InterruptibleIterator[AnyRef](context, new Iterator[AnyRef] { private var fetchingFirstElement = true @@ -65,7 +62,9 @@ class CachedColumnarRDD( if (fetchingFirstElement) { true } else { - delegate = superGetOrCompute(split, context) + if (delegate == null) { + delegate = superGetOrCompute(split, context) + } delegate.hasNext } } @@ -73,16 +72,13 @@ class CachedColumnarRDD( override def next(): AnyRef = { if (fetchingFirstElement) { fetchingFirstElement = false - val mb = metadataBlock.data.next() + val mb = metadataBlock mb.asInstanceOf[InternalRow] } else { delegate.next() } } }) - } else { - superGetOrCompute(split, context) - } - + ).getOrElse(superGetOrCompute(split, context)) } } From c50b74354e3b237b1f2f3924763fe150b0d59025 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 12:06:49 -0700 Subject: [PATCH 16/47] add logging --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 ++ .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 1 + 2 files changed, 3 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 706edcaf4d4f..88669af40ec6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -29,6 +29,8 @@ class CachedColumnarRDD( expectedStorageLevel: StorageLevel) extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { + logInfo(s"Created CachedColumnarRDD with ${partitions.length} partitions") + override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { if (containsPartitionMetadata) { val parentIterator = dataRDD.iterator(split, context) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index b13d79dafa52..4bb1499d3fe3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -236,6 +236,7 @@ case class InMemoryTableScanExec( if (!partitionFilter.eval(partitionStats)) { Iterator[CachedBatch]() } else { + logInfo(s"accept partition $index based on the stats") doFilterCachedBatches(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), schema, partitionFilter) } From 78f774f730792b2a7de52452810aa51138d3b79b Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 12:12:57 -0700 Subject: [PATCH 17/47] add logging for skipped partition --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 +- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 88669af40ec6..66722978a954 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -29,7 +29,7 @@ class CachedColumnarRDD( expectedStorageLevel: StorageLevel) extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { - logInfo(s"Created CachedColumnarRDD with ${partitions.length} partitions") + logInfo(s"Created CachedColumnarRDD $id with ${partitions.length} partitions") override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { if (containsPartitionMetadata) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 4bb1499d3fe3..568b15016809 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -234,6 +234,7 @@ case class InMemoryTableScanExec( iterForPartitionCase.next() match { case partitionStats: InternalRow => if (!partitionFilter.eval(partitionStats)) { + logInfo(s"skip partition $index based on the stats") Iterator[CachedBatch]() } else { logInfo(s"accept partition $index based on the stats") From 71456bd59c51faaef4ef892bbb842ede98e178c8 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 13:15:29 -0700 Subject: [PATCH 18/47] try to print stats --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 973117613ac2..49fb503b91ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -90,8 +90,13 @@ private[columnar] class CachedPartitionIterator( } batchStats.add(totalSize) - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics)) + val statsInSeq = columnBuilders.flatMap(_.columnStats.collectedStatistics) + + // scalastyle:off + println(s"stats ${statsInSeq.toSeq}") + // scalastyle:on + + val stats = InternalRow.fromSeq(statsInSeq) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build())}, stats) From 97544a6125122d960d1aa3b1ffe09413c523c6e6 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 13:20:43 -0700 Subject: [PATCH 19/47] add logging for skipped partition --- .../apache/spark/sql/execution/columnar/InMemoryRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 49fb503b91ba..958ebe5aaa8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -93,7 +93,7 @@ private[columnar] class CachedPartitionIterator( val statsInSeq = columnBuilders.flatMap(_.columnStats.collectedStatistics) // scalastyle:off - println(s"stats ${statsInSeq.toSeq}") + println(s"generate stats ${statsInSeq.toSeq}") // scalastyle:on val stats = InternalRow.fromSeq(statsInSeq) From c131b2d5c0e2cae005e3cda73f01cff86c9bb9b6 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 13:25:18 -0700 Subject: [PATCH 20/47] add logging for skipped partition --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 958ebe5aaa8e..f17de3240b04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -146,8 +146,13 @@ private[columnar] class CachedBatchIterator( batchStats.add(totalSize) - val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics)) + val statsInSeq = columnBuilders.flatMap(_.columnStats.collectedStatistics) + + val stats = InternalRow.fromSeq(statsInSeq) + + // scalastyle:off + println(s"generate stats ${statsInSeq.toSeq}") + // scalastyle:on CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) From d588fb099c5559d44197fdaa1d8c368034496151 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 2 Nov 2017 14:14:57 -0700 Subject: [PATCH 21/47] add logging for skipped partition --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 4 ---- .../sql/execution/columnar/InMemoryTableScanExec.scala | 6 ++++-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index f17de3240b04..e5be062fbd89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -150,10 +150,6 @@ private[columnar] class CachedBatchIterator( val stats = InternalRow.fromSeq(statsInSeq) - // scalastyle:off - println(s"generate stats ${statsInSeq.toSeq}") - // scalastyle:on - CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 568b15016809..1660cb376469 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -233,14 +233,16 @@ case class InMemoryTableScanExec( } else { iterForPartitionCase.next() match { case partitionStats: InternalRow => + // scalastyle:off if (!partitionFilter.eval(partitionStats)) { - logInfo(s"skip partition $index based on the stats") + println(s"skip partition $index based on the stats") Iterator[CachedBatch]() } else { - logInfo(s"accept partition $index based on the stats") + println(s"accept partition $index based on the stats") doFilterCachedBatches(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), schema, partitionFilter) } + // scalastyle:on case _: CachedBatch => doFilterCachedBatches(iterForDefault.map(_.asInstanceOf[CachedBatch]), schema, partitionFilter) From 1ba1f80e2169d49ace437da2279dca1118335e76 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 13:23:33 -0800 Subject: [PATCH 22/47] refactor the code --- .../CachedColumnarPartitionIterator.scala | 27 ++++++++++++ .../columnar/CachedColumnarRDD.scala | 43 +++++-------------- .../execution/columnar/InMemoryRelation.scala | 6 +-- .../columnar/InMemoryTableScanExec.scala | 31 +++++-------- 4 files changed, 52 insertions(+), 55 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala new file mode 100644 index 000000000000..23c37c2e58fd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala @@ -0,0 +1,27 @@ +/* + * 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.execution.columnar + +import org.apache.spark.{InterruptibleIterator, Partition, TaskContext} +import org.apache.spark.sql.catalyst.InternalRow + +private[columnar] class CachedColumnarPartitionIterator( + val metadataBlock: InternalRow, + context: TaskContext, + delegate: Iterator[CachedBatch]) + extends InterruptibleIterator[CachedBatch](context, delegate) {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 66722978a954..348006ee50d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -20,16 +20,14 @@ package org.apache.spark.sql.execution.columnar import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId, StorageLevel} +import org.apache.spark.storage.{RDDPartitionMetadataBlockId, StorageLevel} class CachedColumnarRDD( @transient private var _sc: SparkContext, private var dataRDD: RDD[CachedBatch], containsPartitionMetadata: Boolean, expectedStorageLevel: StorageLevel) - extends RDD[AnyRef](_sc, Seq(new OneToOneDependency(dataRDD))) { - - logInfo(s"Created CachedColumnarRDD $id with ${partitions.length} partitions") + extends RDD[CachedBatch](_sc, Seq(new OneToOneDependency(dataRDD))) { override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { if (containsPartitionMetadata) { @@ -52,35 +50,16 @@ class CachedColumnarRDD( override private[spark] def getOrCompute(split: Partition, context: TaskContext): Iterator[AnyRef] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) - val superGetOrCompute: (Partition, TaskContext) => Iterator[AnyRef] = super.getOrCompute + val superGetOrCompute: (Partition, TaskContext) => Iterator[CachedBatch] = super.getOrCompute SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => - new InterruptibleIterator[AnyRef](context, new Iterator[AnyRef] { - - private var fetchingFirstElement = true - - private var delegate: Iterator[AnyRef] = _ - - override def hasNext: Boolean = { - if (fetchingFirstElement) { - true - } else { - if (delegate == null) { - delegate = superGetOrCompute(split, context) - } - delegate.hasNext - } - } - - override def next(): AnyRef = { - if (fetchingFirstElement) { - fetchingFirstElement = false - val mb = metadataBlock - mb.asInstanceOf[InternalRow] - } else { - delegate.next() - } - } - }) + new CachedColumnarPartitionIterator(metadataBlock, context, superGetOrCompute(split, context)) ).getOrElse(superGetOrCompute(split, context)) } } + +private[columnar] class CachedColumnarPartitionIterator( + val metadataBlock: InternalRow, + split: Partition, + context: TaskContext, + delegate: Iterator[CachedBatch]) + extends InterruptibleIterator[CachedBatch](context, delegate) {} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index e5be062fbd89..74977cbe931f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -165,7 +165,7 @@ case class InMemoryRelation( storageLevel: StorageLevel, @transient child: SparkPlan, tableName: Option[String])( - @transient var _cachedColumnBuffers: RDD[AnyRef] = null, + @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { @@ -196,7 +196,7 @@ case class InMemoryRelation( private def buildBuffers(): Unit = { val output = child.output - // TODO: + // TODO: need better abstraction for two iterators here val batchedRDD = child.execute().mapPartitionsInternal { rowIterator => if (!usePartitionLevelMetadata) { new CachedBatchIterator(rowIterator, output, batchSize, useCompression, batchStats) @@ -232,7 +232,7 @@ case class InMemoryRelation( batchStats).asInstanceOf[this.type] } - def cachedColumnBuffers: RDD[AnyRef] = _cachedColumnBuffers + def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = Seq(_cachedColumnBuffers, batchStats) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 1660cb376469..ef38c01151fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.columnar +import org.apache.spark.SparkEnv import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -227,26 +228,16 @@ case class InMemoryTableScanExec( partitionFilters.reduceOption(And).getOrElse(Literal(true)), schema) partitionFilter.initialize(index) - val (iterForPartitionCase, iterForDefault) = cachedBatchIterator.duplicate - if (!iterForDefault.hasNext) { - Iterator[CachedBatch]() - } else { - iterForPartitionCase.next() match { - case partitionStats: InternalRow => - // scalastyle:off - if (!partitionFilter.eval(partitionStats)) { - println(s"skip partition $index based on the stats") - Iterator[CachedBatch]() - } else { - println(s"accept partition $index based on the stats") - doFilterCachedBatches(iterForPartitionCase.map(_.asInstanceOf[CachedBatch]), - schema, partitionFilter) - } - // scalastyle:on - case _: CachedBatch => - doFilterCachedBatches(iterForDefault.map(_.asInstanceOf[CachedBatch]), schema, - partitionFilter) - } + + cachedBatchIterator match { + case cachedIter: CachedColumnarPartitionIterator + if !partitionFilter.eval(cachedIter.metadataBlock) => + // scalastyle:off + println(s"skipped partition $index") + // scalastyle:on + Iterator[CachedBatch]() + case _ => + cachedBatchIterator } } } From 62f358d0e2f6e4f3ec6aa5a7d53d8a72ac171d07 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 13:30:57 -0800 Subject: [PATCH 23/47] fix compilation issue --- .../CachedColumnarPartitionIterator.scala | 27 ------------------- .../columnar/CachedColumnarRDD.scala | 3 +-- 2 files changed, 1 insertion(+), 29 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala deleted file mode 100644 index 23c37c2e58fd..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarPartitionIterator.scala +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.execution.columnar - -import org.apache.spark.{InterruptibleIterator, Partition, TaskContext} -import org.apache.spark.sql.catalyst.InternalRow - -private[columnar] class CachedColumnarPartitionIterator( - val metadataBlock: InternalRow, - context: TaskContext, - delegate: Iterator[CachedBatch]) - extends InterruptibleIterator[CachedBatch](context, delegate) {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 348006ee50d7..5f5b06fc07f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -59,7 +59,6 @@ class CachedColumnarRDD( private[columnar] class CachedColumnarPartitionIterator( val metadataBlock: InternalRow, - split: Partition, context: TaskContext, delegate: Iterator[CachedBatch]) - extends InterruptibleIterator[CachedBatch](context, delegate) {} \ No newline at end of file + extends InterruptibleIterator[CachedBatch](context, delegate) {} From 500d4fdad882ee34a3f130b86ae041ba22f51ca5 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 13:33:51 -0800 Subject: [PATCH 24/47] refactor the code --- .../spark/sql/execution/columnar/CachedColumnarRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 5f5b06fc07f5..1c531ad67401 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -29,7 +29,7 @@ class CachedColumnarRDD( expectedStorageLevel: StorageLevel) extends RDD[CachedBatch](_sc, Seq(new OneToOneDependency(dataRDD))) { - override def compute(split: Partition, context: TaskContext): Iterator[AnyRef] = { + override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { if (containsPartitionMetadata) { val parentIterator = dataRDD.iterator(split, context) if (!parentIterator.hasNext) { @@ -48,7 +48,7 @@ class CachedColumnarRDD( override protected def getPartitions: Array[Partition] = dataRDD.partitions override private[spark] def getOrCompute(split: Partition, context: TaskContext): - Iterator[AnyRef] = { + Iterator[CachedBatch] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) val superGetOrCompute: (Partition, TaskContext) => Iterator[CachedBatch] = super.getOrCompute SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => From 63c589780c52de3feeadac97a27655a4f3b67313 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 13:49:14 -0800 Subject: [PATCH 25/47] test --- .../sql/execution/columnar/CachedColumnarRDD.scala | 11 +++++++++-- .../execution/columnar/InMemoryTableScanExec.scala | 2 +- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 1c531ad67401..196818b75501 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -58,7 +58,14 @@ class CachedColumnarRDD( } private[columnar] class CachedColumnarPartitionIterator( - val metadataBlock: InternalRow, + val partitionStats: InternalRow, context: TaskContext, delegate: Iterator[CachedBatch]) - extends InterruptibleIterator[CachedBatch](context, delegate) {} + extends InterruptibleIterator[CachedBatch](context, delegate) { + override def next(): CachedBatch = {\ + // scalastyle:off + println("next") + // scalastyle:on + super.next() + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index ef38c01151fe..72178284038b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -231,7 +231,7 @@ case class InMemoryTableScanExec( cachedBatchIterator match { case cachedIter: CachedColumnarPartitionIterator - if !partitionFilter.eval(cachedIter.metadataBlock) => + if !partitionFilter.eval(cachedIter.partitionStats) => // scalastyle:off println(s"skipped partition $index") // scalastyle:on From 9031eaf1b37398ffb0bff007b4a30c20d104e80e Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 13:50:11 -0800 Subject: [PATCH 26/47] fix compilation issue --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 196818b75501..ce6d5aa31f74 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -62,7 +62,7 @@ private[columnar] class CachedColumnarPartitionIterator( context: TaskContext, delegate: Iterator[CachedBatch]) extends InterruptibleIterator[CachedBatch](context, delegate) { - override def next(): CachedBatch = {\ + override def next(): CachedBatch = { // scalastyle:off println("next") // scalastyle:on From 1692303800f766c45dec19bfae7ea7ae2b147073 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 14:11:04 -0800 Subject: [PATCH 27/47] add missing filtering --- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 72178284038b..3b29d968adb0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -235,9 +235,9 @@ case class InMemoryTableScanExec( // scalastyle:off println(s"skipped partition $index") // scalastyle:on - Iterator[CachedBatch]() + Iterator() case _ => - cachedBatchIterator + doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) } } } From f9f3d20a8d0f6a282d7d463a6a326a9295967967 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 14:16:43 -0800 Subject: [PATCH 28/47] test --- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 3b29d968adb0..3d7bf9f94e3d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -229,6 +229,10 @@ case class InMemoryTableScanExec( schema) partitionFilter.initialize(index) + if (cachedBatchIterator.isInstanceOf[CachedColumnarPartitionIterator]) { + cachedBatchIterator.asInstanceOf[CachedColumnarPartitionIterator].partitionStats + } + cachedBatchIterator match { case cachedIter: CachedColumnarPartitionIterator if !partitionFilter.eval(cachedIter.partitionStats) => From da5f06fe80f35f00a91edcdc46032ea12b4418c5 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 14:22:23 -0800 Subject: [PATCH 29/47] test --- .../execution/columnar/InMemoryTableScanExec.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 3d7bf9f94e3d..e01ee2286b12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -230,9 +230,20 @@ case class InMemoryTableScanExec( partitionFilter.initialize(index) if (cachedBatchIterator.isInstanceOf[CachedColumnarPartitionIterator]) { - cachedBatchIterator.asInstanceOf[CachedColumnarPartitionIterator].partitionStats + val cachedIter = cachedBatchIterator.asInstanceOf[CachedColumnarPartitionIterator] + if (!partitionFilter.eval(cachedIter.partitionStats)) { + // scalastyle:off + println(s"skipped partition $index") + // scalastyle:on + Iterator() + } else { + doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) + } + } else { + doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) } + /* cachedBatchIterator match { case cachedIter: CachedColumnarPartitionIterator if !partitionFilter.eval(cachedIter.partitionStats) => @@ -243,6 +254,7 @@ case class InMemoryTableScanExec( case _ => doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) } + */ } } From 2c0b6cde18a5423fc524b762ab49aa81d212c883 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 15:02:03 -0800 Subject: [PATCH 30/47] fix rebundant read --- .../columnar/CachedColumnarRDD.scala | 23 +++++++++++++------ .../columnar/InMemoryTableScanExec.scala | 20 ++-------------- 2 files changed, 18 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index ce6d5aa31f74..022eaecd86ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -52,20 +52,29 @@ class CachedColumnarRDD( val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) val superGetOrCompute: (Partition, TaskContext) => Iterator[CachedBatch] = super.getOrCompute SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => - new CachedColumnarPartitionIterator(metadataBlock, context, superGetOrCompute(split, context)) + new InterruptibleIterator[CachedBatch](context, + new CachedColumnarPartitionIterator(metadataBlock, split, context, superGetOrCompute)) ).getOrElse(superGetOrCompute(split, context)) } } private[columnar] class CachedColumnarPartitionIterator( val partitionStats: InternalRow, + partition: Partition, context: TaskContext, - delegate: Iterator[CachedBatch]) - extends InterruptibleIterator[CachedBatch](context, delegate) { + fetchRDDPartition: (Partition, TaskContext) => Iterator[CachedBatch]) + extends Iterator[CachedBatch] { + + private var delegate: Iterator[CachedBatch] = _ + + override def hasNext: Boolean = { + if (delegate == null) { + delegate = fetchRDDPartition(partition, context) + } + delegate.hasNext + } + override def next(): CachedBatch = { - // scalastyle:off - println("next") - // scalastyle:on - super.next() + delegate.next() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index e01ee2286b12..f6b9282d4f92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.SparkEnv +import org.apache.spark.{InterruptibleIterator, SparkEnv} import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -229,22 +229,7 @@ case class InMemoryTableScanExec( schema) partitionFilter.initialize(index) - if (cachedBatchIterator.isInstanceOf[CachedColumnarPartitionIterator]) { - val cachedIter = cachedBatchIterator.asInstanceOf[CachedColumnarPartitionIterator] - if (!partitionFilter.eval(cachedIter.partitionStats)) { - // scalastyle:off - println(s"skipped partition $index") - // scalastyle:on - Iterator() - } else { - doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) - } - } else { - doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) - } - - /* - cachedBatchIterator match { + cachedBatchIterator.asInstanceOf[InterruptibleIterator[_]].delegate match { case cachedIter: CachedColumnarPartitionIterator if !partitionFilter.eval(cachedIter.partitionStats) => // scalastyle:off @@ -254,7 +239,6 @@ case class InMemoryTableScanExec( case _ => doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) } - */ } } From e1d8c43f8e7cf3805659e8a228ace0f6244592e0 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 16:32:03 -0800 Subject: [PATCH 31/47] compact iterators --- .../execution/columnar/InMemoryRelation.scala | 85 +++++-------------- 1 file changed, 19 insertions(+), 66 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 74977cbe931f..8570ecb1907b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -52,12 +52,13 @@ object InMemoryRelation { private[columnar] case class CachedBatch(numRows: Int, buffers: Array[Array[Byte]], stats: InternalRow) -private[columnar] class CachedPartitionIterator( +private[columnar] class CachedBatchIterator( rowIterator: Iterator[InternalRow], output: Seq[Attribute], batchSize: Int, useCompression: Boolean, - batchStats: LongAccumulator) extends Iterator[CachedBatch] { + batchStats: LongAccumulator, + singleBatchPerPartition: Boolean) extends Iterator[CachedBatch] { def next(): CachedBatch = { val columnBuilders = output.map { attribute => @@ -66,7 +67,17 @@ private[columnar] class CachedPartitionIterator( var rowCount = 0 var totalSize = 0L - while (rowIterator.hasNext) { + + val terminateLoop = (singleBatch: Boolean, rowIter: Iterator[InternalRow], + rowCount: Int, size: Long) => { + if (!singleBatch) { + rowIter.hasNext && rowCount < batchSize && totalSize < ColumnBuilder.MAX_BATCH_SIZE_IN_BYTE + } else { + rowIter.hasNext + } + } + + while (terminateLoop(singleBatchPerPartition, rowIterator, rowCount, totalSize)) { val row = rowIterator.next() // Added for SPARK-6082. This assertion can be useful for scenarios when something @@ -88,76 +99,21 @@ private[columnar] class CachedPartitionIterator( } rowCount += 1 } + batchStats.add(totalSize) val statsInSeq = columnBuilders.flatMap(_.columnStats.collectedStatistics) - // scalastyle:off - println(s"generate stats ${statsInSeq.toSeq}") - // scalastyle:on - val stats = InternalRow.fromSeq(statsInSeq) CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build())}, stats) + JavaUtils.bufferToArray(builder.build()) + }, stats) } def hasNext: Boolean = rowIterator.hasNext } -private[columnar] class CachedBatchIterator( - rowIterator: Iterator[InternalRow], - output: Seq[Attribute], - batchSize: Int, - useCompression: Boolean, - batchStats: LongAccumulator) extends 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 - } - - batchStats.add(totalSize) - - val statsInSeq = columnBuilders.flatMap(_.columnStats.collectedStatistics) - - val stats = InternalRow.fromSeq(statsInSeq) - - CachedBatch(rowCount, columnBuilders.map { builder => - JavaUtils.bufferToArray(builder.build()) - }, stats) - } - - def hasNext: Boolean = rowIterator.hasNext -} - case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, @@ -198,11 +154,8 @@ case class InMemoryRelation( // TODO: need better abstraction for two iterators here val batchedRDD = child.execute().mapPartitionsInternal { rowIterator => - if (!usePartitionLevelMetadata) { - new CachedBatchIterator(rowIterator, output, batchSize, useCompression, batchStats) - } else { - new CachedPartitionIterator(rowIterator, output, batchSize, useCompression, batchStats) - } + new CachedBatchIterator(rowIterator, output, batchSize, useCompression, batchStats, + usePartitionLevelMetadata) } val cached = new CachedColumnarRDD(batchedRDD.sparkContext, batchedRDD, From c900808c279588a5629be736f56f668324b45c48 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 9 Nov 2017 16:39:00 -0800 Subject: [PATCH 32/47] update --- .../spark/sql/execution/columnar/CachedColumnarRDD.scala | 4 ++-- .../spark/sql/execution/columnar/InMemoryTableScanExec.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 022eaecd86ea..08ecbcd26109 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -53,12 +53,12 @@ class CachedColumnarRDD( val superGetOrCompute: (Partition, TaskContext) => Iterator[CachedBatch] = super.getOrCompute SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => new InterruptibleIterator[CachedBatch](context, - new CachedColumnarPartitionIterator(metadataBlock, split, context, superGetOrCompute)) + new CachedColumnarIterator(metadataBlock, split, context, superGetOrCompute)) ).getOrElse(superGetOrCompute(split, context)) } } -private[columnar] class CachedColumnarPartitionIterator( +private[columnar] class CachedColumnarIterator( val partitionStats: InternalRow, partition: Partition, context: TaskContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index f6b9282d4f92..5ef7e19f851b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -230,7 +230,7 @@ case class InMemoryTableScanExec( partitionFilter.initialize(index) cachedBatchIterator.asInstanceOf[InterruptibleIterator[_]].delegate match { - case cachedIter: CachedColumnarPartitionIterator + case cachedIter: CachedColumnarIterator if !partitionFilter.eval(cachedIter.partitionStats) => // scalastyle:off println(s"skipped partition $index") From 2caa7fccc3adeb18aac40ad526116b7b8ea6cb34 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 10 Nov 2017 08:44:13 -0800 Subject: [PATCH 33/47] add first test case fix compilation of tests fix tests revise the test fix test revise the test add missing file revise the test revise the test revise the test revise the test revise the test revise the test revise the test revise the test --- .../execution/columnar/InMemoryRelation.scala | 1 - .../columnar/InMemoryTableScanExec.scala | 2 +- .../columnar/InMemoryColumnarQuerySuite.scala | 30 +++++++++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 8570ecb1907b..4b5323ec87c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -152,7 +152,6 @@ case class InMemoryRelation( private def buildBuffers(): Unit = { val output = child.output - // TODO: need better abstraction for two iterators here val batchedRDD = child.execute().mapPartitionsInternal { rowIterator => new CachedBatchIterator(rowIterator, output, batchSize, useCompression, batchStats, usePartitionLevelMetadata) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 5ef7e19f851b..27a8b81a1cc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -234,10 +234,10 @@ case class InMemoryTableScanExec( if !partitionFilter.eval(cachedIter.partitionStats) => // scalastyle:off println(s"skipped partition $index") - // scalastyle:on Iterator() case _ => doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) + // scalastyle:on } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index e662e294228d..7e7692dfa304 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning @@ -479,4 +480,33 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("table cache can prune unnecessary partitions correctly") { + // scalastyle:off + var bytesReadWithoutPruning = 0L + var bytesReadWithPruning = 0L + var inMemoryPartitionMetadata = false + sparkContext.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val metrics = taskEnd.taskMetrics + if (inMemoryPartitionMetadata) { + bytesReadWithPruning += metrics.inputMetrics.bytesRead + } else { + bytesReadWithoutPruning += metrics.inputMetrics.bytesRead + } + } + }) + Seq("true", "false").foreach { enabled => + withSQLConf(SQLConf.IN_MEMORY_PARTITION_METADATA.key -> enabled) { + inMemoryPartitionMetadata = conf.inMemoryPartitionMetadata + val df1 = (0 until 1000000).toDF("value").repartition(4).cache() + df1.where("value >= 999999").collect() + val resultArr = df1.where("value >= 999999").collect() + assert(resultArr.length == 1) + assert(resultArr.head.getInt(0) == 999999) + df1.unpersist(true) + } + } + assert(bytesReadWithoutPruning > bytesReadWithPruning * 3) + } } From a9f12565b45cc4eb5b3dc94a357c1cba3843edaa Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 10 Nov 2017 11:20:18 -0800 Subject: [PATCH 34/47] test for remove metadata block test for remove metadata block fix the test fix the test fix the test --- .../columnar/CachedColumnarRDD.scala | 26 +++++---- .../columnar/InMemoryColumnarQuerySuite.scala | 53 +++++++++++++++++-- 2 files changed, 60 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 08ecbcd26109..2cd26f72ae0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -30,19 +30,7 @@ class CachedColumnarRDD( extends RDD[CachedBatch](_sc, Seq(new OneToOneDependency(dataRDD))) { override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { - if (containsPartitionMetadata) { - val parentIterator = dataRDD.iterator(split, context) - if (!parentIterator.hasNext) { - Iterator() - } else { - val cachedBatch = parentIterator.next() - SparkEnv.get.blockManager.putSingle(RDDPartitionMetadataBlockId(id, split.index), - cachedBatch.stats, expectedStorageLevel) - Iterator(cachedBatch) - } - } else { - firstParent.iterator(split, context) - } + firstParent.iterator(split, context) } override protected def getPartitions: Array[Partition] = dataRDD.partitions @@ -54,7 +42,17 @@ class CachedColumnarRDD( SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => new InterruptibleIterator[CachedBatch](context, new CachedColumnarIterator(metadataBlock, split, context, superGetOrCompute)) - ).getOrElse(superGetOrCompute(split, context)) + ).getOrElse { + val batchIter = superGetOrCompute(split, context) + if (containsPartitionMetadata && getStorageLevel != StorageLevel.NONE && batchIter.hasNext) { + val cachedBatch = batchIter.next() + SparkEnv.get.blockManager.putSingle(metadataBlockId, cachedBatch.stats, + expectedStorageLevel) + new InterruptibleIterator[CachedBatch](context, Iterator(cachedBatch)) + } else { + batchIter + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 7e7692dfa304..5e4966a9f515 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import org.apache.spark.SparkEnv import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} @@ -30,6 +31,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.RDDPartitionMetadataBlockId import org.apache.spark.storage.StorageLevel._ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { @@ -482,7 +484,6 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("table cache can prune unnecessary partitions correctly") { - // scalastyle:off var bytesReadWithoutPruning = 0L var bytesReadWithPruning = 0L var inMemoryPartitionMetadata = false @@ -499,14 +500,56 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { Seq("true", "false").foreach { enabled => withSQLConf(SQLConf.IN_MEMORY_PARTITION_METADATA.key -> enabled) { inMemoryPartitionMetadata = conf.inMemoryPartitionMetadata - val df1 = (0 until 1000000).toDF("value").repartition(4).cache() - df1.where("value >= 999999").collect() - val resultArr = df1.where("value >= 999999").collect() + val df1 = (0 until 100000).toDF("value").repartition(4).cache() + df1.where("value >= 99999").collect() + val resultArr = df1.where("value >= 99999").collect() assert(resultArr.length == 1) - assert(resultArr.head.getInt(0) == 999999) + assert(resultArr.head.getInt(0) == 99999) df1.unpersist(true) } } + assert(bytesReadWithoutPruning > 0) + assert(bytesReadWithPruning > 0) assert(bytesReadWithoutPruning > bytesReadWithPruning * 3) } + + test("generate correct results when metadata block is removed") { + var bytesReadWithMetadata = 0L + var bytesReadWithoutMetadata = 0L + @volatile var removePartitionMetadata = false + sparkContext.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val metrics = taskEnd.taskMetrics + if (removePartitionMetadata) { + bytesReadWithoutMetadata += metrics.inputMetrics.bytesRead + } else { + bytesReadWithMetadata += metrics.inputMetrics.bytesRead + } + } + }) + Seq("true").foreach { enabled => + withSQLConf(SQLConf.IN_MEMORY_PARTITION_METADATA.key -> enabled) { + removePartitionMetadata = true + val df1 = (0 until 100000).toDF("value").repartition(4).cache() + val inMemoryRelation = df1.queryExecution.optimizedPlan.collect { + case m: InMemoryRelation => m + } + df1.where("value >= 99999").collect() + (0 until 4).foreach(partitionId => SparkEnv.get.blockManager.removeBlock( + RDDPartitionMetadataBlockId(inMemoryRelation.head.cachedColumnBuffers.id, partitionId))) + var resultArr = df1.where("value >= 99999").collect() + assert(resultArr.length === 1) + assert(resultArr.head.getInt(0) === 99999) + // scalastyle:off + removePartitionMetadata = false + resultArr = df1.where("value >= 99999").collect() + assert(resultArr.length === 1) + assert(resultArr.head.getInt(0) === 99999) + df1.unpersist(blocking = true) + assert(bytesReadWithMetadata > 0) + assert(bytesReadWithoutMetadata > 0) + assert(bytesReadWithoutMetadata > bytesReadWithMetadata * 3) + } + } + } } From b5d60943580e9b3095718b87457895ab53da2566 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 10 Nov 2017 20:40:11 -0800 Subject: [PATCH 35/47] generate correct results when data block is removed --- .../columnar/InMemoryColumnarQuerySuite.scala | 43 ++++++++++++++++++- 1 file changed, 41 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 5e4966a9f515..4de87d56b92d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ import org.apache.spark.sql.types._ -import org.apache.spark.storage.RDDPartitionMetadataBlockId +import org.apache.spark.storage.{RDDBlockId, RDDPartitionMetadataBlockId} import org.apache.spark.storage.StorageLevel._ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { @@ -486,7 +486,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { test("table cache can prune unnecessary partitions correctly") { var bytesReadWithoutPruning = 0L var bytesReadWithPruning = 0L - var inMemoryPartitionMetadata = false + @volatile var inMemoryPartitionMetadata = false sparkContext.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val metrics = taskEnd.taskMetrics @@ -552,4 +552,43 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("generate correct results when data block is removed") { + var bytesReadWithCachedBlock = 0L + var bytesReadWithoutCachedBlock = 0L + @volatile var removeCachedBlock = false + sparkContext.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val metrics = taskEnd.taskMetrics + if (removeCachedBlock) { + bytesReadWithoutCachedBlock += metrics.inputMetrics.bytesRead + } else { + bytesReadWithCachedBlock += metrics.inputMetrics.bytesRead + } + } + }) + Seq("true").foreach { enabled => + withSQLConf(SQLConf.IN_MEMORY_PARTITION_METADATA.key -> enabled) { + removeCachedBlock = true + val df1 = (0 until 100000).toDF("value").repartition(4).cache() + val inMemoryRelation = df1.queryExecution.optimizedPlan.collect { + case m: InMemoryRelation => m + } + df1.where("value >= 99999").collect() + (0 until 4).foreach(partitionId => SparkEnv.get.blockManager.removeBlock( + RDDBlockId(inMemoryRelation.head.cachedColumnBuffers.id, partitionId))) + var resultArr = df1.where("value >= 99999").collect() + assert(resultArr.length === 1) + assert(resultArr.head.getInt(0) === 99999) + // scalastyle:off + removeCachedBlock = false + resultArr = df1.where("value >= 99999").collect() + assert(resultArr.length === 1) + assert(resultArr.head.getInt(0) === 99999) + df1.unpersist(blocking = true) + assert(bytesReadWithCachedBlock > 0) + assert(bytesReadWithoutCachedBlock == 0) + } + } + } } From 3b51c9aaabea1e1a6658eb4dd5875af831f1d5c6 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 10:00:46 -0800 Subject: [PATCH 36/47] try to avoid unnecessary tasks --- .../columnar/CachedColumnarRDD.scala | 21 +++++++- .../columnar/FilteredCachedColumnarRDD.scala | 50 +++++++++++++++++++ .../execution/columnar/InMemoryRelation.scala | 3 ++ .../columnar/InMemoryTableScanExec.scala | 41 ++++++++------- 4 files changed, 95 insertions(+), 20 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 2cd26f72ae0f..dc3e945344c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -17,12 +17,16 @@ package org.apache.spark.sql.execution.columnar +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.storage.{RDDPartitionMetadataBlockId, StorageLevel} -class CachedColumnarRDD( +private[columnar] class CachedColumnarRDD( @transient private var _sc: SparkContext, private var dataRDD: RDD[CachedBatch], containsPartitionMetadata: Boolean, @@ -56,6 +60,21 @@ class CachedColumnarRDD( } } +private[columnar] object CachedColumnarRDD { + + private val rddIdToMetadata = new ConcurrentHashMap[Int, Seq[InternalRow]]() + + def collectStats(rdd: RDD[CachedBatch]): Unit = { + val metadataBlocks = rdd.partitions.indices.map { + partitionId => SparkEnv.get.blockManager.getSingle[InternalRow]( + RDDPartitionMetadataBlockId(rdd.id, partitionId)).get + } + rddIdToMetadata.put(rdd.id, metadataBlocks) + } + + def fetchMetadataForRDD(rddId: Int): Option[Seq[InternalRow]] = rddIdToMetadata.asScala.get(rddId) +} + private[columnar] class CachedColumnarIterator( val partitionStats: InternalRow, partition: Partition, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala new file mode 100644 index 000000000000..2972c8d68c33 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala @@ -0,0 +1,50 @@ +/* + * 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.execution.columnar + +import org.apache.spark._ +import org.apache.spark.rdd.RDD + +private[columnar] class FilteredCachedColumnarPartition( + val partitionIndex: Int, + val parentPartitionIndex: Int) extends Partition { + + override def index: Int = partitionIndex +} + +private class PartialDependency[T](rdd: RDD[T], partitions: Array[Partition]) + extends NarrowDependency[T](rdd) { + + override def getParents(partitionId: Int): Seq[Int] = { + List(partitions(partitionId).asInstanceOf[FilteredCachedColumnarPartition].parentPartitionIndex) + } +} + +private[columnar] class FilteredCachedColumnarRDD ( + @transient private var _sc: SparkContext, + private var cachedColumnarRDD: CachedColumnarRDD, + partitions: Seq[Partition]) + extends RDD[CachedBatch](_sc, Seq(new PartialDependency(cachedColumnarRDD, partitions.toArray))) { + + override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { + firstParent.iterator(split, context) + } + + override protected def getPartitions: Array[Partition] = partitions.toArray + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 4b5323ec87c7..ebd2ce3b23cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -147,6 +147,9 @@ case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { buildBuffers() + if (usePartitionLevelMetadata) { + CachedColumnarRDD.collectStats(_cachedColumnBuffers) + } } private def buildBuffers(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 27a8b81a1cc7..0e85c8c7cc89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.columnar -import org.apache.spark.{InterruptibleIterator, SparkEnv} +import org.apache.spark.{InterruptibleIterator, Partition, SparkEnv} import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -216,30 +216,33 @@ case class InMemoryTableScanExec( } } + private def buildFilteredRDDPartitions(metadataPartitionIds: Seq[Int]): Seq[Partition] = { + metadataPartitionIds.zipWithIndex.map { + case (parentPartitionIdx, newPartitionIdx) => + new FilteredCachedColumnarPartition(newPartitionIdx, parentPartitionIdx) + } + } + private def filteredCachedBatches(): RDD[CachedBatch] = { // Using these variables here to avoid serialization of entire objects (if referenced directly) // within the map Partitions closure. val schema = relation.partitionStatistics.schema val buffers = relation.cachedColumnBuffers - buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => - - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(index) - - cachedBatchIterator.asInstanceOf[InterruptibleIterator[_]].delegate match { - case cachedIter: CachedColumnarIterator - if !partitionFilter.eval(cachedIter.partitionStats) => - // scalastyle:off - println(s"skipped partition $index") - Iterator() - case _ => - doFilterCachedBatches(cachedBatchIterator, schema, partitionFilter) - // scalastyle:on - } - } + val metadataOfValidPartitions = CachedColumnarRDD.fetchMetadataForRDD(buffers.id).map { + metadata => + metadata.zipWithIndex. + filter { case (m, partitionIndex) => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(partitionIndex) + partitionFilter.eval(m) + }.map(_._2) + }.getOrElse(buffers.partitions.indices) + + new FilteredCachedColumnarRDD(buffers.sparkContext, buffers.asInstanceOf[CachedColumnarRDD], + buildFilteredRDDPartitions(metadataOfValidPartitions)) } protected override def doExecute(): RDD[InternalRow] = { From 59b0684a22e8a839707f22bddc3f8b6409affe64 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 11:50:45 -0800 Subject: [PATCH 37/47] collect data in the fly --- .../columnar/CachedColumnarRDD.scala | 43 ++++++++++++++----- .../execution/columnar/InMemoryRelation.scala | 3 -- .../columnar/InMemoryTableScanExec.scala | 22 +++++----- 3 files changed, 43 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index dc3e945344c2..5d13dbfe57b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -29,7 +30,7 @@ import org.apache.spark.storage.{RDDPartitionMetadataBlockId, StorageLevel} private[columnar] class CachedColumnarRDD( @transient private var _sc: SparkContext, private var dataRDD: RDD[CachedBatch], - containsPartitionMetadata: Boolean, + private[columnar] val containsPartitionMetadata: Boolean, expectedStorageLevel: StorageLevel) extends RDD[CachedBatch](_sc, Seq(new OneToOneDependency(dataRDD))) { @@ -37,6 +38,12 @@ private[columnar] class CachedColumnarRDD( firstParent.iterator(split, context) } + override def unpersist(blocking: Boolean = true): Unit = { + super.unpersist(blocking) + CachedColumnarRDD.allMetadataFetched.remove(id) + CachedColumnarRDD.rddIdToMetadata.remove(id) + } + override protected def getPartitions: Array[Partition] = dataRDD.partitions override private[spark] def getOrCompute(split: Partition, context: TaskContext): @@ -62,17 +69,33 @@ private[columnar] class CachedColumnarRDD( private[columnar] object CachedColumnarRDD { - private val rddIdToMetadata = new ConcurrentHashMap[Int, Seq[InternalRow]]() - - def collectStats(rdd: RDD[CachedBatch]): Unit = { - val metadataBlocks = rdd.partitions.indices.map { - partitionId => SparkEnv.get.blockManager.getSingle[InternalRow]( - RDDPartitionMetadataBlockId(rdd.id, partitionId)).get + private val rddIdToMetadata = new ConcurrentHashMap[Int, mutable.ArraySeq[Option[InternalRow]]]() + private val allMetadataFetched = new ConcurrentHashMap[Int, Boolean]() + + def collectStats(rdd: RDD[CachedBatch]): IndexedSeq[Option[InternalRow]] = { + if (allMetadataFetched.contains(rdd.id)) { + rddIdToMetadata.get(rdd.id) + } else { + val updatedMetadataBlocks = rdd.partitions.indices.map { + partitionId => { + if (!rddIdToMetadata.contains(rdd.id)) { + rddIdToMetadata.put(rdd.id, new mutable.ArraySeq[Option[InternalRow]]( + rdd.partitions.length)) + } + rddIdToMetadata.get(rdd.id)(partitionId).orElse{ + val metadata = SparkEnv.get.blockManager.getSingle[InternalRow]( + RDDPartitionMetadataBlockId(rdd.id, partitionId)) + rddIdToMetadata.get(rdd.id).update(partitionId, metadata) + metadata + } + } + } + if (updatedMetadataBlocks.forall(_.isDefined)) { + allMetadataFetched.put(rdd.id, true) + } + updatedMetadataBlocks } - rddIdToMetadata.put(rdd.id, metadataBlocks) } - - def fetchMetadataForRDD(rddId: Int): Option[Seq[InternalRow]] = rddIdToMetadata.asScala.get(rddId) } private[columnar] class CachedColumnarIterator( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index ebd2ce3b23cb..4b5323ec87c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -147,9 +147,6 @@ case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { buildBuffers() - if (usePartitionLevelMetadata) { - CachedColumnarRDD.collectStats(_cachedColumnBuffers) - } } private def buildBuffers(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 0e85c8c7cc89..1d266b7662ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -229,18 +229,16 @@ case class InMemoryTableScanExec( val schema = relation.partitionStatistics.schema val buffers = relation.cachedColumnBuffers - val metadataOfValidPartitions = CachedColumnarRDD.fetchMetadataForRDD(buffers.id).map { - metadata => - metadata.zipWithIndex. - filter { case (m, partitionIndex) => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(partitionIndex) - partitionFilter.eval(m) - }.map(_._2) - }.getOrElse(buffers.partitions.indices) - + val metadataOfValidPartitions = CachedColumnarRDD.collectStats(buffers).zipWithIndex.filter { + case (partitionStatsOpt, partitionIndex) => + partitionStatsOpt.forall { partitionStats => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(partitionIndex) + partitionFilter.eval(partitionStats) + } + }.map(_._2) new FilteredCachedColumnarRDD(buffers.sparkContext, buffers.asInstanceOf[CachedColumnarRDD], buildFilteredRDDPartitions(metadataOfValidPartitions)) } From 3e972cef3e5d6c0fc91cd773d5a556bc9a2dfd3a Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 11:55:39 -0800 Subject: [PATCH 38/47] fix the compilation issue --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 5d13dbfe57b5..fa1e4c1fc4d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -38,7 +38,7 @@ private[columnar] class CachedColumnarRDD( firstParent.iterator(split, context) } - override def unpersist(blocking: Boolean = true): Unit = { + override def unpersist(blocking: Boolean = true): this.type = { super.unpersist(blocking) CachedColumnarRDD.allMetadataFetched.remove(id) CachedColumnarRDD.rddIdToMetadata.remove(id) From 35c9361db0c90022212a0c2654c9a84d1e3519b4 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 11:58:15 -0800 Subject: [PATCH 39/47] fix the compilation issue --- .../apache/spark/sql/execution/columnar/CachedColumnarRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index fa1e4c1fc4d2..45739a94ba64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -39,9 +39,9 @@ private[columnar] class CachedColumnarRDD( } override def unpersist(blocking: Boolean = true): this.type = { - super.unpersist(blocking) CachedColumnarRDD.allMetadataFetched.remove(id) CachedColumnarRDD.rddIdToMetadata.remove(id) + super.unpersist(blocking) } override protected def getPartitions: Array[Partition] = dataRDD.partitions From 20b72a87a4fd41bd9c0021ef36371ca34a763058 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 12:23:13 -0800 Subject: [PATCH 40/47] fix NPE --- .../spark/sql/execution/columnar/CachedColumnarRDD.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 45739a94ba64..b639b5879d19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -73,12 +73,12 @@ private[columnar] object CachedColumnarRDD { private val allMetadataFetched = new ConcurrentHashMap[Int, Boolean]() def collectStats(rdd: RDD[CachedBatch]): IndexedSeq[Option[InternalRow]] = { - if (allMetadataFetched.contains(rdd.id)) { + if (allMetadataFetched.containsKey(rdd.id)) { rddIdToMetadata.get(rdd.id) } else { val updatedMetadataBlocks = rdd.partitions.indices.map { partitionId => { - if (!rddIdToMetadata.contains(rdd.id)) { + if (!rddIdToMetadata.containsKey(rdd.id)) { rddIdToMetadata.put(rdd.id, new mutable.ArraySeq[Option[InternalRow]]( rdd.partitions.length)) } From bfa357a24d90f964ccd64b84fb18de5336ba5553 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 12:29:03 -0800 Subject: [PATCH 41/47] fix NPE --- .../spark/sql/execution/columnar/CachedColumnarRDD.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index b639b5879d19..9eddd20bfe23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -79,8 +79,9 @@ private[columnar] object CachedColumnarRDD { val updatedMetadataBlocks = rdd.partitions.indices.map { partitionId => { if (!rddIdToMetadata.containsKey(rdd.id)) { - rddIdToMetadata.put(rdd.id, new mutable.ArraySeq[Option[InternalRow]]( - rdd.partitions.length)) + val initSeq = new mutable.ArraySeq[Option[InternalRow]](rdd.partitions.length) + initSeq.indices.foreach(idx => initSeq(idx) = None) + rddIdToMetadata.put(rdd.id, initSeq) } rddIdToMetadata.get(rdd.id)(partitionId).orElse{ val metadata = SparkEnv.get.blockManager.getSingle[InternalRow]( From 0fd6b6841ee7c0bca7812b8c458fdd0c3d6f6ff0 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 12:56:24 -0800 Subject: [PATCH 42/47] fix imcompatibility of partition --- .../sql/execution/columnar/FilteredCachedColumnarRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala index 2972c8d68c33..e10f4eeef321 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala @@ -45,6 +45,6 @@ private[columnar] class FilteredCachedColumnarRDD ( firstParent.iterator(split, context) } - override protected def getPartitions: Array[Partition] = partitions.toArray + override protected def getPartitions: Array[Partition] = cachedColumnarRDD.partitions } From dc54be516c0767d9bb1234a506b30ad07daa4841 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 13:18:07 -0800 Subject: [PATCH 43/47] fix the parent inherience mechanism --- .../columnar/FilteredCachedColumnarRDD.scala | 15 +++++++++------ .../columnar/InMemoryTableScanExec.scala | 8 ++++---- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala index e10f4eeef321..eaed5b35559f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/FilteredCachedColumnarRDD.scala @@ -22,7 +22,7 @@ import org.apache.spark.rdd.RDD private[columnar] class FilteredCachedColumnarPartition( val partitionIndex: Int, - val parentPartitionIndex: Int) extends Partition { + val parentPartition: Partition) extends Partition { override def index: Int = partitionIndex } @@ -31,20 +31,23 @@ private class PartialDependency[T](rdd: RDD[T], partitions: Array[Partition]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int): Seq[Int] = { - List(partitions(partitionId).asInstanceOf[FilteredCachedColumnarPartition].parentPartitionIndex) + List(partitions(partitionId).asInstanceOf[FilteredCachedColumnarPartition]. + parentPartition.index) } } private[columnar] class FilteredCachedColumnarRDD ( @transient private var _sc: SparkContext, private var cachedColumnarRDD: CachedColumnarRDD, - partitions: Seq[Partition]) - extends RDD[CachedBatch](_sc, Seq(new PartialDependency(cachedColumnarRDD, partitions.toArray))) { + acceptedPartitions: Seq[Partition]) + extends RDD[CachedBatch]( + _sc, Seq(new PartialDependency(cachedColumnarRDD, acceptedPartitions.toArray))) { override def compute(split: Partition, context: TaskContext): Iterator[CachedBatch] = { - firstParent.iterator(split, context) + val filteredCachedColumnarPartition = split.asInstanceOf[FilteredCachedColumnarPartition] + firstParent.iterator(filteredCachedColumnarPartition.parentPartition, context) } - override protected def getPartitions: Array[Partition] = cachedColumnarRDD.partitions + override protected def getPartitions: Array[Partition] = acceptedPartitions.toArray } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 1d266b7662ae..c4c9042a10ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -216,10 +216,10 @@ case class InMemoryTableScanExec( } } - private def buildFilteredRDDPartitions(metadataPartitionIds: Seq[Int]): Seq[Partition] = { + private def buildFilteredRDDPartitions(metadataPartitionIds: Seq[Partition]): Seq[Partition] = { metadataPartitionIds.zipWithIndex.map { - case (parentPartitionIdx, newPartitionIdx) => - new FilteredCachedColumnarPartition(newPartitionIdx, parentPartitionIdx) + case (parentPartition, newPartitionIdx) => + new FilteredCachedColumnarPartition(newPartitionIdx, parentPartition) } } @@ -238,7 +238,7 @@ case class InMemoryTableScanExec( partitionFilter.initialize(partitionIndex) partitionFilter.eval(partitionStats) } - }.map(_._2) + }.map(_._2).map(partitionIndex => buffers.partitions(partitionIndex)) new FilteredCachedColumnarRDD(buffers.sparkContext, buffers.asInstanceOf[CachedColumnarRDD], buildFilteredRDDPartitions(metadataOfValidPartitions)) } From 1afbf130051dc1d0a9b997d8a47ae5aaa5bb3a10 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 21 Nov 2017 19:33:02 -0800 Subject: [PATCH 44/47] add missing filtering for cachedBatch --- .../columnar/InMemoryTableScanExec.scala | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index c4c9042a10ec..343ecd7321fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -196,23 +196,29 @@ case class InMemoryTableScanExec( private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning private def doFilterCachedBatches( - cachedBatchIterator: Iterator[CachedBatch], - partitionStatsSchema: Seq[AttributeReference], - partitionFilter: GenPredicate): Iterator[CachedBatch] = { + rdd: RDD[CachedBatch], + partitionStatsSchema: Seq[AttributeReference]): RDD[CachedBatch] = { val schemaIndex = partitionStatsSchema.zipWithIndex - cachedBatchIterator.filter { cachedBatch => - 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" + rdd.mapPartitionsWithIndex { + case (partitionIndex, cachedBatches) => + cachedBatches.filter { cachedBatch => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + partitionStatsSchema) + partitionFilter.initialize(partitionIndex) + 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 + } } - false - } else { - true - } } } @@ -239,8 +245,15 @@ case class InMemoryTableScanExec( partitionFilter.eval(partitionStats) } }.map(_._2).map(partitionIndex => buffers.partitions(partitionIndex)) - new FilteredCachedColumnarRDD(buffers.sparkContext, buffers.asInstanceOf[CachedColumnarRDD], + val prunedRDD = new FilteredCachedColumnarRDD( + buffers.sparkContext, + buffers.asInstanceOf[CachedColumnarRDD], buildFilteredRDDPartitions(metadataOfValidPartitions)) + if (!inMemoryPartitionPruningEnabled) { + prunedRDD + } else { + doFilterCachedBatches(prunedRDD, schema) + } } protected override def doExecute(): RDD[InternalRow] = { From accd5493464d5bd5f7401285defbd8ca2628ae68 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 23 Nov 2017 23:23:57 -0800 Subject: [PATCH 45/47] remove CachedColumnarIterator --- .../columnar/CachedColumnarRDD.scala | 26 ++----------------- 1 file changed, 2 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala index 9eddd20bfe23..3ac3574b2ab5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/CachedColumnarRDD.scala @@ -50,9 +50,8 @@ private[columnar] class CachedColumnarRDD( Iterator[CachedBatch] = { val metadataBlockId = RDDPartitionMetadataBlockId(id, split.index) val superGetOrCompute: (Partition, TaskContext) => Iterator[CachedBatch] = super.getOrCompute - SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(metadataBlock => - new InterruptibleIterator[CachedBatch](context, - new CachedColumnarIterator(metadataBlock, split, context, superGetOrCompute)) + SparkEnv.get.blockManager.getSingle[InternalRow](metadataBlockId).map(_ => + superGetOrCompute(split, context) ).getOrElse { val batchIter = superGetOrCompute(split, context) if (containsPartitionMetadata && getStorageLevel != StorageLevel.NONE && batchIter.hasNext) { @@ -98,24 +97,3 @@ private[columnar] object CachedColumnarRDD { } } } - -private[columnar] class CachedColumnarIterator( - val partitionStats: InternalRow, - partition: Partition, - context: TaskContext, - fetchRDDPartition: (Partition, TaskContext) => Iterator[CachedBatch]) - extends Iterator[CachedBatch] { - - private var delegate: Iterator[CachedBatch] = _ - - override def hasNext: Boolean = { - if (delegate == null) { - delegate = fetchRDDPartition(partition, context) - } - delegate.hasNext - } - - override def next(): CachedBatch = { - delegate.next() - } -} From a853ce6ffa8c8b09271441473bcf7c1f9d463a9d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 24 Nov 2017 00:27:25 -0800 Subject: [PATCH 46/47] fix filtering logic --- .../columnar/InMemoryTableScanExec.scala | 30 +++++++++---------- 1 file changed, 15 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 343ecd7321fe..3420e05bd014 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -235,23 +235,23 @@ case class InMemoryTableScanExec( val schema = relation.partitionStatistics.schema val buffers = relation.cachedColumnBuffers - val metadataOfValidPartitions = CachedColumnarRDD.collectStats(buffers).zipWithIndex.filter { - case (partitionStatsOpt, partitionIndex) => - partitionStatsOpt.forall { partitionStats => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - schema) - partitionFilter.initialize(partitionIndex) - partitionFilter.eval(partitionStats) - } - }.map(_._2).map(partitionIndex => buffers.partitions(partitionIndex)) - val prunedRDD = new FilteredCachedColumnarRDD( - buffers.sparkContext, - buffers.asInstanceOf[CachedColumnarRDD], - buildFilteredRDDPartitions(metadataOfValidPartitions)) if (!inMemoryPartitionPruningEnabled) { - prunedRDD + doFilterCachedBatches(buffers, schema) } else { + val metadataOfValidPartitions = CachedColumnarRDD.collectStats(buffers).zipWithIndex.filter { + case (partitionStatsOpt, partitionIndex) => + partitionStatsOpt.forall { partitionStats => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + schema) + partitionFilter.initialize(partitionIndex) + partitionFilter.eval(partitionStats) + } + }.map(_._2).map(partitionIndex => buffers.partitions(partitionIndex)) + val prunedRDD = new FilteredCachedColumnarRDD( + buffers.sparkContext, + buffers.asInstanceOf[CachedColumnarRDD], + buildFilteredRDDPartitions(metadataOfValidPartitions)) doFilterCachedBatches(prunedRDD, schema) } } From 9d450addd095e2ac78770bdd03c3bf77817379e8 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 24 Nov 2017 16:20:33 -0800 Subject: [PATCH 47/47] fix the failed test --- .../columnar/InMemoryTableScanExec.scala | 34 +++++++++++-------- 1 file changed, 19 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 3420e05bd014..c030a125aced 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -201,23 +201,27 @@ case class InMemoryTableScanExec( val schemaIndex = partitionStatsSchema.zipWithIndex rdd.mapPartitionsWithIndex { case (partitionIndex, cachedBatches) => - cachedBatches.filter { cachedBatch => - val partitionFilter = newPredicate( - partitionFilters.reduceOption(And).getOrElse(Literal(true)), - partitionStatsSchema) - partitionFilter.initialize(partitionIndex) - 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" + if (inMemoryPartitionPruningEnabled) { + cachedBatches.filter { cachedBatch => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + partitionStatsSchema) + partitionFilter.initialize(partitionIndex) + 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 } - false - } else { - true } + } else { + cachedBatches } } }