-
Notifications
You must be signed in to change notification settings - Fork 3.7k
[Bug-fix] Fix the replay error in stream load manager #5722
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
Changes from all commits
59209d6
7d0b996
57f5101
c7b24b0
4468ea7
0b86044
24c752a
3cd0cec
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,18 +17,18 @@ | |
|
|
||
| package org.apache.doris.load; | ||
|
|
||
| import com.google.common.base.Strings; | ||
| import com.google.common.collect.Lists; | ||
| import org.apache.doris.analysis.ShowStreamLoadStmt.StreamLoadState; | ||
| import org.apache.doris.catalog.Catalog; | ||
| import org.apache.doris.catalog.Database; | ||
| import org.apache.doris.cluster.ClusterNamespace; | ||
| import org.apache.doris.common.ClientPool; | ||
| import org.apache.doris.common.Config; | ||
| import org.apache.doris.common.UserException; | ||
| import org.apache.doris.common.io.Text; | ||
| import org.apache.doris.common.io.Writable; | ||
| import org.apache.doris.common.util.MasterDaemon; | ||
| import org.apache.doris.common.util.TimeUtils; | ||
| import org.apache.doris.persist.gson.GsonUtils; | ||
| import org.apache.doris.plugin.AuditEvent; | ||
| import org.apache.doris.plugin.AuditEvent.EventType; | ||
| import org.apache.doris.plugin.StreamLoadAuditEvent; | ||
|
|
@@ -38,11 +38,14 @@ | |
| import org.apache.doris.thrift.TNetworkAddress; | ||
| import org.apache.doris.thrift.TStreamLoadRecord; | ||
| import org.apache.doris.thrift.TStreamLoadRecordResult; | ||
| import org.apache.logging.log4j.LogManager; | ||
| import org.apache.logging.log4j.Logger; | ||
|
|
||
| import com.google.common.collect.Maps; | ||
| import com.google.common.base.Strings; | ||
| import com.google.common.collect.ImmutableMap; | ||
| import com.google.common.collect.Lists; | ||
| import com.google.common.collect.Maps; | ||
|
|
||
| import org.apache.logging.log4j.LogManager; | ||
| import org.apache.logging.log4j.Logger; | ||
|
|
||
| import java.io.DataInput; | ||
| import java.io.DataOutput; | ||
|
|
@@ -53,12 +56,14 @@ | |
| import java.util.LinkedList; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
| import java.util.PriorityQueue; | ||
| import java.util.Queue; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import com.google.gson.annotations.SerializedName; | ||
|
|
||
| public class StreamLoadRecordMgr extends MasterDaemon { | ||
| private static final Logger LOG = LogManager.getLogger(StreamLoadRecordMgr.class); | ||
|
|
||
|
|
@@ -231,12 +236,14 @@ protected void runAfterCatalogReady() { | |
| TStreamLoadRecord streamLoadItem= entry.getValue(); | ||
| String startTime = TimeUtils.longToTimeString(streamLoadItem.getStartTime(), new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS")); | ||
| String finishTime = TimeUtils.longToTimeString(streamLoadItem.getFinishTime(), new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS")); | ||
| LOG.debug("receive stream load record info from backend: {}. label: {}, db: {}, tbl: {}, user: {}, user_ip: {}," + | ||
| " status: {}, message: {}, error_url: {}, total_rows: {}, loaded_rows: {}, filtered_rows: {}," + | ||
| " unselected_rows: {}, load_bytes: {}, start_time: {}, finish_time: {}.", | ||
| backend.getHost(), streamLoadItem.getLabel(), streamLoadItem.getDb(), streamLoadItem.getTbl(), streamLoadItem.getUser(), streamLoadItem.getUserIp(), | ||
| streamLoadItem.getStatus(), streamLoadItem.getMessage(), streamLoadItem.getUrl(), streamLoadItem.getTotalRows(), streamLoadItem.getLoadedRows(), | ||
| streamLoadItem.getFilteredRows(), streamLoadItem.getUnselectedRows(), streamLoadItem.getLoadBytes(), startTime, finishTime); | ||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("receive stream load record info from backend: {}. label: {}, db: {}, tbl: {}, user: {}, user_ip: {}," + | ||
| " status: {}, message: {}, error_url: {}, total_rows: {}, loaded_rows: {}, filtered_rows: {}," + | ||
| " unselected_rows: {}, load_bytes: {}, start_time: {}, finish_time: {}.", | ||
| backend.getHost(), streamLoadItem.getLabel(), streamLoadItem.getDb(), streamLoadItem.getTbl(), streamLoadItem.getUser(), streamLoadItem.getUserIp(), | ||
| streamLoadItem.getStatus(), streamLoadItem.getMessage(), streamLoadItem.getUrl(), streamLoadItem.getTotalRows(), streamLoadItem.getLoadedRows(), | ||
| streamLoadItem.getFilteredRows(), streamLoadItem.getUnselectedRows(), streamLoadItem.getLoadBytes(), startTime, finishTime); | ||
| } | ||
|
|
||
| AuditEvent auditEvent = new StreamLoadAuditEvent.AuditEventBuilder().setEventType(EventType.STREAM_LOAD_FINISH) | ||
| .setLabel(streamLoadItem.getLabel()).setDb(streamLoadItem.getDb()).setTable(streamLoadItem.getTbl()) | ||
|
|
@@ -313,13 +320,14 @@ public void replayFetchStreamLoadRecord(FetchStreamLoadRecord fetchStreamLoadRec | |
| for (Backend backend : backends.values()) { | ||
| if (beIdToLastStreamLoad.containsKey(backend.getId())) { | ||
| long lastStreamLoadTime = beIdToLastStreamLoad.get(backend.getId()); | ||
| LOG.info("Replay stream load bdbje. backend: {}, last stream load version: {}", backend.getHost(), lastStreamLoadTime); | ||
| LOG.info("Replay stream load bdbje. backend: {}, last stream load time: {}", backend.getHost(), lastStreamLoadTime); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what if disable stream load the value for lastStreamLoadTime will be?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In fact, even if the collection process of be is disabled. In fact, FE can also add another variable to stop obtaining stream load record. |
||
| backend.setLastStreamLoadTime(lastStreamLoadTime); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public static class FetchStreamLoadRecord implements Writable { | ||
| @SerializedName("beIdToLastStreamLoad") | ||
| private Map<Long, Long> beIdToLastStreamLoad; | ||
|
|
||
| public FetchStreamLoadRecord(Map<Long, Long> beIdToLastStreamLoad) { | ||
|
|
@@ -336,34 +344,13 @@ public Map<Long, Long> getBeIdToLastStreamLoad() { | |
|
|
||
| @Override | ||
| public void write(DataOutput out) throws IOException { | ||
| for (Map.Entry<Long, Long> entry : beIdToLastStreamLoad.entrySet()) { | ||
| out.writeBoolean(true); | ||
| out.writeLong(entry.getKey()); | ||
| out.writeBoolean(true); | ||
| out.writeLong(entry.getValue()); | ||
| LOG.debug("Write stream load bdbje. key: {}, value: {} ", entry.getKey(), entry.getValue()); | ||
| } | ||
| String json = GsonUtils.GSON.toJson(this); | ||
| Text.writeString(out, json); | ||
| } | ||
|
|
||
| public static FetchStreamLoadRecord read(DataInput in) throws IOException { | ||
| Map<Long, Long> idToLastStreamLoad = Maps.newHashMap(); | ||
| int beNum = Catalog.getCurrentSystemInfo().getIdToBackend().size(); | ||
| for (int i = 0; i < beNum; i++) { | ||
| long beId = -1; | ||
| long lastStreamLoad = -1; | ||
| if (in.readBoolean()) { | ||
| beId = in.readLong(); | ||
| } | ||
| if (in.readBoolean()) { | ||
| lastStreamLoad = in.readLong(); | ||
| } | ||
| if (beId != -1 && lastStreamLoad != -1) { | ||
| idToLastStreamLoad.put(beId, lastStreamLoad); | ||
| } | ||
| LOG.debug("Read stream load bdbje. key: {}, value: {} ", beId, lastStreamLoad); | ||
| } | ||
| FetchStreamLoadRecord fetchStreamLoadRecord = new FetchStreamLoadRecord(idToLastStreamLoad); | ||
| return fetchStreamLoadRecord; | ||
| String json = Text.readString(in); | ||
| return GsonUtils.GSON.fromJson(json, FetchStreamLoadRecord.class); | ||
| } | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.