Skip to content

Commit

Permalink
version1
Browse files Browse the repository at this point in the history
  • Loading branch information
Zouxxyy committed Jan 26, 2024
1 parent c3c364e commit 53028c9
Show file tree
Hide file tree
Showing 12 changed files with 386 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,5 +27,6 @@ case class PaimonScan(
table: Table,
requiredSchema: StructType,
filters: Array[Predicate],
reservedFilters: Array[Filter],
pushDownLimit: Option[Int])
extends PaimonBaseScan(table, requiredSchema, filters, pushDownLimit)
extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit)
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.paimon.spark.statistics

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.connector.read.Statistics
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation

trait StatisticsHelper extends StatisticsHelperBase {
protected def toV1Stats(v2Stats: Statistics, attrs: Seq[Attribute]): logical.Statistics = {
DataSourceV2Relation.transformV2Stats(v2Stats, None, conf.defaultSizeInBytes)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,4 +17,44 @@
*/
package org.apache.paimon.spark.sql

class AnalyzeTableTest extends AnalyzeTableTestBase {}
import org.junit.jupiter.api.Assertions

class AnalyzeTableTest extends AnalyzeTableTestBase {
test("Paimon analyze: spark use col stats") {
spark.sql(s"""
|CREATE TABLE T (id STRING, name STRING, i INT, l LONG)
|USING PAIMON
|TBLPROPERTIES ('primary-key'='id')
|""".stripMargin)

spark.sql(s"INSERT INTO T VALUES ('1', 'a', 1, 1)")
spark.sql(s"INSERT INTO T VALUES ('2', 'aaa', 1, 2)")
spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS")

val stats = getScanStatistic("SELECT * FROM T")
Assertions.assertEquals(2L, stats.rowCount.get.longValue())
Assertions.assertEquals(4, stats.attributeStats.size)
}

test("Paimon analyze: partition filter push down hit") {
spark.sql(s"""
|CREATE TABLE T (id INT, name STRING, pt INT)
|TBLPROPERTIES ('primary-key'='id, pt', 'bucket'='2')
|PARTITIONED BY (pt)
|""".stripMargin)

spark.sql("INSERT INTO T VALUES (1, 'a', 1), (2, 'b', 1), (3, 'c', 2), (4, 'd', 3)")
spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS")

// paimon will reserve partition filter and not return it to spark, we need to ensure stats are filtered correctly.
// partition push down hit
var sql = "SELECT * FROM T WHERE pt < 1"
Assertions.assertEquals(0L, getScanStatistic(sql).rowCount.get.longValue())
checkAnswer(spark.sql(sql), Nil)

// partition push down not hit
sql = "SELECT * FROM T WHERE id < 1"
Assertions.assertEquals(4L, getScanStatistic(sql).rowCount.get.longValue())
checkAnswer(spark.sql(sql), Nil)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,4 +17,44 @@
*/
package org.apache.paimon.spark.sql

class AnalyzeTableTest extends AnalyzeTableTestBase {}
import org.junit.jupiter.api.Assertions

class AnalyzeTableTest extends AnalyzeTableTestBase {
test("Paimon analyze: spark use col stats") {
spark.sql(s"""
|CREATE TABLE T (id STRING, name STRING, i INT, l LONG)
|USING PAIMON
|TBLPROPERTIES ('primary-key'='id')
|""".stripMargin)

spark.sql(s"INSERT INTO T VALUES ('1', 'a', 1, 1)")
spark.sql(s"INSERT INTO T VALUES ('2', 'aaa', 1, 2)")
spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS")

val stats = getScanStatistic("SELECT * FROM T")
Assertions.assertEquals(2L, stats.rowCount.get.longValue())
Assertions.assertEquals(4, stats.attributeStats.size)
}

test("Paimon analyze: partition filter push down hit") {
spark.sql(s"""
|CREATE TABLE T (id INT, name STRING, pt INT)
|TBLPROPERTIES ('primary-key'='id, pt', 'bucket'='2')
|PARTITIONED BY (pt)
|""".stripMargin)

spark.sql("INSERT INTO T VALUES (1, 'a', 1), (2, 'b', 1), (3, 'c', 2), (4, 'd', 3)")
spark.sql(s"ANALYZE TABLE T COMPUTE STATISTICS FOR ALL COLUMNS")

// paimon will reserve partition filter and not return it to spark, we need to ensure stats are filtered correctly.
// partition push down hit
var sql = "SELECT * FROM T WHERE pt < 1"
Assertions.assertEquals(0L, getScanStatistic(sql).rowCount.get.longValue())
checkAnswer(spark.sql(sql), Nil)

// partition push down not hit
sql = "SELECT * FROM T WHERE id < 1"
Assertions.assertEquals(4L, getScanStatistic(sql).rowCount.get.longValue())
checkAnswer(spark.sql(sql), Nil)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,30 +17,36 @@
*/
package org.apache.paimon.spark

import org.apache.paimon.CoreOptions
import org.apache.paimon.{stats, CoreOptions}
import org.apache.paimon.predicate.{Predicate, PredicateBuilder}
import org.apache.paimon.spark.sources.PaimonMicroBatchStream
import org.apache.paimon.spark.statistics.StatisticsHelper
import org.apache.paimon.table.{DataTable, FileStoreTable, Table}
import org.apache.paimon.table.source.{ReadBuilder, Split}
import org.apache.paimon.types.RowType

import org.apache.spark.sql.connector.metric.CustomMetric
import org.apache.spark.sql.connector.read.{Batch, Scan, Statistics, SupportsReportStatistics}
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType

import java.util.Optional

import scala.collection.JavaConverters._

abstract class PaimonBaseScan(
table: Table,
requiredSchema: StructType,
filters: Array[Predicate],
reservedFilters: Array[Filter],
pushDownLimit: Option[Int])
extends Scan
with SupportsReportStatistics
with ScanHelper {
with ScanHelper
with StatisticsHelper {

private val tableRowType = table.rowType
val tableRowType: RowType = table.rowType

private lazy val tableSchema = SparkTypeUtils.fromPaimonRowType(tableRowType)

Expand All @@ -50,6 +56,8 @@ abstract class PaimonBaseScan(

override val coreOptions: CoreOptions = CoreOptions.fromMap(table.options())

lazy val statistics: Optional[stats.Statistics] = table.statistics()

lazy val readBuilder: ReadBuilder = {
val _readBuilder = table.newReadBuilder()

Expand Down Expand Up @@ -89,7 +97,12 @@ abstract class PaimonBaseScan(
}

override def estimateStatistics(): Statistics = {
PaimonStatistics(this)
val stats = PaimonStatistics(this)
if (reservedFilters.nonEmpty) {
filterStatistics(stats, reservedFilters)
} else {
stats
}
}

override def supportedCustomMetrics: Array[CustomMetric] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,12 @@ abstract class PaimonBaseScanBuilder(table: Table)

protected var pushed: Array[(Filter, Predicate)] = Array.empty

protected var reservedFilters: Array[Filter] = Array.empty

protected var pushDownLimit: Option[Int] = None

override def build(): Scan = {
PaimonScan(table, requiredSchema, pushed.map(_._2), pushDownLimit)
PaimonScan(table, requiredSchema, pushed.map(_._2), reservedFilters, pushDownLimit)
}

/**
Expand All @@ -51,6 +53,7 @@ abstract class PaimonBaseScanBuilder(table: Table)
override def pushFilters(filters: Array[Filter]): Array[Filter] = {
val pushable = mutable.ArrayBuffer.empty[(Filter, Predicate)]
val postScan = mutable.ArrayBuffer.empty[Filter]
val reserved = mutable.ArrayBuffer.empty[Filter]

val converter = new SparkFilterConverter(table.rowType)
val visitor = new PartitionPredicateVisitor(table.partitionKeys())
Expand All @@ -59,7 +62,11 @@ abstract class PaimonBaseScanBuilder(table: Table)
try {
val predicate = converter.convert(filter)
pushable.append((filter, predicate))
if (!predicate.visit(visitor)) postScan.append(filter)
if (!predicate.visit(visitor)) {
postScan.append(filter)
} else {
reserved.append(filter)
}
} catch {
case e: UnsupportedOperationException =>
logWarning(e.getMessage)
Expand All @@ -70,6 +77,9 @@ abstract class PaimonBaseScanBuilder(table: Table)
if (pushable.nonEmpty) {
this.pushed = pushable.toArray
}
if (reserved.nonEmpty) {
this.reservedFilters = reserved.toArray
}
postScan.toArray
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,16 @@ import scala.language.implicitConversions
object PaimonImplicits {
implicit def toScalaOption[T](o: Optional[T]): Option[T] = {
if (o.isPresent) {
Option.apply(o.get())
Some(o.get)
} else {
None
}
}

implicit def toJavaOptional[T, U](o: Option[T]): Optional[U] = {
o match {
case Some(t) => Optional.ofNullable(t.asInstanceOf[U])
case _ => Optional.empty[U]
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,9 @@ case class PaimonScan(
table: Table,
requiredSchema: StructType,
filters: Array[Predicate],
reservedFilters: Array[Filter],
pushDownLimit: Option[Int])
extends PaimonBaseScan(table, requiredSchema, filters, pushDownLimit)
extends PaimonBaseScan(table, requiredSchema, filters, reservedFilters, pushDownLimit)
with SupportsRuntimeFiltering {

override def filterAttributes(): Array[NamedReference] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,19 +17,89 @@
*/
package org.apache.paimon.spark

import org.apache.paimon.stats
import org.apache.paimon.stats.ColStats
import org.apache.paimon.types.DataType

import org.apache.spark.sql.Utils
import org.apache.spark.sql.catalyst.plans.logical.ColumnStat
import org.apache.spark.sql.connector.expressions.NamedReference
import org.apache.spark.sql.connector.read.Statistics
import org.apache.spark.sql.connector.read.colstats.ColumnStatistics

import java.util.{Optional, OptionalLong}

import java.util.OptionalLong
import scala.collection.JavaConverters._

case class PaimonStatistics[T <: PaimonBaseScan](scan: T) extends Statistics {

private lazy val rowCount: Long = scan.getSplits.map(_.rowCount).sum

private lazy val scannedTotalSize: Long = rowCount * scan.readSchema().defaultSize

override def sizeInBytes(): OptionalLong = OptionalLong.of(scannedTotalSize)
private lazy val paimonStats: Optional[stats.Statistics] = scan.statistics

override def sizeInBytes(): OptionalLong = if (paimonStats.isPresent)
paimonStats.get().mergedRecordSize()
else OptionalLong.of(scannedTotalSize)

override def numRows(): OptionalLong =
if (paimonStats.isPresent) paimonStats.get().mergedRecordCount() else OptionalLong.of(rowCount)

override def columnStats(): java.util.Map[NamedReference, ColumnStatistics] = {
val requiredFields = scan.readSchema().fieldNames.toList.asJava
val resultMap = new java.util.HashMap[NamedReference, ColumnStatistics]()
if (paimonStats.isPresent) {
val paimonColStats = paimonStats.get().colStats()
scan.tableRowType.getFields
.stream()
.filter(
field => requiredFields.contains(field.name) && paimonColStats.containsKey(field.name()))
.forEach(
f =>
resultMap.put(
Utils.fieldReference(f.name()),
PaimonColumnStats(f.`type`(), paimonColStats.get(f.name()))))
}
resultMap
}
}

case class PaimonColumnStats(
override val nullCount: OptionalLong,
override val min: Optional[Object],
override val max: Optional[Object],
override val distinctCount: OptionalLong,
override val avgLen: OptionalLong,
override val maxLen: OptionalLong)
extends ColumnStatistics

override def numRows(): OptionalLong = OptionalLong.of(rowCount)
object PaimonColumnStats {
def apply(dateType: DataType, paimonColStats: ColStats[_]): PaimonColumnStats = {
PaimonColumnStats(
paimonColStats.nullCount,
Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.min().orElse(null), dateType)),
Optional.ofNullable(SparkInternalRow.fromPaimon(paimonColStats.max().orElse(null), dateType)),
paimonColStats.distinctCount,
paimonColStats.avgLen,
paimonColStats.maxLen
)
}

// TODO: extend columnStats for CBO
def apply(v1ColStats: ColumnStat): PaimonColumnStats = {
import PaimonImplicits._
PaimonColumnStats(
if (v1ColStats.nullCount.isDefined) OptionalLong.of(v1ColStats.nullCount.get.longValue())
else OptionalLong.empty(),
v1ColStats.min,
v1ColStats.max,
if (v1ColStats.distinctCount.isDefined)
OptionalLong.of(v1ColStats.distinctCount.get.longValue())
else OptionalLong.empty(),
if (v1ColStats.avgLen.isDefined) OptionalLong.of(v1ColStats.avgLen.get.longValue())
else OptionalLong.empty(),
if (v1ColStats.maxLen.isDefined) OptionalLong.of(v1ColStats.maxLen.get.longValue())
else OptionalLong.empty()
)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.paimon.spark.statistics

import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.connector.read.Statistics
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation

trait StatisticsHelper extends StatisticsHelperBase {
protected def toV1Stats(v2Stats: Statistics, attrs: Seq[Attribute]): logical.Statistics = {
DataSourceV2Relation.transformV2Stats(v2Stats, None, conf.defaultSizeInBytes, attrs)
}
}
Loading

0 comments on commit 53028c9

Please sign in to comment.