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

WIP: optionally configure DirectDruidClient to use /druid/v3 instead of /druid/v2 #3323

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
17 changes: 14 additions & 3 deletions server/src/main/java/io/druid/client/BrokerServerView.java
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,6 @@
import io.druid.timeline.VersionedIntervalTimeline;
import io.druid.timeline.partition.PartitionChunk;

import javax.annotation.Nullable;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
Expand Down Expand Up @@ -72,6 +71,8 @@ public class BrokerServerView implements TimelineServerView
private final ServiceEmitter emitter;
private final Predicate<Pair<DruidServerMetadata, DataSegment>> segmentFilter;

private final DirectDruidClientConfig directDruidClientConfig;

private volatile boolean initialized = false;

@Inject
Expand All @@ -83,7 +84,8 @@ public BrokerServerView(
FilteredServerInventoryView baseView,
TierSelectorStrategy tierSelectorStrategy,
ServiceEmitter emitter,
final BrokerSegmentWatcherConfig segmentWatcherConfig
final BrokerSegmentWatcherConfig segmentWatcherConfig,
final DirectDruidClientConfig directDruidClientConfig
)
{
this.warehouse = warehouse;
Expand All @@ -93,6 +95,7 @@ public BrokerServerView(
this.baseView = baseView;
this.tierSelectorStrategy = tierSelectorStrategy;
this.emitter = emitter;
this.directDruidClientConfig = directDruidClientConfig;
this.clients = Maps.newConcurrentMap();
this.selectors = Maps.newHashMap();
this.timelines = Maps.newHashMap();
Expand Down Expand Up @@ -200,7 +203,15 @@ private QueryableDruidServer addServer(DruidServer server)

private DirectDruidClient makeDirectClient(DruidServer server)
{
return new DirectDruidClient(warehouse, queryWatcher, smileMapper, httpClient, server.getHost(), emitter);
return new DirectDruidClient(
warehouse,
queryWatcher,
smileMapper,
httpClient,
server.getHost(),
emitter,
directDruidClientConfig
);
}

private QueryableDruidServer removeServer(DruidServer server)
Expand Down
113 changes: 10 additions & 103 deletions server/src/main/java/io/druid/client/DirectDruidClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,24 +19,19 @@

package io.druid.client;

import com.fasterxml.jackson.core.JsonParser;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.core.ObjectCodec;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.type.TypeFactory;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes;
import com.google.common.base.Charsets;
import com.google.common.base.Strings;
import com.google.common.base.Throwables;
import com.google.common.collect.Maps;
import com.google.common.io.ByteSource;
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.metamx.common.IAE;
import com.metamx.common.Pair;
import com.metamx.common.RE;
import com.metamx.common.guava.BaseSequence;
Expand All @@ -54,9 +49,7 @@
import com.metamx.http.client.response.StatusResponseHolder;
import io.druid.query.BaseQuery;
import io.druid.query.BySegmentResultValueClass;
import io.druid.query.DruidMetrics;
import io.druid.query.Query;
import io.druid.query.QueryInterruptedException;
import io.druid.query.QueryRunner;
import io.druid.query.QueryToolChest;
import io.druid.query.QueryToolChestWarehouse;
Expand All @@ -71,18 +64,14 @@
import org.jboss.netty.handler.codec.http.HttpResponse;

import javax.ws.rs.core.MediaType;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.io.SequenceInputStream;
import java.net.URL;
import java.util.Enumeration;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CancellationException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
Expand All @@ -106,13 +95,16 @@ public class DirectDruidClient<T> implements QueryRunner<T>
private final AtomicInteger openConnections;
private final boolean isSmile;

private final DirectDruidClientConfig config;

public DirectDruidClient(
QueryToolChestWarehouse warehouse,
QueryWatcher queryWatcher,
ObjectMapper objectMapper,
HttpClient httpClient,
String host,
ServiceEmitter emitter
ServiceEmitter emitter,
DirectDruidClientConfig config
)
{
this.warehouse = warehouse;
Expand All @@ -121,6 +113,7 @@ public DirectDruidClient(
this.httpClient = httpClient;
this.host = host;
this.emitter = emitter;
this.config = config;

this.isSmile = this.objectMapper.getFactory() instanceof SmileFactory;
this.openConnections = new AtomicInteger();
Expand Down Expand Up @@ -156,7 +149,7 @@ public Sequence<T> run(final Query<T> query, final Map<String, Object> context)
}

final ListenableFuture<InputStream> future;
final String url = String.format("http://%s/druid/v2/", host);
final String url = String.format("http://%s/%s", host, config.getQueryResponseIteratorFactory().getQueryUrlPath());
final String cancelUrl = String.format("http://%s/druid/v2/%s", host, query.getId());

try {
Expand Down Expand Up @@ -387,16 +380,16 @@ public void onFailure(Throwable t)
}

Sequence<T> retVal = new BaseSequence<>(
new BaseSequence.IteratorMaker<T, JsonParserIterator<T>>()
new BaseSequence.IteratorMaker<T, QueryResponseIterator<T>>()
{
@Override
public JsonParserIterator<T> make()
public QueryResponseIterator make()
{
return new JsonParserIterator<T>(typeRef, future, url);
return config.getQueryResponseIteratorFactory().make(typeRef, future, url, objectMapper, host, context);
}

@Override
public void cleanup(JsonParserIterator<T> iterFromMake)
public void cleanup(QueryResponseIterator<T> iterFromMake)
{
CloseQuietly.close(iterFromMake);
}
Expand All @@ -417,90 +410,4 @@ public void cleanup(JsonParserIterator<T> iterFromMake)

return retVal;
}

private class JsonParserIterator<T> implements Iterator<T>, Closeable
{
private JsonParser jp;
private ObjectCodec objectCodec;
private final JavaType typeRef;
private final Future<InputStream> future;
private final String url;

public JsonParserIterator(JavaType typeRef, Future<InputStream> future, String url)
{
this.typeRef = typeRef;
this.future = future;
this.url = url;
jp = null;
}

@Override
public boolean hasNext()
{
init();

if (jp.isClosed()) {
return false;
}
if (jp.getCurrentToken() == JsonToken.END_ARRAY) {
CloseQuietly.close(jp);
return false;
}

return true;
}

@Override
public T next()
{
init();
try {
final T retVal = objectCodec.readValue(jp, typeRef);
jp.nextToken();
return retVal;
}
catch (IOException e) {
throw Throwables.propagate(e);
}
}

@Override
public void remove()
{
throw new UnsupportedOperationException();
}

private void init()
{
if (jp == null) {
try {
jp = objectMapper.getFactory().createParser(future.get());
final JsonToken nextToken = jp.nextToken();
if (nextToken == JsonToken.START_OBJECT) {
QueryInterruptedException cause = jp.getCodec().readValue(jp, QueryInterruptedException.class);
throw new QueryInterruptedException(cause, host);
} else if (nextToken != JsonToken.START_ARRAY) {
throw new IAE("Next token wasn't a START_ARRAY, was[%s] from url [%s]", jp.getCurrentToken(), url);
} else {
jp.nextToken();
objectCodec = jp.getCodec();
}
}
catch (IOException | InterruptedException | ExecutionException e) {
throw new RE(e, "Failure getting results from[%s] because of [%s]", url, e.getMessage());
}
catch (CancellationException e) {
throw new QueryInterruptedException(e, host);
}
}
}

@Override
public void close() throws IOException
{
if (jp != null) {
jp.close();
}
}
}
}
43 changes: 43 additions & 0 deletions server/src/main/java/io/druid/client/DirectDruidClientConfig.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.client;

import com.fasterxml.jackson.annotation.JsonProperty;

/**
*/
public class DirectDruidClientConfig
{
@JsonProperty("queryResponseIterator")
private QueryResponseIteratorFactory queryResponseIteratorFactory = new V2QueryResponseIteratorFactory();

@JsonProperty
private String stuff;

public QueryResponseIteratorFactory getQueryResponseIteratorFactory()
{
return queryResponseIteratorFactory;
}

public String getStuff()
{
return stuff;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.client;

import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.databind.JavaType;
import com.fasterxml.jackson.databind.ObjectMapper;

import java.io.Closeable;
import java.io.InputStream;
import java.util.Iterator;
import java.util.Map;
import java.util.concurrent.Future;

@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = V2QueryResponseIteratorFactory.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "v2", value = V2QueryResponseIteratorFactory.class),
@JsonSubTypes.Type(name = "v3", value = V3QueryResponseIteratorFactory.class)
})
interface QueryResponseIteratorFactory
{
String getQueryUrlPath();

QueryResponseIterator make(
JavaType typeRef,
Future<InputStream> future,
String url,
ObjectMapper objectMapper,
String host,
Map<String, Object> responseContext
);
}

// marker interface for QueryResponseIterator for the response received from historicals/realtime-tasks
interface QueryResponseIterator<T> extends Iterator<T>, Closeable
{

}