Skip to content

Commit

Permalink
[improve](routine load) improve routine load observability (#46238)
Browse files Browse the repository at this point in the history
1. **reset other msg in the a stream window**
The routine load job is a continuously scheduled job, and as the job
runs, previous errors do not need to be constantly displayed.

2. **show error info when  transaction of sub task failed**
If a subtask fails, it will continuously retry, and there may be some
errors that prevent the job from scheduling and consuming data properly,
such as continuous too many segments error(code: -235). At this time, it
is necessary to display it in a timely manner to make the user aware.

3. **set pause reason to other msg when reschedule job**
For jobs that are unexpectedly paused, the job manager has an auto
resume mechanism. However, for some scenarios, such as not being able to
connect to Kafka and being auto resumed after pause to retry, it may
cause users to not see the problem for a long time. Unexpectedly paused
jobs always have issues, even if auto resume occurs, the reason for the
error needs to be displayed.
  • Loading branch information
sollhui authored Jan 7, 2025
1 parent ed07dc8 commit d7b28f5
Show file tree
Hide file tree
Showing 3 changed files with 303 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,12 @@ public long getDbId() {
}

public void setOtherMsg(String otherMsg) {
this.otherMsg = TimeUtils.getCurrentFormatTime() + ":" + Strings.nullToEmpty(otherMsg);
writeLock();
try {
this.otherMsg = TimeUtils.getCurrentFormatTime() + ":" + Strings.nullToEmpty(otherMsg);
} finally {
writeUnlock();
}
}

public String getDbFullName() throws MetaNotFoundException {
Expand Down Expand Up @@ -920,9 +925,10 @@ private void updateNumOfData(long numOfTotalRows, long numOfErrorRows, long unse
+ "when current total rows is more than base or the filter ratio is more than the max")
.build());
}
// reset currentTotalNum and currentErrorNum
// reset currentTotalNum, currentErrorNum and otherMsg
this.jobStatistic.currentErrorRows = 0;
this.jobStatistic.currentTotalRows = 0;
this.otherMsg = "";
} else if (this.jobStatistic.currentErrorRows > maxErrorNum
|| (this.jobStatistic.currentTotalRows > 0
&& ((double) this.jobStatistic.currentErrorRows
Expand All @@ -941,9 +947,10 @@ private void updateNumOfData(long numOfTotalRows, long numOfErrorRows, long unse
"current error rows is more than max_error_number "
+ "or the max_filter_ratio is more than the value set"), isReplay);
}
// reset currentTotalNum and currentErrorNum
// reset currentTotalNum, currentErrorNum and otherMsg
this.jobStatistic.currentErrorRows = 0;
this.jobStatistic.currentTotalRows = 0;
this.otherMsg = "";
}
}

