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
1 change: 1 addition & 0 deletions be/src/runtime/routine_load/routine_load_task_executor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ Status RoutineLoadTaskExecutor::submit_task(const TRoutineLoadTask& task) {
// the routine load task'txn has alreay began in FE.
// so it need to rollback if encounter error.
ctx->need_rollback = true;
ctx->max_filter_ratio = 1.0;

// set source related params
switch (task.type) {
Expand Down
6 changes: 4 additions & 2 deletions be/src/runtime/stream_load/stream_load_context.h
Original file line number Diff line number Diff line change
Expand Up @@ -43,9 +43,11 @@ class KafkaLoadInfo {
KafkaLoadInfo(const TKafkaLoadInfo& t_info):
brokers(t_info.brokers),
topic(t_info.topic),
begin_offset(t_info.partition_begin_offset),
cmt_offset(t_info.partition_begin_offset) {
begin_offset(t_info.partition_begin_offset) {

for (auto& p : t_info.partition_begin_offset) {
cmt_offset[p.first] = p.second -1;
}
if (t_info.__isset.max_interval_s) { max_interval_s = t_info.max_interval_s; }
if (t_info.__isset.max_batch_rows) { max_batch_rows = t_info.max_batch_rows; }
if (t_info.__isset.max_batch_size) { max_batch_size = t_info.max_batch_size; }
Expand Down
10 changes: 4 additions & 6 deletions be/src/runtime/stream_load/stream_load_executor.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -219,12 +219,10 @@ bool StreamLoadExecutor::collect_load_stat(StreamLoadContext* ctx, TTxnCommitAtt
rl_attach.__set_loadedBytes(ctx->loaded_bytes);
rl_attach.__set_loadCostMs(ctx->load_cost_nanos / 1000 / 1000);

if (ctx->status.ok()) {
TKafkaRLTaskProgress kafka_progress;
kafka_progress.partitionCmtOffset = std::move(ctx->kafka_info->cmt_offset);
rl_attach.kafkaRLTaskProgress = std::move(kafka_progress);
rl_attach.__isset.kafkaRLTaskProgress = true;
}
TKafkaRLTaskProgress kafka_progress;
kafka_progress.partitionCmtOffset = std::move(ctx->kafka_info->cmt_offset);
rl_attach.kafkaRLTaskProgress = std::move(kafka_progress);
rl_attach.__isset.kafkaRLTaskProgress = true;

attach->rlTaskTxnCommitAttachment = std::move(rl_attach);
attach->__isset.rlTaskTxnCommitAttachment = true;
Expand Down
4 changes: 4 additions & 0 deletions fe/src/main/java/org/apache/doris/catalog/Catalog.java
Original file line number Diff line number Diff line change
Expand Up @@ -4540,6 +4540,10 @@ public RoutineLoadManager getRoutineLoadManager() {
return routineLoadManager;
}

public RoutineLoadTaskScheduler getRoutineLoadTaskScheduler(){
return routineLoadTaskScheduler;
}

public ExportMgr getExportMgr() {
return this.exportMgr;
}
Expand Down
117 changes: 117 additions & 0 deletions fe/src/main/java/org/apache/doris/common/util/LogBuilder.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.common.util;

import com.google.common.collect.Lists;
import org.apache.doris.thrift.TUniqueId;

import java.util.Iterator;
import java.util.List;
import java.util.UUID;

public class LogBuilder {

private final StringBuffer stringBuffer;
private final List<LogEntry> entries;

public LogBuilder(String identifier) {
stringBuffer = new StringBuffer(identifier).append("-");
entries = Lists.newLinkedList();
}

public LogBuilder(LogKey key, Long identifier) {
stringBuffer = new StringBuffer().append(key.name()).append("=").append(identifier).append(", ");
entries = Lists.newLinkedList();
}

public LogBuilder(LogKey key, UUID identifier) {
TUniqueId tUniqueId = new TUniqueId(identifier.getMostSignificantBits(), identifier.getLeastSignificantBits());
stringBuffer = new StringBuffer().append(key.name()).append("=").append(tUniqueId.toString()).append(", ");
entries = Lists.newLinkedList();
}

public LogBuilder(LogKey key, String identifier) {
stringBuffer = new StringBuffer().append(key.name()).append("=").append(identifier).append(", ");
entries = Lists.newLinkedList();
}


public LogBuilder add(String key, long value) {
entries.add(new LogEntry(key, String.valueOf(value)));
return this;
}

public LogBuilder add(String key, int value) {
entries.add(new LogEntry(key, String.valueOf(value)));
return this;
}

public LogBuilder add(String key, float value) {
entries.add(new LogEntry(key, String.valueOf(value)));
return this;
}

public LogBuilder add(String key, boolean value) {
entries.add(new LogEntry(key, String.valueOf(value)));
return this;
}

public LogBuilder add(String key, String value) {
entries.add(new LogEntry(key, String.valueOf(value)));
return this;
}

public LogBuilder add(String key, Object value) {
if (value == null) {
entries.add(new LogEntry(key, "null"));
} else {
entries.add(new LogEntry(key, value.toString()));
}
return this;
}

public String build() {
Iterator<LogEntry> it = entries.iterator();
while (it.hasNext()) {
LogEntry logEntry = it.next();
stringBuffer.append(logEntry.key).append("={").append(logEntry.value).append("}");
if (it.hasNext()) {
stringBuffer.append(", ");
}
}
return stringBuffer.toString();
}

private class LogEntry {
String key;
String value;

public LogEntry(String key, String value) {
this.key = key;
this.value = value;
}
}

@Override
public String toString() {
return build();
}
}
26 changes: 26 additions & 0 deletions fe/src/main/java/org/apache/doris/common/util/LogKey.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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.common.util;

public enum LogKey{
ROUTINE_LOAD_JOB,
ROUINTE_LOAD_TASK
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import org.apache.doris.common.LoadException;
import org.apache.doris.common.MetaNotFoundException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.LogBuilder;
import org.apache.doris.common.util.LogKey;
import org.apache.doris.load.RoutineLoadDesc;
import org.apache.doris.system.SystemInfoService;
import org.apache.doris.transaction.BeginTransactionException;
Expand Down Expand Up @@ -128,7 +130,6 @@ public void divideRoutineLoadJob(int currentConcurrentTaskNum) {
for (int i = 0; i < currentConcurrentTaskNum; i++) {
KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo(UUID.randomUUID(), id);
routineLoadTaskInfoList.add(kafkaTaskInfo);
needScheduleTaskInfoList.add(kafkaTaskInfo);
result.add(kafkaTaskInfo);
}
if (result.size() != 0) {
Expand All @@ -144,7 +145,7 @@ public void divideRoutineLoadJob(int currentConcurrentTaskNum) {
LOG.debug("Ignore to divide routine load job while job state {}", state);
}
// save task into queue of needScheduleTasks
Catalog.getCurrentCatalog().getRoutineLoadManager().addTasksToNeedScheduleQueue(result);
Catalog.getCurrentCatalog().getRoutineLoadTaskScheduler().addTaskInQueue(result);
} finally {
writeUnlock();
}
Expand Down Expand Up @@ -174,12 +175,12 @@ protected void updateProgress(RLTaskTxnCommitAttachment attachment) {
@Override
protected RoutineLoadTaskInfo unprotectRenewTask(RoutineLoadTaskInfo routineLoadTaskInfo) throws AnalysisException,
LabelAlreadyUsedException, BeginTransactionException {
// add new task
KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo((KafkaTaskInfo) routineLoadTaskInfo);
// remove old task
routineLoadTaskInfoList.remove(routineLoadTaskInfo);
// add new task
KafkaTaskInfo kafkaTaskInfo = new KafkaTaskInfo((KafkaTaskInfo) routineLoadTaskInfo);
routineLoadTaskInfoList.add(kafkaTaskInfo);
needScheduleTaskInfoList.add(kafkaTaskInfo);
return kafkaTaskInfo;
}

Expand Down Expand Up @@ -262,6 +263,10 @@ private void updateNewPartitionProgress() {
for (Integer kafkaPartition : currentKafkaPartitions) {
if (!((KafkaProgress) progress).getPartitionIdToOffset().containsKey(kafkaPartition)) {
((KafkaProgress) progress).getPartitionIdToOffset().put(kafkaPartition, 0L);
LOG.debug(new LogBuilder(LogKey.ROUTINE_LOAD_JOB, id)
.add("kafka_partition_id", kafkaPartition)
.add("begin_offset", 0)
.add("msg", "The new partition has been added in job"));
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public KafkaTaskInfo(UUID id, long jobId) {

public KafkaTaskInfo(KafkaTaskInfo kafkaTaskInfo) throws LabelAlreadyUsedException,
BeginTransactionException, AnalysisException {
super(UUID.randomUUID(), kafkaTaskInfo.getJobId(), kafkaTaskInfo.getPreviousBeId());
super(UUID.randomUUID(), kafkaTaskInfo.getJobId(), kafkaTaskInfo.getBeId());
this.partitions = kafkaTaskInfo.getPartitions();
}

Expand Down Expand Up @@ -106,10 +106,11 @@ public TRoutineLoadTask createRoutineLoadTask() throws LoadException, UserExcept


private TExecPlanFragmentParams updateTExecPlanFragmentParams(RoutineLoadJob routineLoadJob) throws UserException {
TExecPlanFragmentParams tExecPlanFragmentParams = routineLoadJob.gettExecPlanFragmentParams();
TExecPlanFragmentParams tExecPlanFragmentParams = routineLoadJob.gettExecPlanFragmentParams().deepCopy();
TPlanFragment tPlanFragment = tExecPlanFragmentParams.getFragment();
tPlanFragment.getOutput_sink().getOlap_table_sink().setTxn_id(this.txnId);
TUniqueId queryId = new TUniqueId(id.getMostSignificantBits(), id.getLeastSignificantBits());
tPlanFragment.getOutput_sink().getOlap_table_sink().setLoad_id(queryId);
tExecPlanFragmentParams.getParams().setQuery_id(queryId);
tExecPlanFragmentParams.getParams().getPer_node_scan_ranges().values().stream()
.forEach(entity -> entity.get(0).getScan_range().getBroker_scan_range().getRanges().get(0).setLoad_id(queryId));
Expand Down
Loading