mirror of https://github.com/apache/druid.git
Merge branch 'druid-0.7.x' into offheap-incremental-index
This commit is contained in:
commit
a15d32da9b
|
@ -4,7 +4,7 @@ Druid is a distributed, column-oriented, real-time analytics data store that is
|
|||
commonly used to power exploratory dashboards in multi-tenant environments.
|
||||
Druid excels as a data warehousing solution for fast aggregate queries on
|
||||
petabyte sized data sets. Druid supports a variety of flexible filters, exact
|
||||
calculations, approximate algorithms, and other useful calculations Druid can
|
||||
calculations, approximate algorithms, and other useful calculations. Druid can
|
||||
load both streaming and batch data and integrates with Storm and Hadoop.
|
||||
|
||||
### More Information
|
||||
|
|
|
@ -20,11 +20,12 @@
|
|||
package io.druid.server.initialization;
|
||||
|
||||
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.inject.Binder;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.Injector;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Module;
|
||||
import com.google.inject.servlet.GuiceFilter;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
|
@ -33,10 +34,14 @@ import com.metamx.http.client.response.InputStreamResponseHandler;
|
|||
import com.metamx.http.client.response.StatusResponseHandler;
|
||||
import com.metamx.http.client.response.StatusResponseHolder;
|
||||
import io.druid.guice.Jerseys;
|
||||
import io.druid.guice.JsonConfigProvider;
|
||||
import io.druid.guice.LazySingleton;
|
||||
import io.druid.guice.LifecycleModule;
|
||||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
import io.druid.guice.annotations.Self;
|
||||
import io.druid.initialization.Initialization;
|
||||
import io.druid.server.DruidNode;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.eclipse.jetty.server.Handler;
|
||||
import org.eclipse.jetty.server.Server;
|
||||
|
@ -78,10 +83,7 @@ public class JettyTest
|
|||
|
||||
public static void setProperties()
|
||||
{
|
||||
System.setProperty("druid.host", "localhost:9999");
|
||||
System.setProperty("druid.port", "9999");
|
||||
System.setProperty("druid.server.http.numThreads", "20");
|
||||
System.setProperty("druid.service", "test");
|
||||
System.setProperty("druid.server.http.maxIdleTime", "PT1S");
|
||||
System.setProperty("druid.global.http.readTimeout", "PT1S");
|
||||
}
|
||||
|
@ -91,22 +93,24 @@ public class JettyTest
|
|||
{
|
||||
setProperties();
|
||||
Injector injector = Initialization.makeInjectorWithModules(
|
||||
GuiceInjectors.makeStartupInjector(), Lists.<Object>newArrayList(
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class);
|
||||
Jerseys.addResource(binder, SlowResource.class);
|
||||
Jerseys.addResource(binder, ExceptionResource.class);
|
||||
}
|
||||
}
|
||||
)
|
||||
GuiceInjectors.makeStartupInjector(), ImmutableList.<Object>of(
|
||||
new Module()
|
||||
{
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
JsonConfigProvider.bindInstance(
|
||||
binder, Key.get(DruidNode.class, Self.class), new DruidNode("test", "localhost", 9999)
|
||||
);
|
||||
binder.bind(JettyServerInitializer.class).to(JettyServerInit.class).in(LazySingleton.class);
|
||||
Jerseys.addResource(binder, SlowResource.class);
|
||||
Jerseys.addResource(binder, ExceptionResource.class);
|
||||
LifecycleModule.register(binder, Server.class);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
lifecycle = injector.getInstance(Lifecycle.class);
|
||||
// Jetty is Lazy Initialized do a getInstance
|
||||
injector.getInstance(Server.class);
|
||||
lifecycle.start();
|
||||
ClientHolder holder = injector.getInstance(ClientHolder.class);
|
||||
client = holder.getClient();
|
||||
|
@ -136,12 +140,9 @@ public class JettyTest
|
|||
long startTime = System.currentTimeMillis();
|
||||
long startTime2 = 0;
|
||||
try {
|
||||
ListenableFuture<StatusResponseHolder> go = client.get(
|
||||
new URL(
|
||||
"http://localhost:9999/slow/hello"
|
||||
)
|
||||
)
|
||||
.go(new StatusResponseHandler(Charset.defaultCharset()));
|
||||
ListenableFuture<StatusResponseHolder> go =
|
||||
client.get(new URL("http://localhost:9999/slow/hello"))
|
||||
.go(new StatusResponseHandler(Charset.defaultCharset()));
|
||||
startTime2 = System.currentTimeMillis();
|
||||
go.get();
|
||||
}
|
||||
|
@ -150,13 +151,13 @@ public class JettyTest
|
|||
}
|
||||
finally {
|
||||
System.out
|
||||
.println(
|
||||
"Response time client"
|
||||
+ (System.currentTimeMillis() - startTime)
|
||||
+ "time taken for getting future"
|
||||
+ (System.currentTimeMillis() - startTime2)
|
||||
+ "Counter " + count.incrementAndGet()
|
||||
);
|
||||
.println(
|
||||
"Response time client"
|
||||
+ (System.currentTimeMillis() - startTime)
|
||||
+ "time taken for getting future"
|
||||
+ (System.currentTimeMillis() - startTime2)
|
||||
+ "Counter " + count.incrementAndGet()
|
||||
);
|
||||
latch.countDown();
|
||||
|
||||
}
|
||||
|
@ -178,13 +179,9 @@ public class JettyTest
|
|||
// above bug is not fixed in jetty for gzip encoding, and the chunk is still finalized instead of throwing exception.
|
||||
public void testChunkNotFinalized() throws Exception
|
||||
{
|
||||
ListenableFuture<InputStream> go = client.get(
|
||||
new URL(
|
||||
"http://localhost:9999/exception/exception"
|
||||
)
|
||||
|
||||
)
|
||||
.go(new InputStreamResponseHandler());
|
||||
ListenableFuture<InputStream> go =
|
||||
client.get(new URL("http://localhost:9999/exception/exception"))
|
||||
.go(new InputStreamResponseHandler());
|
||||
try {
|
||||
StringWriter writer = new StringWriter();
|
||||
IOUtils.copy(go.get(), writer, "utf-8");
|
||||
|
|
Loading…
Reference in New Issue