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

support for federated clusters #15181

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from 3 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 @@ -85,6 +85,7 @@ public class QueryContexts
public static final String SERIALIZE_DATE_TIME_AS_LONG_INNER_KEY = "serializeDateTimeAsLongInner";
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
public static final String FEDERATED_CLUSSTER_BROKERS = "federatedClusterBrokers";
599166320 marked this conversation as resolved.
Show resolved Hide resolved

// SQL query context keys
public static final String CTX_SQL_QUERY_ID = BaseQuery.SQL_QUERY_ID;
Expand Down
17 changes: 17 additions & 0 deletions server/src/main/java/org/apache/druid/client/BrokerServerView.java
Original file line number Diff line number Diff line change
Expand Up @@ -72,6 +72,7 @@ public class BrokerServerView implements TimelineServerView
private final Object lock = new Object();

private final ConcurrentMap<String, QueryableDruidServer> clients = new ConcurrentHashMap<>();
private final ConcurrentMap<String, QueryableDruidServer> federatedClients;
private final Map<SegmentId, ServerSelector> selectors = new HashMap<>();
private final Map<String, VersionedIntervalTimeline<String, ServerSelector>> timelines = new HashMap<>();
private final ConcurrentMap<TimelineCallback, Executor> timelineCallbacks = new ConcurrentHashMap<>();
Expand Down Expand Up @@ -104,6 +105,7 @@ public BrokerServerView(
this.queryWatcher = queryWatcher;
this.smileMapper = smileMapper;
this.httpClient = httpClient;
this.federatedClients = new ConcurrentHashMap<>();
this.baseView = baseView;
this.tierSelectorStrategy = tierSelectorStrategy;
this.emitter = emitter;
Expand Down Expand Up @@ -241,6 +243,21 @@ private QueryableDruidServer addServer(DruidServer server)
return retVal;
}

@Override
public QueryableDruidServer getAndAddServer(String hostAndPort)
{
if (federatedClients.containsKey(hostAndPort)) {
return federatedClients.get(hostAndPort);
}
DruidServer server = new DruidServer(hostAndPort, hostAndPort, null, 0, ServerType.BROKER, hostAndPort, 0);
QueryableDruidServer retVal = new QueryableDruidServer<>(server, makeDirectClient(server));
QueryableDruidServer exists = federatedClients.put(server.getName(), retVal);
if (exists != null) {
log.warn("QueryRunner for server[%s] already exists!? Well it's getting replaced", server);
}
return retVal;
}

private DirectDruidClient makeDirectClient(DruidServer server)
{
return new DirectDruidClient(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -372,6 +372,7 @@ ClusterQueryResult<T> run(
List<Sequence<T>> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size());
addSequencesFromCache(sequencesByInterval, alreadyCachedResults);
addSequencesFromServer(sequencesByInterval, segmentsByServer);
addSequencesFromFederatedCluster(sequencesByInterval);
return merge(sequencesByInterval);
});

Expand Down Expand Up @@ -675,6 +676,32 @@ private void addSequencesFromServer(
});
}

private void addSequencesFromFederatedCluster(
final List<Sequence<T>> listOfSequences
)
{
if (query.context().containsKey(QueryContexts.FEDERATED_CLUSSTER_BROKERS)) {
String[] brokers = query.context().getString(QueryContexts.FEDERATED_CLUSSTER_BROKERS).split(",");
for (String hostName : brokers) {
if (hostName.length() > 0) {
final QueryRunner serverRunner = serverView.getAndAddServer(hostName).getQueryRunner();
// Divide user-provided maxQueuedBytes by the number of servers, and limit each server to that much.
final long maxQueuedBytes = query.context().getMaxQueuedBytes(httpClientConfig.getMaxQueuedBytes()) / brokers.length;
final Sequence<T> serverResults = serverRunner.run(
queryPlus.withQuery(queryPlus.getQuery()
.withOverriddenContext(ImmutableMap.of(
QueryContexts.FEDERATED_CLUSSTER_BROKERS,
""
)))
.withMaxQueuedBytes(maxQueuedBytes),
responseContext
);
listOfSequences.add(serverResults);
}
}
}
}

@SuppressWarnings("unchecked")
private Sequence<T> getBySegmentServerResults(
final QueryRunner serverRunner,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@

package org.apache.druid.client;

import org.apache.druid.client.selector.QueryableDruidServer;
import org.apache.druid.client.selector.ServerSelector;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.server.coordination.DruidServerMetadata;
Expand Down Expand Up @@ -102,4 +104,10 @@ interface TimelineCallback
*/
CallbackAction serverSegmentRemoved(DruidServerMetadata server, DataSegment segment);
}

