Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[feature] support two phase commit #122

Merged
merged 5 commits into from
Jul 28, 2023
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 @@ -97,4 +97,7 @@ public interface ConfigurationOptions {
*/
String DORIS_IGNORE_TYPE = "doris.ignore-type";

String DORIS_SINK_ENABLE_2PC = "doris.sink.enable-2pc";
boolean DORIS_SINK_ENABLE_2PC_DEFAULT = false;

}
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,17 @@ public Integer getIntegerProperty(String name, Integer defaultValue) {
return defaultValue;
}

public Boolean getBooleanProperty(String name) {
return getBooleanProperty(name, null);
}

public Boolean getBooleanProperty(String name, Boolean defaultValue) {
if (getProperty(name) != null) {
return Boolean.valueOf(getProperty(name));
}
return defaultValue;
}

public Settings merge(Properties properties) {
if (properties == null || properties.isEmpty()) {
return this;
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,10 @@ public class RespContent {
@JsonProperty(value = "ErrorURL")
private String ErrorURL;

public int getTxnId() {
return TxnId;
}

public String getStatus() {
return Status;
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
// 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.doris.spark.util;
gnehil marked this conversation as resolved.
Show resolved Hide resolved

import java.util.regex.Pattern;

public class ResponseUtil {
public static final Pattern LABEL_EXIST_PATTERN =
Pattern.compile("errCode = 2, detailMessage = Label \\[(.*)\\] " +
"has already been used, relate to txn \\[(\\d+)\\]");
public static final Pattern COMMITTED_PATTERN =
Pattern.compile("errCode = 2, detailMessage = transaction \\[(\\d+)\\] " +
"is already \\b(COMMITTED|committed|VISIBLE|visible)\\b, not pre-committed.");

public static boolean isCommitted(String msg) {
return COMMITTED_PATTERN.matcher(msg).matches();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
// 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.doris.spark.listener
gnehil marked this conversation as resolved.
Show resolved Hide resolved

import org.apache.doris.spark.load.DorisStreamLoad
import org.apache.doris.spark.sql.Utils
import org.apache.spark.scheduler._
import org.apache.spark.util.CollectionAccumulator
import org.slf4j.{Logger, LoggerFactory}

import java.time.Duration
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.{Failure, Success}

class DorisTransactionListener(preCommittedTxnAcc: CollectionAccumulator[Int], dorisStreamLoad: DorisStreamLoad)
extends SparkListener {

val logger: Logger = LoggerFactory.getLogger(classOf[DorisTransactionListener])

override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
val txnIds: mutable.Buffer[Int] = preCommittedTxnAcc.value.asScala
val failedTxnIds = mutable.Buffer[Int]()
jobEnd.jobResult match {
// if job succeed, commit all transactions
case JobSucceeded =>
if (txnIds.isEmpty) {
logger.warn("job run succeed, but there is no pre-committed txn ids")
return
}
logger.info("job run succeed, start committing transactions")
txnIds.foreach(txnId =>
Utils.retry(3, Duration.ofSeconds(1), logger) {
dorisStreamLoad.commit(txnId)
} match {
case Success(_) =>
case Failure(_) => failedTxnIds += txnId
}
)

if (failedTxnIds.nonEmpty) {
logger.error("uncommitted txn ids: {}", failedTxnIds.mkString(","))
} else {
logger.info("commit transaction success")
}
// if job failed, abort all pre committed transactions
case _ =>
if (txnIds.isEmpty) {
logger.warn("job run failed, but there is no pre-committed txn ids")
return
}
logger.info("job run failed, start aborting transactions")
txnIds.foreach(txnId =>
Utils.retry(3, Duration.ofSeconds(1), logger) {
dorisStreamLoad.abort(txnId)
} match {
case Success(_) =>
case Failure(_) => failedTxnIds += txnId
})
if (failedTxnIds.nonEmpty) {
logger.error("not aborted txn ids: {}", failedTxnIds.mkString(","))
} else {
logger.info("abort transaction success")
}
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ private[spark] object Utils {
Success(result)
case Failure(exception: T) if retryTimes > 0 =>
logger.warn(s"Execution failed caused by: ", exception)
logger.warn(s"$retryTimes times retry remaining, the next will be in ${interval.toMillis}ms")
logger.warn(s"$retryTimes times retry remaining, the next attempt will be in ${interval.toMillis} ms")
LockSupport.parkNanos(interval.toNanos)
retry(retryTimes - 1, interval, logger)(f)
case Failure(exception) => Failure(exception)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.doris.spark.writer

import org.apache.doris.spark.cfg.{ConfigurationOptions, SparkSettings}
import org.apache.doris.spark.listener.DorisTransactionListener
import org.apache.doris.spark.load.{CachedDorisStreamLoadClient, DorisStreamLoad}
import org.apache.doris.spark.sql.Utils
import org.apache.spark.sql.DataFrame
Expand All @@ -28,6 +29,7 @@ import java.time.Duration
import java.util
import java.util.Objects
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.{Failure, Success}

class DorisWriter(settings: SparkSettings) extends Serializable {
Expand All @@ -44,9 +46,19 @@ class DorisWriter(settings: SparkSettings) extends Serializable {
private val batchInterValMs: Integer = settings.getIntegerProperty(ConfigurationOptions.DORIS_SINK_BATCH_INTERVAL_MS,
ConfigurationOptions.DORIS_SINK_BATCH_INTERVAL_MS_DEFAULT)

private val enable2PC: Boolean = settings.getBooleanProperty(ConfigurationOptions.DORIS_SINK_ENABLE_2PC,
ConfigurationOptions.DORIS_SINK_ENABLE_2PC_DEFAULT);

private val dorisStreamLoader: DorisStreamLoad = CachedDorisStreamLoadClient.getOrCreate(settings)

def write(dataFrame: DataFrame): Unit = {

val sc = dataFrame.sqlContext.sparkContext
val preCommittedTxnAcc = sc.collectionAccumulator[Int]("preCommittedTxnAcc")
if (enable2PC) {
sc.addSparkListener(new DorisTransactionListener(preCommittedTxnAcc, dorisStreamLoader))
}

var resultRdd = dataFrame.rdd
val dfColumns = dataFrame.columns
if (Objects.nonNull(sinkTaskPartitionSize)) {
Expand All @@ -65,11 +77,27 @@ class DorisWriter(settings: SparkSettings) extends Serializable {
*
*/
def flush(batch: Iterable[util.List[Object]], dfColumns: Array[String]): Unit = {
Utils.retry[Unit, Exception](maxRetryTimes, Duration.ofMillis(batchInterValMs.toLong), logger) {
dorisStreamLoader.loadV2(batch.toList.asJava, dfColumns)
Utils.retry[util.List[Integer], Exception](maxRetryTimes, Duration.ofMillis(batchInterValMs.toLong), logger) {
dorisStreamLoader.loadV2(batch.toList.asJava, dfColumns, enable2PC)
} match {
case Success(_) =>
case Success(txnIds) => if (enable2PC) txnIds.asScala.foreach(txnId => preCommittedTxnAcc.add(txnId))
case Failure(e) =>
if (enable2PC) {
// if task run failed, acc value will not be returned to driver,
// should abort all pre committed transactions inside the task
logger.info("load task failed, start aborting previously pre-committed transactions")
val abortFailedTxnIds = mutable.Buffer[Int]()
preCommittedTxnAcc.value.asScala.foreach(txnId => {
Utils.retry[Unit, Exception](3, Duration.ofSeconds(1), logger) {
dorisStreamLoader.abort(txnId)
} match {
case Success(_) =>
case Failure(_) => abortFailedTxnIds += txnId
}
})
if (abortFailedTxnIds.nonEmpty) logger.warn("not aborted txn ids: {}", abortFailedTxnIds.mkString(","))
preCommittedTxnAcc.reset()
}
throw new IOException(
s"Failed to load batch data on BE: ${dorisStreamLoader.getLoadUrlStr} node and exceeded the max ${maxRetryTimes} retry times.", e)
}
Expand Down