Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,20 +23,11 @@
import org.apache.hudi.cli.HoodieTableHeaderFields;
import org.apache.hudi.cli.utils.InputStreamConsumer;
import org.apache.hudi.cli.utils.SparkUtil;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.index.HoodieIndex;

import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.launcher.SparkLauncher;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand;
Expand Down Expand Up @@ -162,13 +153,4 @@ public String deleteSavepoint(@CliOption(key = {"commit"}, help = "Delete a save
}
return String.format("Savepoint \"%s\" deleted.", instantTime);
}

private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.NEVER).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ abstract class BaseProcedure extends Procedure {
}
}

protected def getBasePath(tableName: Option[Any], tablePath: Option[Any]): String = {
protected def getBasePath(tableName: Option[Any], tablePath: Option[Any] = Option.empty): String = {
tableName.map(
t => HoodieCatalogTable(sparkSession, new TableIdentifier(t.asInstanceOf[String])).tableLocation)
.getOrElse(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.command.procedures

import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.exception.{HoodieException, HoodieSavepointException}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType}

import java.util.function.Supplier

class CreateSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging {
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.required(0, "table", DataTypes.StringType, None),
ProcedureParameter.required(1, "commit_Time", DataTypes.StringType, None),
ProcedureParameter.optional(2, "user", DataTypes.StringType, ""),
ProcedureParameter.optional(3, "comments", DataTypes.StringType, "")
)

private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("create_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty))
)

def parameters: Array[ProcedureParameter] = PARAMETERS

def outputType: StructType = OUTPUT_TYPE

override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)

val tableName = getArgValueOrDefault(args, PARAMETERS(0))
val commitTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String]
val user = getArgValueOrDefault(args, PARAMETERS(2)).get.asInstanceOf[String]
val comments = getArgValueOrDefault(args, PARAMETERS(3)).get.asInstanceOf[String]

val basePath: String = getBasePath(tableName)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build

val activeTimeline: HoodieActiveTimeline = metaClient.getActiveTimeline
if (!activeTimeline.getCommitsTimeline.filterCompletedInstants.containsInstant(commitTime)) {
throw new HoodieException("Commit " + commitTime + " not found in Commits " + activeTimeline)
}

val client = createHoodieClient(jsc, basePath)
var result = false

try {
client.savepoint(commitTime, user, comments)
logInfo(s"The commit $commitTime has been savepointed.")
result = true
} catch {
case _: HoodieSavepointException =>
logWarning(s"Failed: Could not create savepoint $commitTime.")
}

Seq(Row(result))
}

override def build: Procedure = new CreateSavepointsProcedure()
}

object CreateSavepointsProcedure {
val NAME: String = "create_savepoints"

def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get(): CreateSavepointsProcedure = new CreateSavepointsProcedure()
}
}



Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.command.procedures

import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.exception.{HoodieException, HoodieSavepointException}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType}

import java.util.function.Supplier

class DeleteSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging {
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.required(0, "table", DataTypes.StringType, None),
ProcedureParameter.required(1, "instant_time", DataTypes.StringType, None)
)

private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("delete_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty))
)

def parameters: Array[ProcedureParameter] = PARAMETERS

def outputType: StructType = OUTPUT_TYPE

override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)

val tableName = getArgValueOrDefault(args, PARAMETERS(0))
val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String]

val basePath: String = getBasePath(tableName)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build

val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants
if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete")
val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime)

if (!completedInstants.containsInstant(savePoint)) {
throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants)
}

val client = createHoodieClient(jsc, basePath)
var result = false

try {
client.deleteSavepoint(instantTime)
logInfo(s"The commit $instantTime has been deleted savepoint.")
result = true
} catch {
case _: HoodieSavepointException =>
logWarning(s"Failed: Could not delete savepoint $instantTime.")
}

Seq(Row(result))
}

override def build: Procedure = new DeleteSavepointsProcedure()
}

object DeleteSavepointsProcedure {
val NAME: String = "delete_savepoints"

def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get(): DeleteSavepointsProcedure = new DeleteSavepointsProcedure()
}
}