default QueryableDruidServer getAndAddServer(String hostAndPort)
{
throw new ISE("This function currently does not support being called.");
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy;
import org.apache.druid.client.selector.QueryableDruidServer;
import org.apache.druid.client.selector.RandomServerSelectorStrategy;
import org.apache.druid.client.selector.ServerSelector;
import org.apache.druid.curator.CuratorTestBase;
Expand All @@ -37,9 +38,11 @@
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryToolChestWarehouse;
import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.TableDataSource;
import org.apache.druid.query.planning.DataSourceAnalysis;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType;
Expand All @@ -61,6 +64,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Executor;
Expand Down Expand Up @@ -526,6 +530,63 @@ public void testEmptyIgnoredTiersConfig() throws Exception
setupViews(null, Collections.emptySet(), true);
}

@Test
public void testGetAndAddServer() throws Exception
{
setupViews(null, null, false);
String hostAndPort = "127.0.0.1";
QueryableDruidServer queryableDruidServer = brokerServerView.getAndAddServer(hostAndPort);
Assert.assertEquals(queryableDruidServer.getServer().getHostAndPort(), hostAndPort);
Assert.assertEquals(queryableDruidServer, brokerServerView.getAndAddServer(hostAndPort));
}

@Test(expected = ISE.class)
public void testGetAndAddServerException() throws Exception
{
setupViews(null, null, false);
new TimelineServerView()
{
@Override
public Optional<? extends TimelineLookup<String, ServerSelector>> getTimeline(DataSourceAnalysis analysis)
{
return Optional.empty();
}

@Override
public List<ImmutableDruidServer> getDruidServers()
{
return null;
}

@Override
public <T> QueryRunner<T> getQueryRunner(DruidServer server)
{
return null;
}

@Override
public void registerTimelineCallback(Executor exec, TimelineCallback callback)
{

}

@Override
public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback)
{

}

@Override
public void registerSegmentCallback(Executor exec, SegmentCallback callback)
{

}

}.getAndAddServer("localhost:8888");

}


/**
* Creates a DruidServer of type HISTORICAL and sets up a ZNode for it.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2675,6 +2675,12 @@ public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback c
{

}

@Override
public QueryableDruidServer getAndAddServer(String hostAndPort)
{
return serverView.getAndAddServer(hostAndPort);
}
},
cache,
JSON_MAPPER,
Expand Down Expand Up @@ -3139,6 +3145,72 @@ public void testEtagforDifferentQueryInterval()
Assert.assertNotEquals(etag1, etag2);
}

@Test
public void testAddSequencesFromFederatedCluster()
{

Map<String, Object> context = new HashMap<>();
context.put(QueryContexts.FEDERATED_CLUSSTER_BROKERS, "test1");
context.putAll(CONTEXT);

final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder()
.dataSource(DATA_SOURCE)
.granularity(GRANULARITY)
.intervals(SEG_SPEC)
.context(context)
.intervals("2011-01-05/2011-01-10")
.aggregators(RENAMED_AGGS)
.postAggregators(RENAMED_POST_AGGS);

TimeseriesQuery query = builder.randomQueryId().build();

final Interval interval1 = Intervals.of("2011-01-06/2011-01-07");

QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest());

final DruidServer lastServer = servers[random.nextInt(servers.length)];
ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 0);

timeline.add(interval1, "v", new NumberedPartitionChunk<>(0, 3, selector1));

final Capture<QueryPlus> capture = Capture.newInstance();
final Capture<ResponseContext> contextCap = Capture.newInstance();

QueryRunner mockRunner = EasyMock.createNiceMock(QueryRunner.class);
QueryRunner mockFederatedClusterRunner = EasyMock.createNiceMock(QueryRunner.class);
QueryableDruidServer queryableDruidServer = EasyMock.createNiceMock(QueryableDruidServer.class);

EasyMock.expect(mockRunner.run(EasyMock.capture(capture), EasyMock.capture(contextCap)))
.andReturn(Sequences.empty())
.anyTimes();
EasyMock.expect(serverView.getQueryRunner(lastServer))
.andReturn(mockRunner)
.anyTimes();

EasyMock.expect(serverView.getAndAddServer("test1"))
.andReturn(queryableDruidServer)
.anyTimes();

EasyMock.expect(queryableDruidServer.getQueryRunner())
.andReturn(mockFederatedClusterRunner)
.anyTimes();

EasyMock.expect(mockFederatedClusterRunner.run(
EasyMock.capture(Capture.newInstance()),
EasyMock.capture(Capture.newInstance())
))
.andReturn(Sequences.empty())
.anyTimes();

EasyMock.replay(serverView);
EasyMock.replay(mockRunner);
EasyMock.replay(mockFederatedClusterRunner);
EasyMock.replay(queryableDruidServer);

runner.run(QueryPlus.wrap(query)).toList();
}


@SuppressWarnings("unchecked")
private QueryRunner getDefaultQueryRunner()
{
Expand Down