Expand Down Expand Up @@ -1202,6 +1209,7 @@ public void afterAborted(TransactionState txnState, boolean txnOperated, String
long taskBeId = -1L;
try {
this.jobStatistic.runningTxnIds.remove(txnState.getTransactionId());
setOtherMsg(txnStatusChangeReasonString);
if (txnOperated) {
// step0: find task in job
Optional<RoutineLoadTaskInfo> routineLoadTaskInfoOptional = routineLoadTaskInfoList.stream().filter(
Expand Down Expand Up @@ -1521,6 +1529,7 @@ public void update() throws UserException {
LOG.info(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id)
.add("msg", "Job need to be rescheduled")
.build());
this.otherMsg = pauseReason == null ? "" : pauseReason.getMsg();
unprotectUpdateProgress();
unprotectUpdateState(JobState.NEED_SCHEDULE, null, false);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
57|2023-08-19|TRUE|2|-25462|-74112029|6458082754318544493|-7910671781690629051|-15205.859375|-306870797.484914|759730669.0|-628556336.0|2023-07-10 18:39:10|2023-02-12|2023-01-27 07:26:06|y||Xi9nDVrLv8m6AwEpUxmtzFAuK48sQ|{"name": "John", "age": 25, "city": "New York"}
Original file line number Diff line number Diff line change
@@ -0,0 +1,290 @@
// 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.

import org.apache.kafka.clients.admin.AdminClient
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.clients.producer.ProducerConfig

suite("test_routine_load_error_info","nonConcurrent") {
def kafkaCsvTpoics = [
"test_error_info",
]

String enabled = context.config.otherConfigs.get("enableKafkaTest")
String kafka_port = context.config.otherConfigs.get("kafka_port")
String externalEnvIp = context.config.otherConfigs.get("externalEnvIp")
def kafka_broker = "${externalEnvIp}:${kafka_port}"

// send data to kafka
if (enabled != null && enabled.equalsIgnoreCase("true")) {
def props = new Properties()
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "${kafka_broker}".toString())
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
def producer = new KafkaProducer<>(props)
for (String kafkaCsvTopic in kafkaCsvTpoics) {
def txt = new File("""${context.file.parent}/data/${kafkaCsvTopic}.csv""").text
def lines = txt.readLines()
lines.each { line ->
logger.info("=====${line}========")
def record = new ProducerRecord<>(kafkaCsvTopic, null, line)
producer.send(record)
}
}
}

// case 1: task failed
if (enabled != null && enabled.equalsIgnoreCase("true")) {
// create table
def jobName = "test_error_info"
def tableName = "test_routine_error_info"
try {
sql """
CREATE TABLE IF NOT EXISTS ${tableName}
(
k00 INT NOT NULL,
k01 DATE NOT NULL,
k02 BOOLEAN NULL,
k03 TINYINT NULL,
k04 SMALLINT NULL,
k05 INT NULL,
k06 BIGINT NULL,
k07 LARGEINT NULL,
k08 FLOAT NULL,
k09 DOUBLE NULL,
k10 DECIMAL(9,1) NULL,
k11 DECIMALV3(9,1) NULL,
k12 DATETIME NULL,
k13 DATEV2 NULL,
k14 DATETIMEV2 NULL,
k15 CHAR NULL,
k16 VARCHAR NULL,
k17 STRING NULL,
k18 JSON NULL,
kd01 BOOLEAN NOT NULL DEFAULT "TRUE",
kd02 TINYINT NOT NULL DEFAULT "1",
kd03 SMALLINT NOT NULL DEFAULT "2",
kd04 INT NOT NULL DEFAULT "3",
kd05 BIGINT NOT NULL DEFAULT "4",
kd06 LARGEINT NOT NULL DEFAULT "5",
kd07 FLOAT NOT NULL DEFAULT "6.0",
kd08 DOUBLE NOT NULL DEFAULT "7.0",
kd09 DECIMAL NOT NULL DEFAULT "888888888",
kd10 DECIMALV3 NOT NULL DEFAULT "999999999",
kd11 DATE NOT NULL DEFAULT "2023-08-24",
kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00",
kd13 DATEV2 NOT NULL DEFAULT "2023-08-24",
kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00",
kd15 CHAR(255) NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd18 JSON NULL,
INDEX idx_inverted_k104 (`k05`) USING INVERTED,
INDEX idx_inverted_k110 (`k11`) USING INVERTED,
INDEX idx_inverted_k113 (`k13`) USING INVERTED,
INDEX idx_inverted_k114 (`k14`) USING INVERTED,
INDEX idx_inverted_k117 (`k17`) USING INVERTED PROPERTIES("parser" = "english"),
INDEX idx_ngrambf_k115 (`k15`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_ngrambf_k116 (`k16`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_bitmap_k104 (`k02`) USING BITMAP,
INDEX idx_bitmap_k110 (`kd01`) USING BITMAP
)
DUPLICATE KEY(k00)
PARTITION BY RANGE(k01)
(
PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')),
PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')),
PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01'))
)
DISTRIBUTED BY HASH(k00) BUCKETS 32
PROPERTIES (
"bloom_filter_columns"="k05",
"replication_num" = "1"
);
"""
sql "sync"

// create job
GetDebugPoint().enableDebugPointForAllBEs("BetaRowsetWriter._check_segment_number_limit_too_many_segments")
sql """
CREATE ROUTINE LOAD ${jobName} on ${tableName}
COLUMNS(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18),
COLUMNS TERMINATED BY "|"
PROPERTIES
(
"max_batch_interval" = "5",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200"
)
FROM KAFKA
(
"kafka_broker_list" = "${externalEnvIp}:${kafka_port}",
"kafka_topic" = "${kafkaCsvTpoics[0]}",
"property.kafka_default_offsets" = "OFFSET_BEGINNING"
);
"""
sql "sync"

// check error info
def count = 0
while (true) {
def res = sql "show routine load for ${jobName}"
log.info("show routine load: ${res[0].toString()}".toString())
log.info("other msg: ${res[0][19].toString()}".toString())
if (res[0][19].toString() != "") {
assertTrue(res[0][19].toString().contains("too many segments in rowset"))
break;
}
count++
if (count > 60) {
assertEquals(1, 2)
break;
} else {
sleep(1000)
continue;
}
}
} finally {
GetDebugPoint().disableDebugPointForAllBEs("BetaRowsetWriter._check_segment_number_limit_too_many_segments")
sql "stop routine load for ${jobName}"
sql "DROP TABLE IF EXISTS ${tableName}"
}
}

// case 2: reschedule job
if (enabled != null && enabled.equalsIgnoreCase("true")) {
def jobName = "test_error_info"
def tableName = "test_routine_error_info"
try {
sql """
CREATE TABLE IF NOT EXISTS ${tableName}
(
k00 INT NOT NULL,
k01 DATE NOT NULL,
k02 BOOLEAN NULL,
k03 TINYINT NULL,
k04 SMALLINT NULL,
k05 INT NULL,
k06 BIGINT NULL,
k07 LARGEINT NULL,
k08 FLOAT NULL,
k09 DOUBLE NULL,
k10 DECIMAL(9,1) NULL,
k11 DECIMALV3(9,1) NULL,
k12 DATETIME NULL,
k13 DATEV2 NULL,
k14 DATETIMEV2 NULL,
k15 CHAR NULL,
k16 VARCHAR NULL,
k17 STRING NULL,
k18 JSON NULL,
kd01 BOOLEAN NOT NULL DEFAULT "TRUE",
kd02 TINYINT NOT NULL DEFAULT "1",
kd03 SMALLINT NOT NULL DEFAULT "2",
kd04 INT NOT NULL DEFAULT "3",
kd05 BIGINT NOT NULL DEFAULT "4",
kd06 LARGEINT NOT NULL DEFAULT "5",
kd07 FLOAT NOT NULL DEFAULT "6.0",
kd08 DOUBLE NOT NULL DEFAULT "7.0",
kd09 DECIMAL NOT NULL DEFAULT "888888888",
kd10 DECIMALV3 NOT NULL DEFAULT "999999999",
kd11 DATE NOT NULL DEFAULT "2023-08-24",
kd12 DATETIME NOT NULL DEFAULT "2023-08-24 12:00:00",
kd13 DATEV2 NOT NULL DEFAULT "2023-08-24",
kd14 DATETIMEV2 NOT NULL DEFAULT "2023-08-24 12:00:00",
kd15 CHAR(255) NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd16 VARCHAR(300) NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd17 STRING NOT NULL DEFAULT "我能吞下玻璃而不伤身体",
kd18 JSON NULL,
INDEX idx_inverted_k104 (`k05`) USING INVERTED,
INDEX idx_inverted_k110 (`k11`) USING INVERTED,
INDEX idx_inverted_k113 (`k13`) USING INVERTED,
INDEX idx_inverted_k114 (`k14`) USING INVERTED,
INDEX idx_inverted_k117 (`k17`) USING INVERTED PROPERTIES("parser" = "english"),
INDEX idx_ngrambf_k115 (`k15`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_ngrambf_k116 (`k16`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_ngrambf_k117 (`k17`) USING NGRAM_BF PROPERTIES("gram_size"="3", "bf_size"="256"),
INDEX idx_bitmap_k104 (`k02`) USING BITMAP,
INDEX idx_bitmap_k110 (`kd01`) USING BITMAP
)
DUPLICATE KEY(k00)
PARTITION BY RANGE(k01)
(
PARTITION p1 VALUES [('2023-08-01'), ('2023-08-11')),
PARTITION p2 VALUES [('2023-08-11'), ('2023-08-21')),
PARTITION p3 VALUES [('2023-08-21'), ('2023-09-01'))
)
DISTRIBUTED BY HASH(k00) BUCKETS 32
PROPERTIES (
"bloom_filter_columns"="k05",
"replication_num" = "1"
);
"""
sql "sync"

// create job
sql """
CREATE ROUTINE LOAD ${jobName} on ${tableName}
COLUMNS(k00,k01,k02,k03,k04,k05,k06,k07,k08,k09,k10,k11,k12,k13,k14,k15,k16,k17,k18),
COLUMNS TERMINATED BY "|"
PROPERTIES
(
"max_batch_interval" = "5",
"max_batch_rows" = "300000",
"max_batch_size" = "209715200"
)
FROM KAFKA
(
"kafka_broker_list" = "${externalEnvIp}:${kafka_port}",
"kafka_topic" = "invalid_job",
"property.kafka_default_offsets" = "OFFSET_BEGINNING"
);
"""

// check error info
def count = 0
while (true) {
def res = sql "show routine load for ${jobName}"
log.info("show routine load: ${res[0].toString()}".toString())
log.info("other msg: ${res[0][19].toString()}".toString())
if (res[0][19].toString() != "" && res[0][8].toString() == "NEED_SCHEDULE") {
assertTrue(res[0][19].toString().contains("may be Kafka properties set in job is error or no partition in this topic that should check Kafka"))
break;
}
count++
if (count > 60) {
assertEquals(1, 2)
break;
} else {
sleep(1000)
continue;
}
}
} finally {
sql "stop routine load for ${jobName}"
sql "DROP TABLE IF EXISTS ${tableName}"
}
}
}

0 comments on commit d7b28f5

Please sign in to comment.