[feature-wip](executor)Fe send topic info to be (#25798)
This commit is contained in:
@ -113,6 +113,9 @@ import org.apache.doris.common.ThreadPoolManager;
|
||||
import org.apache.doris.common.UserException;
|
||||
import org.apache.doris.common.io.CountingDataOutputStream;
|
||||
import org.apache.doris.common.io.Text;
|
||||
import org.apache.doris.common.publish.TopicPublisher;
|
||||
import org.apache.doris.common.publish.TopicPublisherThread;
|
||||
import org.apache.doris.common.publish.WorkloadGroupPublisher;
|
||||
import org.apache.doris.common.util.Daemon;
|
||||
import org.apache.doris.common.util.DynamicPartitionUtil;
|
||||
import org.apache.doris.common.util.HttpURLUtil;
|
||||
@ -492,6 +495,8 @@ public class Env {
|
||||
|
||||
private HiveTransactionMgr hiveTransactionMgr;
|
||||
|
||||
private TopicPublisherThread topicPublisherThread;
|
||||
|
||||
public List<TFrontendInfo> getFrontendInfos() {
|
||||
List<TFrontendInfo> res = new ArrayList<>();
|
||||
|
||||
@ -726,6 +731,8 @@ public class Env {
|
||||
this.binlogGcer = new BinlogGcer();
|
||||
this.columnIdFlusher = new ColumnIdFlushDaemon();
|
||||
this.queryCancelWorker = new QueryCancelWorker(systemInfo);
|
||||
this.topicPublisherThread = new TopicPublisherThread(
|
||||
"TopicPublisher", Config.publish_topic_info_interval_ms, systemInfo);
|
||||
}
|
||||
|
||||
public static void destroyCheckpoint() {
|
||||
@ -970,6 +977,10 @@ public class Env {
|
||||
}
|
||||
|
||||
queryCancelWorker.start();
|
||||
|
||||
TopicPublisher wgPublisher = new WorkloadGroupPublisher(this);
|
||||
topicPublisherThread.addToTopicPublisherList(wgPublisher);
|
||||
topicPublisherThread.start();
|
||||
}
|
||||
|
||||
// wait until FE is ready.
|
||||
|
||||
@ -30,15 +30,24 @@ public class AckResponseHandler extends ResponseHandler {
|
||||
this.listener = listener;
|
||||
}
|
||||
|
||||
public AckResponseHandler(Collection<Backend> nodes) {
|
||||
super(nodes);
|
||||
this.listener = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponse(Backend node) {
|
||||
super.onResponse(node);
|
||||
listener.onResponse(node);
|
||||
if (listener != null) {
|
||||
listener.onResponse(node);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Backend node, Throwable t) {
|
||||
super.onFailure(node, t);
|
||||
listener.onFailure(node, t);
|
||||
if (listener != null) {
|
||||
listener.onFailure(node, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -0,0 +1,24 @@
|
||||
// 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.publish;
|
||||
|
||||
import org.apache.doris.thrift.TPublishTopicRequest;
|
||||
|
||||
public interface TopicPublisher {
|
||||
public void getTopicInfo(TPublishTopicRequest req);
|
||||
}
|
||||
@ -0,0 +1,121 @@
|
||||
// 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.publish;
|
||||
|
||||
import org.apache.doris.common.ClientPool;
|
||||
import org.apache.doris.common.Config;
|
||||
import org.apache.doris.common.ThreadPoolManager;
|
||||
import org.apache.doris.common.util.MasterDaemon;
|
||||
import org.apache.doris.system.Backend;
|
||||
import org.apache.doris.system.SystemInfoService;
|
||||
import org.apache.doris.thrift.BackendService;
|
||||
import org.apache.doris.thrift.TNetworkAddress;
|
||||
import org.apache.doris.thrift.TPublishTopicRequest;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
|
||||
public class TopicPublisherThread extends MasterDaemon {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TopicPublisherThread.class);
|
||||
|
||||
private SystemInfoService clusterInfoService;
|
||||
|
||||
private ExecutorService executor = ThreadPoolManager
|
||||
.newDaemonFixedThreadPool(6, 256, "topic-publish-thread", true);
|
||||
|
||||
public TopicPublisherThread(String name, long intervalMs,
|
||||
SystemInfoService clusterInfoService) {
|
||||
super(name, intervalMs);
|
||||
this.clusterInfoService = clusterInfoService;
|
||||
}
|
||||
|
||||
private List<TopicPublisher> topicPublisherList = new ArrayList<>();
|
||||
|
||||
public void addToTopicPublisherList(TopicPublisher topicPublisher) {
|
||||
this.topicPublisherList.add(topicPublisher);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void runAfterCatalogReady() {
|
||||
if (!Config.enable_workload_group) {
|
||||
return;
|
||||
}
|
||||
LOG.info("begin publish topic info");
|
||||
// step 1: get all publish topic info
|
||||
TPublishTopicRequest request = new TPublishTopicRequest();
|
||||
for (TopicPublisher topicPublisher : topicPublisherList) {
|
||||
topicPublisher.getTopicInfo(request);
|
||||
}
|
||||
|
||||
// step 2: publish topic info to all be
|
||||
Collection<Backend> nodesToPublish = clusterInfoService.getIdToBackend().values();
|
||||
AckResponseHandler handler = new AckResponseHandler(nodesToPublish);
|
||||
for (Backend be : nodesToPublish) {
|
||||
executor.submit(new TopicPublishWorker(request, be, handler));
|
||||
}
|
||||
try {
|
||||
int timeoutMs = Config.publish_topic_info_interval_ms / 3 * 2;
|
||||
if (!handler.awaitAllInMs(timeoutMs)) {
|
||||
Backend[] backends = handler.pendingNodes();
|
||||
if (backends.length > 0) {
|
||||
LOG.warn("timed out waiting for all nodes to publish. (pending nodes: {})",
|
||||
Arrays.toString(backends));
|
||||
}
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
public class TopicPublishWorker implements Runnable {
|
||||
private TPublishTopicRequest request;
|
||||
private Backend be;
|
||||
private ResponseHandler handler;
|
||||
|
||||
public TopicPublishWorker(TPublishTopicRequest request, Backend node, ResponseHandler handler) {
|
||||
this.request = request;
|
||||
this.be = node;
|
||||
this.handler = handler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
long beginTime = System.currentTimeMillis();
|
||||
try {
|
||||
TNetworkAddress addr = new TNetworkAddress(be.getHost(), be.getBePort());
|
||||
BackendService.Client client = ClientPool.backendPool.borrowObject(addr);
|
||||
client.publishTopicInfo(request);
|
||||
LOG.info("publish topic info to be {} success, time cost={} ms",
|
||||
be.getHost(), (System.currentTimeMillis() - beginTime));
|
||||
} catch (Exception e) {
|
||||
LOG.warn("publish topic info to be {} error happens: , time cost={} ms",
|
||||
be.getHost(), (System.currentTimeMillis() - beginTime), e);
|
||||
} finally {
|
||||
handler.onResponse(be);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@ -0,0 +1,39 @@
|
||||
// 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.publish;
|
||||
|
||||
import org.apache.doris.catalog.Env;
|
||||
import org.apache.doris.thrift.TPublishTopicRequest;
|
||||
import org.apache.doris.thrift.TopicInfo;
|
||||
|
||||
public class WorkloadGroupPublisher implements TopicPublisher {
|
||||
|
||||
private Env env;
|
||||
|
||||
public WorkloadGroupPublisher(Env env) {
|
||||
this.env = env;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void getTopicInfo(TPublishTopicRequest req) {
|
||||
for (TopicInfo topicInfo : env.getWorkloadGroupMgr()
|
||||
.getPublishTopicInfo()) {
|
||||
req.addToTopicList(topicInfo);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -25,6 +25,8 @@ import org.apache.doris.common.proc.BaseProcResult;
|
||||
import org.apache.doris.persist.gson.GsonPostProcessable;
|
||||
import org.apache.doris.persist.gson.GsonUtils;
|
||||
import org.apache.doris.thrift.TPipelineWorkloadGroup;
|
||||
import org.apache.doris.thrift.TTopicInfoType;
|
||||
import org.apache.doris.thrift.TopicInfo;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
@ -290,6 +292,16 @@ public class WorkloadGroup implements Writable, GsonPostProcessable {
|
||||
return new TPipelineWorkloadGroup().setId(id).setName(name).setProperties(clonedHashMap).setVersion(version);
|
||||
}
|
||||
|
||||
public TopicInfo toTopicInfo() {
|
||||
HashMap<String, String> newHashMap = new HashMap<>();
|
||||
newHashMap.put("id", String.valueOf(id));
|
||||
TopicInfo topicInfo = new TopicInfo();
|
||||
topicInfo.setTopicType(TTopicInfoType.WORKLOAD_GROUP);
|
||||
topicInfo.setInfoMap(newHashMap);
|
||||
topicInfo.setTopicKey(name);
|
||||
return topicInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(DataOutput out) throws IOException {
|
||||
String json = GsonUtils.GSON.toJson(this);
|
||||
|
||||
@ -40,6 +40,7 @@ import org.apache.doris.persist.gson.GsonUtils;
|
||||
import org.apache.doris.qe.ConnectContext;
|
||||
import org.apache.doris.thrift.TPipelineWorkloadGroup;
|
||||
import org.apache.doris.thrift.TUserIdentity;
|
||||
import org.apache.doris.thrift.TopicInfo;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
@ -52,6 +53,7 @@ import org.apache.logging.log4j.Logger;
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
@ -133,6 +135,19 @@ public class WorkloadGroupMgr implements Writable, GsonPostProcessable {
|
||||
return workloadGroups;
|
||||
}
|
||||
|
||||
public List<TopicInfo> getPublishTopicInfo() {
|
||||
List<TopicInfo> workloadGroups = new ArrayList();
|
||||
readLock();
|
||||
try {
|
||||
for (WorkloadGroup wg : idToWorkloadGroup.values()) {
|
||||
workloadGroups.add(wg.toTopicInfo());
|
||||
}
|
||||
} finally {
|
||||
readUnlock();
|
||||
}
|
||||
return workloadGroups;
|
||||
}
|
||||
|
||||
public QueryQueue getWorkloadGroupQueryQueue(ConnectContext context) throws UserException {
|
||||
String groupName = getWorkloadGroupNameAndCheckPriv(context);
|
||||
readLock();
|
||||
|
||||
Reference in New Issue
Block a user