1
0

[HUDI-3567] Refactor HoodieCommonUtils to make code more reasonable (#4982)

This commit is contained in:
huberylee
2022-03-12 05:23:19 +08:00
committed by GitHub
parent b00180342e
commit 56cb49485d
11 changed files with 706 additions and 593 deletions

View File

@@ -17,7 +17,7 @@
package org.apache.spark.sql.hudi.command
import org.apache.hudi.HoodieCommonUtils
import org.apache.hudi.HoodieCLIUtils
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
@@ -42,7 +42,7 @@ case class CompactionHoodiePathCommand(path: String,
assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
s"Must compaction on a Merge On Read table.")
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, path, Map.empty)
val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, path, Map.empty)
operation match {
case SCHEDULE =>

View File

@@ -17,29 +17,24 @@
package org.apache.spark.sql.hudi.command.procedures
import org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.ValidationUtils.checkArgument
import org.apache.hudi.common.util.{ClusteringUtils, Option => HOption}
import org.apache.hudi.config.HoodieClusteringConfig
import org.apache.hudi.exception.HoodieClusteringException
import org.apache.hudi.metadata.HoodieTableMetadata
import org.apache.hudi.{HoodieCommonUtils, SparkAdapterSupport}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.hudi.{AvroConversionUtils, HoodieCLIUtils, HoodieFileIndex}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, Row}
import org.apache.spark.sql.catalyst.expressions.PredicateHelper
import org.apache.spark.sql.execution.datasources.FileStatusCache
import org.apache.spark.sql.types._
import java.util.Properties
import java.util.function.Supplier
import scala.collection.JavaConverters
import scala.collection.JavaConverters._
class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with SparkAdapterSupport with Logging {
class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with PredicateHelper with Logging {
/**
* OPTIMIZE table_name|table_path [WHERE predicate]
* [ORDER BY (col_name1 [, ...] ) ]
@@ -74,15 +69,12 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
var conf: Map[String, String] = Map.empty
predicate match {
case Some(p) =>
val partitionColumnsSchema = HoodieCommonUtils.getPartitionSchemaFromProperty(metaClient, None)
val partitionPredicate = HoodieCommonUtils.resolveFilterExpr(
spark, p.asInstanceOf[String], partitionColumnsSchema)
val partitionSelected = prunePartition(metaClient, partitionPredicate)
val prunedPartitions = prunePartition(metaClient, p.asInstanceOf[String])
conf = conf ++ Map(
HoodieClusteringConfig.PLAN_PARTITION_FILTER_MODE_NAME.key() -> "SELECTED_PARTITIONS",
HoodieClusteringConfig.PARTITION_SELECTED.key() -> partitionSelected
HoodieClusteringConfig.PARTITION_SELECTED.key() -> prunedPartitions
)
logInfo(s"Partition predicates: ${p}, partition selected: ${partitionSelected}")
logInfo(s"Partition predicates: ${p}, partition selected: ${prunedPartitions}")
case _ =>
logInfo("No partition predicates")
}
@@ -104,7 +96,7 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
.iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f)
logInfo(s"Pending clustering instants: ${pendingClustering.mkString(",")}")
val client = HoodieCommonUtils.createHoodieClientFromPath(sparkSession, basePath, conf)
val client = HoodieCLIUtils.createHoodieClientFromPath(sparkSession, basePath, conf)
val instantTime = HoodieActiveTimeline.createNewInstantTime
if (client.scheduleClusteringAtInstant(instantTime, HOption.empty())) {
pendingClustering ++= Seq(instantTime)
@@ -120,37 +112,26 @@ class RunClusteringProcedure extends BaseProcedure with ProcedureBuilder with Sp
override def build: Procedure = new RunClusteringProcedure()
def prunePartition(metaClient: HoodieTableMetaClient, partitionPredicate: Expression): String = {
val partitionSchema = HoodieCommonUtils.getPartitionSchemaFromProperty(metaClient, None)
// Get tableName meta data
val engineContext = new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext))
val properties = new Properties()
properties.putAll(JavaConverters.mapAsJavaMapConverter(sparkSession.sessionState.conf.getAllConfs).asJava)
val metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(properties).build()
val tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath,
FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue)
val sparkParsePartitionUtil = sparkAdapter.createSparkParsePartitionUtil(sparkSession.sessionState.conf)
val typedProperties = HoodieCommonUtils.getConfigProperties(sparkSession, Map.empty)
def prunePartition(metaClient: HoodieTableMetaClient, predicate: String): String = {
val options = Map(QUERY_TYPE.key() -> QUERY_TYPE_SNAPSHOT_OPT_VAL, "path" -> metaClient.getBasePath)
val hoodieFileIndex = HoodieFileIndex(sparkSession, metaClient, None, options,
FileStatusCache.getOrCreate(sparkSession))
// Resolve partition predicates
val schemaResolver = new TableSchemaResolver(metaClient)
val tableSchema = AvroConversionUtils.convertAvroSchemaToStructType(schemaResolver.getTableAvroSchema)
val condition = HoodieCatalystExpressionUtils.resolveFilterExpr(sparkSession, predicate, tableSchema)
val partitionColumns = metaClient.getTableConfig.getPartitionFields.orElse(Array[String]())
val (partitionPredicates, dataPredicates) = HoodieCatalystExpressionUtils.splitPartitionAndDataPredicates(
sparkSession, splitConjunctivePredicates(condition).toArray, partitionColumns)
checkArgument(dataPredicates.isEmpty, "Only partition predicates are allowed")
// Translate all partition path to {@code org.apache.hudi.BaseHoodieTableFileIndex.PartitionPath}
val partitionPaths = tableMetadata.getAllPartitionPaths.asScala.map(partitionPath => {
val partitionColumnValues = HoodieCommonUtils.parsePartitionColumnValues(
sparkParsePartitionUtil, typedProperties, metaClient.getBasePath,
partitionSchema, partitionColumns, partitionPath)
new PartitionPath(partitionPath, partitionColumnValues)
})
// Filter partition by predicates
val selectedPartitions = HoodieCommonUtils.prunePartition(
partitionSchema, partitionPaths, partitionPredicate)
selectedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",")
// Get all partitions and prune partition by predicates
val prunedPartitions = hoodieFileIndex.getPartitionPaths(partitionPredicates)
prunedPartitions.map(partitionPath => partitionPath.getPath).toSet.mkString(",")
}
def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = {
private def validateOrderColumns(orderColumns: String, metaClient: HoodieTableMetaClient): Unit = {
if (orderColumns == null) {
throw new HoodieClusteringException("Order columns is null")
}

View File

@@ -22,8 +22,8 @@ import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.spark.sql.catalyst.expressions.{Expression, Not}
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.hudi.DataSkippingUtils
import org.apache.spark.sql.types._
import org.apache.spark.sql.{Column, SparkSession}
import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType, VarcharType}
import org.apache.spark.sql.{Column, HoodieCatalystExpressionUtils, SparkSession}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.BeforeEach
import org.junit.jupiter.params.ParameterizedTest
@@ -73,8 +73,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase {
@ParameterizedTest
@MethodSource(Array("testBaseLookupFilterExpressionsSource", "testAdvancedLookupFilterExpressionsSource"))
def testLookupFilterExpressions(sourceExpr: String, input: Seq[IndexRow], output: Seq[String]): Unit = {
val resolvedExpr: Expression = HoodieCommonUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema)
val resolvedExpr: Expression = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema)
val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema)
val spark2 = spark
@@ -94,7 +93,7 @@ class TestDataSkippingUtils extends HoodieClientTestBase {
@ParameterizedTest
@MethodSource(Array("testStringsLookupFilterExpressionsSource"))
def testStringsLookupFilterExpressions(sourceExpr: Expression, input: Seq[IndexRow], output: Seq[String]): Unit = {
val resolvedExpr = HoodieCommonUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema)
val resolvedExpr = HoodieCatalystExpressionUtils.resolveFilterExpr(spark, sourceExpr, sourceTableSchema)
val lookupFilter = DataSkippingUtils.createColumnStatsIndexFilterExpr(resolvedExpr, indexSchema)
val spark2 = spark

View File

@@ -17,13 +17,6 @@
package org.apache.spark.sql.hudi
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
import org.apache.hudi.common.util.{Option => HOption}
import org.apache.hudi.{HoodieCommonUtils, HoodieDataSourceHelpers}
import scala.collection.JavaConverters.asScalaIteratorConverter
class TestCallProcedure extends TestHoodieSqlBase {
test("Test Call show_commits Procedure") {
@@ -136,222 +129,4 @@ class TestCallProcedure extends TestHoodieSqlBase {
assertResult(1){commits.length}
}
}
test("Test Call run_clustering Procedure By Table") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
val client = HoodieCommonUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
// Generate the first clustering plan
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())
// Generate the second clustering plan
spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)")
val secondScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(secondScheduleInstant, HOption.empty())
checkAnswer(s"call show_clustering('$tableName')")(
Seq(firstScheduleInstant, 3),
Seq(secondScheduleInstant, 1)
)
// Do clustering for all clustering plan generated above, and no new clustering
// instant will be generated because of there is no commit after the second
// clustering plan generated
spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')")
// No new commits
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, secondScheduleInstant))
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003)
)
// After clustering there should be no pending clustering.
checkAnswer(s"call show_clustering(table => '$tableName')")()
// Check the number of finished clustering instants
val finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(2)(finishedClustering.size)
// Do clustering without manual schedule(which will do the schedule if no pending clustering exists)
spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)")
spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)")
spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')")
val thirdClusteringInstant = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.findInstantsAfter(secondScheduleInstant)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
// Should have a new replace commit after the second clustering command.
assertResult(1)(thirdClusteringInstant.size)
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004),
Seq(6, "a6", 10.0, 1005)
)
}
}
}
test("Test Call run_clustering Procedure By Path") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
spark.sql(s"call run_clustering(path => '$basePath')")
checkAnswer(s"call show_clustering(path => '$basePath')")()
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
val client = HoodieCommonUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
// Generate the first clustering plan
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())
checkAnswer(s"call show_clustering(path => '$basePath')")(
Seq(firstScheduleInstant, 3)
)
// Do clustering for all the clustering plan
spark.sql(s"call run_clustering(path => '$basePath', order => 'ts')")
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002)
)
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
HoodieDataSourceHelpers.hasNewCommits(fs, basePath, firstScheduleInstant)
// Check the number of finished clustering instants
var finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(1)(finishedClustering.size)
// Do clustering without manual schedule(which will do the schedule if no pending clustering exists)
spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)")
spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)")
spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts >= 1003L')")
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004)
)
finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(2)(finishedClustering.size)
}
}
}
test("Test Call run_clustering Procedure With Partition Pruning") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
// Do clustering table with partition predicate
spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L', order => 'ts')")
// Check the num of completed clustering instant
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(1)(clusteringInstants.size)
val clusteringInstant = clusteringInstants.head
val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp)
assertResult(true)(clusteringPlan.isPresent)
assertResult(2)(clusteringPlan.get().getInputGroups.size())
checkAnswer(s"call show_clustering(table => '$tableName')")()
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002)
)
}
}
}
}

View File

@@ -0,0 +1,344 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.spark.sql.hudi
import org.apache.hadoop.fs.Path
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
import org.apache.hudi.common.util.{Option => HOption}
import org.apache.hudi.{HoodieCLIUtils, HoodieDataSourceHelpers}
import scala.collection.JavaConverters.asScalaIteratorConverter
class TestRunClusteringProcedure extends TestHoodieSqlBase {
test("Test Call run_clustering Procedure By Table") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
// Generate the first clustering plan
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())
// Generate the second clustering plan
spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)")
val secondScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(secondScheduleInstant, HOption.empty())
checkAnswer(s"call show_clustering('$tableName')")(
Seq(firstScheduleInstant, 3),
Seq(secondScheduleInstant, 1)
)
// Do clustering for all clustering plan generated above, and no new clustering
// instant will be generated because of there is no commit after the second
// clustering plan generated
spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')")
// No new commits
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
assertResult(false)(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, secondScheduleInstant))
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003)
)
// After clustering there should be no pending clustering.
checkAnswer(s"call show_clustering(table => '$tableName')")()
// Check the number of finished clustering instants
val finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(2)(finishedClustering.size)
// Do clustering without manual schedule(which will do the schedule if no pending clustering exists)
spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)")
spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)")
spark.sql(s"call run_clustering(table => '$tableName', order => 'ts')")
val thirdClusteringInstant = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.findInstantsAfter(secondScheduleInstant)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
// Should have a new replace commit after the second clustering command.
assertResult(1)(thirdClusteringInstant.size)
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004),
Seq(6, "a6", 10.0, 1005)
)
}
}
}
test("Test Call run_clustering Procedure By Path") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
spark.sql(s"call run_clustering(path => '$basePath')")
checkAnswer(s"call show_clustering(path => '$basePath')")()
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
val client = HoodieCLIUtils.createHoodieClientFromPath(spark, basePath, Map.empty)
// Generate the first clustering plan
val firstScheduleInstant = HoodieActiveTimeline.createNewInstantTime
client.scheduleClusteringAtInstant(firstScheduleInstant, HOption.empty())
checkAnswer(s"call show_clustering(path => '$basePath')")(
Seq(firstScheduleInstant, 3)
)
// Do clustering for all the clustering plan
spark.sql(s"call run_clustering(path => '$basePath', order => 'ts')")
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002)
)
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
HoodieDataSourceHelpers.hasNewCommits(fs, basePath, firstScheduleInstant)
// Check the number of finished clustering instants
var finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(1)(finishedClustering.size)
// Do clustering without manual schedule(which will do the schedule if no pending clustering exists)
spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)")
spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)")
spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts >= 1003L')")
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004)
)
finishedClustering = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(2)(finishedClustering.size)
}
}
}
test("Test Call run_clustering Procedure With Partition Pruning") {
withTempDir { tmp =>
Seq("cow", "mor").foreach { tableType =>
val tableName = generateTableName
val basePath = s"${tmp.getCanonicalPath}/$tableName"
spark.sql(
s"""
|create table $tableName (
| id int,
| name string,
| price double,
| ts long
|) using hudi
| options (
| primaryKey ='id',
| type = '$tableType',
| preCombineField = 'ts'
| )
| partitioned by(ts)
| location '$basePath'
""".stripMargin)
val fs = new Path(basePath).getFileSystem(spark.sessionState.newHadoopConf())
// Test partition pruning with single predicate
{
spark.sql(s"insert into $tableName values(1, 'a1', 10, 1000)")
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1001)")
spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002)")
checkException(
s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L and id = 10', order => 'ts')"
)("Only partition predicates are allowed")
// Do clustering table with partition predicate
spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts <= 1001L', order => 'ts')")
// There is 1 completed clustering instant
val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(1)(clusteringInstants.size)
// The latest clustering should contain 2 file groups
val clusteringInstant = clusteringInstants.last
val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp)
assertResult(true)(clusteringPlan.isPresent)
assertResult(2)(clusteringPlan.get().getInputGroups.size())
// No pending clustering instant
checkAnswer(s"call show_clustering(table => '$tableName')")()
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002)
)
}
// Test partition pruning with {@code And} predicates
{
spark.sql(s"insert into $tableName values(4, 'a4', 10, 1003)")
spark.sql(s"insert into $tableName values(5, 'a5', 10, 1004)")
spark.sql(s"insert into $tableName values(6, 'a6', 10, 1005)")
checkException(
s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L and id = 10', order => 'ts')"
)("Only partition predicates are allowed")
// Do clustering table with partition predicate
spark.sql(s"call run_clustering(table => '$tableName', predicate => 'ts > 1001L and ts <= 1005L', order => 'ts')")
// There are 2 completed clustering instants
val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(2)(clusteringInstants.size)
// The latest clustering should contain 4 file groups(1002,1003,1004,1005)
val clusteringInstant = clusteringInstants.last
val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp)
assertResult(true)(clusteringPlan.isPresent)
assertResult(4)(clusteringPlan.get().getInputGroups.size())
// No pending clustering instant
checkAnswer(s"call show_clustering(table => '$tableName')")()
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004),
Seq(6, "a6", 10.0, 1005)
)
}
// Test partition pruning with {@code And}-{@code Or} predicates
{
spark.sql(s"insert into $tableName values(7, 'a7', 10, 1006)")
spark.sql(s"insert into $tableName values(8, 'a8', 10, 1007)")
spark.sql(s"insert into $tableName values(9, 'a9', 10, 1008)")
spark.sql(s"insert into $tableName values(10, 'a10', 10, 1009)")
checkException(
s"call run_clustering(table => '$tableName', predicate => 'ts < 1007L or ts >= 1008L or id = 10', order => 'ts')"
)("Only partition predicates are allowed")
// Do clustering table with partition predicate
spark.sql(s"call run_clustering(table => '$tableName', predicate => '(ts >= 1006L and ts < 1008L) or ts >= 1009L', order => 'ts')")
// There are 3 completed clustering instants
val clusteringInstants = HoodieDataSourceHelpers.allCompletedCommitsCompactions(fs, basePath)
.getInstants
.iterator().asScala
.filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION)
.toSeq
assertResult(3)(clusteringInstants.size)
// The latest clustering should contain 3 file groups(1006,1007,1009)
val clusteringInstant = clusteringInstants.last
val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp)
assertResult(true)(clusteringPlan.isPresent)
assertResult(3)(clusteringPlan.get().getInputGroups.size())
// No pending clustering instant
checkAnswer(s"call show_clustering(table => '$tableName')")()
checkAnswer(s"select id, name, price, ts from $tableName order by id")(
Seq(1, "a1", 10.0, 1000),
Seq(2, "a2", 10.0, 1001),
Seq(3, "a3", 10.0, 1002),
Seq(4, "a4", 10.0, 1003),
Seq(5, "a5", 10.0, 1004),
Seq(6, "a6", 10.0, 1005),
Seq(7, "a7", 10.0, 1006),
Seq(8, "a8", 10.0, 1007),
Seq(9, "a9", 10.0, 1008),
Seq(10, "a10", 10.0, 1009)
)
}
}
}
}
}