Original file line number Diff line number Diff line change
Expand Up @@ -33,11 +33,15 @@ object HoodieProcedures {

private def initProcedureBuilders: util.Map[String, Supplier[ProcedureBuilder]] = {
val mapBuilder: ImmutableMap.Builder[String, Supplier[ProcedureBuilder]] = ImmutableMap.builder()
mapBuilder.put(ShowCommitsProcedure.NAME, ShowCommitsProcedure.builder)
mapBuilder.put(ShowCommitsMetadataProcedure.NAME, ShowCommitsMetadataProcedure.builder)
mapBuilder.put(CreateSavepointsProcedure.NAME, CreateSavepointsProcedure.builder)
mapBuilder.put(DeleteSavepointsProcedure.NAME, DeleteSavepointsProcedure.builder)
mapBuilder.put(RollbackSavepointsProcedure.NAME, RollbackSavepointsProcedure.builder)
mapBuilder.put(RollbackToInstantTimeProcedure.NAME, RollbackToInstantTimeProcedure.builder)
mapBuilder.put(RunClusteringProcedure.NAME, RunClusteringProcedure.builder)
mapBuilder.put(ShowClusteringProcedure.NAME, ShowClusteringProcedure.builder)
mapBuilder.put(ShowCommitsProcedure.NAME, ShowCommitsProcedure.builder)
mapBuilder.put(ShowCommitsMetadataProcedure.NAME, ShowCommitsMetadataProcedure.builder)
mapBuilder.put(ShowSavepointsProcedure.NAME, ShowSavepointsProcedure.builder)
mapBuilder.build
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
/*
* 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.command.procedures

import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.exception.{HoodieException, HoodieSavepointException}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType}

import java.util.function.Supplier

class RollbackSavepointsProcedure extends BaseProcedure with ProcedureBuilder with Logging {
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.required(0, "table", DataTypes.StringType, None),
ProcedureParameter.required(1, "instant_time", DataTypes.StringType, None)
)

private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("rollback_savepoint_result", DataTypes.BooleanType, nullable = true, Metadata.empty))
)

def parameters: Array[ProcedureParameter] = PARAMETERS

def outputType: StructType = OUTPUT_TYPE

override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)

val tableName = getArgValueOrDefault(args, PARAMETERS(0))
val instantTime = getArgValueOrDefault(args, PARAMETERS(1)).get.asInstanceOf[String]

val basePath: String = getBasePath(tableName)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build

val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants
if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete")
val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime)

if (!completedInstants.containsInstant(savePoint)) {
throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants)
}

val client = createHoodieClient(jsc, basePath)
var result = false

try {
client.restoreToSavepoint(instantTime)
logInfo("The commit $instantTime rolled back.")
result = true
} catch {
case _: HoodieSavepointException =>
logWarning(s"The commit $instantTime failed to roll back.")
}

Seq(Row(result))
}

override def build: Procedure = new RollbackSavepointsProcedure()
}

object RollbackSavepointsProcedure {
val NAME: String = "rollback_savepoints"

def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get(): RollbackSavepointsProcedure = new RollbackSavepointsProcedure()
}
}




Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
/*
* 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.command.procedures

import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, HoodieTimeline}
import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType}

import java.util
import java.util.function.Supplier
import java.util.stream.Collectors

class ShowSavepointsProcedure extends BaseProcedure with ProcedureBuilder {
private val PARAMETERS = Array[ProcedureParameter](
ProcedureParameter.required(0, "table", DataTypes.StringType, None)
)

private val OUTPUT_TYPE = new StructType(Array[StructField](
StructField("savepoint_time", DataTypes.StringType, nullable = true, Metadata.empty))
)

def parameters: Array[ProcedureParameter] = PARAMETERS

def outputType: StructType = OUTPUT_TYPE

override def call(args: ProcedureArgs): Seq[Row] = {
super.checkArgs(PARAMETERS, args)

val tableName = getArgValueOrDefault(args, PARAMETERS(0))

val basePath: String = getBasePath(tableName)
val metaClient = HoodieTableMetaClient.builder.setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build

val activeTimeline: HoodieActiveTimeline = metaClient.getActiveTimeline
val timeline: HoodieTimeline = activeTimeline.getSavePointTimeline.filterCompletedInstants
val commits: util.List[HoodieInstant] = timeline.getReverseOrderedInstants.collect(Collectors.toList[HoodieInstant])

if (commits.isEmpty) Seq.empty[Row] else {
commits.toArray.map(instant => instant.asInstanceOf[HoodieInstant].getTimestamp).map(p => Row(p)).toSeq
}
}

override def build: Procedure = new ShowSavepointsProcedure()
}

object ShowSavepointsProcedure {
val NAME: String = "show_savepoints"

def builder: Supplier[ProcedureBuilder] = new Supplier[ProcedureBuilder] {
override def get(): ShowSavepointsProcedure = new ShowSavepointsProcedure()
}
}

Loading