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

[IOTDB-6327] Random choosing available nodes to send sql requests #12511

Merged
merged 8 commits into from
May 15, 2024
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 @@ -22,9 +22,12 @@
import org.apache.iotdb.common.rpc.thrift.TEndPoint;

import java.util.List;
import java.util.Optional;
import java.util.function.Supplier;

public interface INodeSupplier extends Supplier<List<TEndPoint>> {

void close();

Optional<TEndPoint> getQueryEndPoint();
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,11 +24,14 @@

import java.util.Collections;
import java.util.List;
import java.util.Optional;

public class DummyNodesSupplier implements INodeSupplier {

private final List<TEndPoint> availableNodes;

private final QueryEndPointPolicy policy = new RoundRobinPolicy();

public DummyNodesSupplier(List<TEndPoint> availableNodes) {
this.availableNodes = Collections.unmodifiableList(availableNodes);
}
Expand All @@ -38,6 +41,15 @@ public void close() {
// do nothing
}

@Override
public Optional<TEndPoint> getQueryEndPoint() {
if (availableNodes == null || availableNodes.isEmpty()) {
return Optional.empty();
} else {
return Optional.of(policy.chooseOne(availableNodes));
}
}

@Override
public List<TEndPoint> get() {
return availableNodes;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,10 @@

import java.time.ZoneId;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArraySet;
import java.util.Optional;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;

Expand All @@ -55,9 +56,9 @@ public class NodesSupplier implements INodeSupplier, Runnable {

private static final int FETCH_SIZE = 10_000;

// availableNodes won't be updated frequently, so we use CopyOnWriteArraySet which is thread-safe
// availableNodes won't be updated frequently, so we use CopyOnWriteArrayList which is thread-safe
// and is optimized for scenarios of reading more and writing less
private volatile Set<TEndPoint> availableNodes = new CopyOnWriteArraySet<>();
private volatile List<TEndPoint> availableNodes = new CopyOnWriteArrayList<>();

private final boolean useSSL;
private final String trustStore;
Expand All @@ -77,6 +78,8 @@ public class NodesSupplier implements INodeSupplier, Runnable {

private final String version;

private final QueryEndPointPolicy policy = new RoundRobinPolicy();

private ThriftConnection client;

private volatile boolean closed = false;
Expand Down Expand Up @@ -133,7 +136,7 @@ private NodesSupplier(
String trustStorePwd,
boolean enableRPCCompression,
String version) {
this.availableNodes.addAll(endPointList);
this.availableNodes.addAll(new HashSet<>(endPointList));
this.userName = userName;
this.password = password;
this.useSSL = useSSL;
Expand All @@ -151,7 +154,7 @@ private NodesSupplier(
// and the List needn't be thread-safe, because it will only be used in one thread.
@Override
public List<TEndPoint> get() {
return new ArrayList<>(availableNodes);
return availableNodes;
}

@Override
Expand Down Expand Up @@ -213,6 +216,15 @@ public void close() {
destroyCurrentClient();
}

@Override
public Optional<TEndPoint> getQueryEndPoint() {
if (availableNodes == null || availableNodes.isEmpty()) {
return Optional.empty();
} else {
return Optional.of(policy.chooseOne(get()));
}
}

private boolean updateDataNodeList() {
try (SessionDataSet sessionDataSet =
client.executeQueryStatement(SHOW_DATA_NODES_COMMAND, TIMEOUT_IN_MS, FETCH_SIZE)) {
Expand All @@ -231,7 +243,7 @@ private boolean updateDataNodeList() {
}
// replace the older ones.
if (!res.isEmpty()) {
availableNodes = new CopyOnWriteArraySet<>(res);
availableNodes = res;
}
return true;
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.iotdb.session;

import org.apache.iotdb.common.rpc.thrift.TEndPoint;

import java.util.List;

public interface QueryEndPointPolicy {

TEndPoint chooseOne(List<TEndPoint> endPointList);
}
Original file line number Diff line number Diff line change
@@ -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.iotdb.session;

import org.apache.iotdb.common.rpc.thrift.TEndPoint;

import java.util.List;

public class RoundRobinPolicy implements QueryEndPointPolicy {

private int index = 0;

@Override
public TEndPoint chooseOne(List<TEndPoint> endPointList) {
int tmp = index;
if (tmp >= endPointList.size()) {
tmp = 0;
}
index = tmp + 1;
return endPointList.get(tmp);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
Expand Down Expand Up @@ -888,7 +889,7 @@ public SessionDataSet executeQueryStatement(String sql, long timeoutInMs)
private SessionDataSet executeStatementMayRedirect(String sql, long timeoutInMs)
throws StatementExecutionException, IoTDBConnectionException {
try {
return defaultSessionConnection.executeQueryStatement(sql, timeoutInMs);
return getQuerySessionConnection().executeQueryStatement(sql, timeoutInMs);
} catch (RedirectException e) {
handleQueryRedirection(e.getEndPoint());
if (enableQueryRedirection) {
Expand All @@ -905,6 +906,25 @@ private SessionDataSet executeStatementMayRedirect(String sql, long timeoutInMs)
}
}

private SessionConnection getQuerySessionConnection() {
Optional<TEndPoint> endPoint =
availableNodes == null ? Optional.empty() : availableNodes.getQueryEndPoint();
if (!endPoint.isPresent() || endPointToSessionConnection == null) {
return defaultSessionConnection;
}
SessionConnection connection =
endPointToSessionConnection.computeIfAbsent(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If both enableRedirection and enableQueryRedirection are set to false, this will cause an NPE since endPointToSessionConnection has not been initialized.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

fixed

endPoint.get(),
k -> {
try {
return constructSessionConnection(this, endPoint.get(), zoneId);
} catch (IoTDBConnectionException ex) {
return null;
}
});
return connection == null ? defaultSessionConnection : connection;
}

/**
* execute non query statement
*
Expand Down
Loading