You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2013/05/14 03:52:41 UTC
[01/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Updated Branches:
refs/heads/execwork f0be80dcd -> e57a8d6d4
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/pom.xml b/sandbox/prototype/pom.xml
index fa89417..25f156d 100644
--- a/sandbox/prototype/pom.xml
+++ b/sandbox/prototype/pom.xml
@@ -137,7 +137,7 @@
<dependency>
<groupId>de.huxhorn.lilith</groupId>
<artifactId>de.huxhorn.lilith.logback.appender.multiplex-classic</artifactId>
- <version>0.9.42.1</version>
+ <version>0.9.43</version>
<scope>test</scope>
</dependency>
[05/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index bc942ac..52bb0a2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -17,9 +17,9 @@
******************************************************************************/
package org.apache.drill.exec.rpc;
-import com.google.protobuf.Internal.EnumLite;
import io.netty.bootstrap.ServerBootstrap;
import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
@@ -27,23 +27,32 @@ import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioServerSocketChannel;
import io.netty.handler.logging.LogLevel;
import io.netty.handler.logging.LoggingHandler;
-import org.apache.drill.exec.exception.DrillbitStartupException;
import java.io.IOException;
import java.net.BindException;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
+
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
/**
* A server is bound to a port and is responsible for responding to various type of requests. In some cases, the inbound
* requests will generate more than one outbound request.
*/
-public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
+public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection> extends RpcBus<T, C>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicServer.class);
private ServerBootstrap b;
private volatile boolean connect = false;
+ private final EventLoopGroup eventLoopGroup;
- public BasicServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
-
+ public BasicServer(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+ super(rpcMapping);
+ this.eventLoopGroup = eventLoopGroup;
+
b = new ServerBootstrap() //
.channel(NioServerSocketChannel.class) //
.option(ChannelOption.SO_BACKLOG, 100) //
@@ -56,17 +65,19 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
@Override
protected void initChannel(SocketChannel ch) throws Exception {
- ch.closeFuture().addListener(getCloseHandler(ch));
+ C connection = initRemoteConnection(ch);
+ ch.closeFuture().addListener(getCloseHandler(connection));
ch.pipeline().addLast( //
new ZeroCopyProtobufLengthDecoder(), //
- new RpcDecoder(), //
- new RpcEncoder(), //
- new InboundHandler(ch), //
+ new RpcDecoder(rpcConfig.getName()), //
+ new RpcEncoder(rpcConfig.getName()), //
+ getHandshakeHandler(),
+ new InboundHandler(connection), //
new RpcExceptionHandler() //
);
- channel = ch;
connect = true;
+
}
});
}
@@ -76,12 +87,34 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
return false;
}
+
+ protected abstract ServerHandshakeHandler<?> getHandshakeHandler();
+ protected static abstract class ServerHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
+
+ public ServerHandshakeHandler(EnumLite handshakeType, Parser<T> parser) {
+ super(handshakeType, parser);
+ }
+
+ @Override
+ protected final void consumeHandshake(Channel c, T inbound) throws Exception {
+ OutboundRpcMessage msg = new OutboundRpcMessage(RpcMode.RESPONSE, this.handshakeType, coordinationId, getHandshakeResponse(inbound));
+ c.write(msg);
+ }
+
+ public abstract MessageLite getHandshakeResponse(T inbound) throws Exception;
+
+
+
+
+ }
+
+
public int bind(final int initialPort) throws InterruptedException, DrillbitStartupException{
- int port = initialPort;
+ int port = initialPort-1;
while (true) {
try {
- b.bind(port++).sync();
+ b.bind(++port).sync();
break;
} catch (Exception e) {
if (e instanceof BindException)
@@ -89,13 +122,15 @@ public abstract class BasicServer<T extends EnumLite> extends RpcBus<T>{
throw new DrillbitStartupException("Could not bind Drillbit", e);
}
}
+
connect = !connect;
+ logger.debug("Server started on port {} of type {} ", port, this.getClass().getSimpleName());
return port;
}
@Override
public void close() throws IOException {
- if(b != null) b.shutdown();
+ eventLoopGroup.shutdownGracefully();
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
index c796e2d..70142bb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/CoordinationQueue.java
@@ -29,21 +29,21 @@ public class CoordinationQueue {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CoordinationQueue.class);
private final PositiveAtomicInteger circularInt = new PositiveAtomicInteger();
- private final Map<Integer, DrillRpcFuture<?>> map;
+ private final Map<Integer, DrillRpcFutureImpl<?>> map;
public CoordinationQueue(int segmentSize, int segmentCount) {
- map = new ConcurrentHashMap<Integer, DrillRpcFuture<?>>(segmentSize, 0.75f, segmentCount);
+ map = new ConcurrentHashMap<Integer, DrillRpcFutureImpl<?>>(segmentSize, 0.75f, segmentCount);
}
void channelClosed(Exception ex) {
- for (DrillRpcFuture<?> f : map.values()) {
+ for (DrillRpcFutureImpl<?> f : map.values()) {
f.setException(ex);
}
}
- public <V> DrillRpcFuture<V> getNewFuture(Class<V> clazz) {
+ public <V> DrillRpcFutureImpl<V> getNewFuture(Class<V> clazz) {
int i = circularInt.getNext();
- DrillRpcFuture<V> future = DrillRpcFuture.getNewFuture(i, clazz);
+ DrillRpcFutureImpl<V> future = DrillRpcFutureImpl.getNewFuture(i, clazz);
// logger.debug("Writing to map coord {}, future {}", i, future);
Object old = map.put(i, future);
if (old != null)
@@ -52,8 +52,8 @@ public class CoordinationQueue {
return future;
}
- private DrillRpcFuture<?> removeFromMap(int coordinationId) {
- DrillRpcFuture<?> rpc = map.remove(coordinationId);
+ private DrillRpcFutureImpl<?> removeFromMap(int coordinationId) {
+ DrillRpcFutureImpl<?> rpc = map.remove(coordinationId);
if (rpc == null) {
logger.error("Rpc is null.");
throw new IllegalStateException(
@@ -62,23 +62,25 @@ public class CoordinationQueue {
return rpc;
}
- public <V> DrillRpcFuture<V> getFuture(int coordinationId, Class<V> clazz) {
+ public <V> DrillRpcFutureImpl<V> getFuture(int rpcType, int coordinationId, Class<V> clazz) {
// logger.debug("Getting future for coordinationId {} and class {}", coordinationId, clazz);
- DrillRpcFuture<?> rpc = removeFromMap(coordinationId);
+ DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
// logger.debug("Got rpc from map {}", rpc);
Class<?> outcomeClass = rpc.getOutcomeClass();
+
if (outcomeClass != clazz) {
- logger.error("Rpc class is not expected class. Original: {}, requested: {}", outcomeClass.getCanonicalName(), clazz.getCanonicalName());
+
throw new IllegalStateException(
String
.format(
- "You attempted to request a future for a coordination id that has a different value class than was used when you "
- + "initially created the coordination id. Requested class %s, originally expected class %s. This shouldn't happen. ",
- clazz.getCanonicalName(), outcomeClass.getCanonicalName()));
+ "RPC Engine had a submission and response configuration mismatch. The RPC request that you submitted was defined with an expected response type of %s. However, "
+ + "when the response returned, a call to getResponseDefaultInstance() with Rpc number %d provided an expected class of %s. This means either your submission uses the wrong type definition"
+ + "or your getResponseDefaultInstance() method responds the wrong instance type ",
+ clazz.getCanonicalName(), rpcType, outcomeClass.getCanonicalName()));
}
@SuppressWarnings("unchecked")
- DrillRpcFuture<V> crpc = (DrillRpcFuture<V>) rpc;
+ DrillRpcFutureImpl<V> crpc = (DrillRpcFutureImpl<V>) rpc;
// logger.debug("Returning casted future");
return crpc;
@@ -86,7 +88,7 @@ public class CoordinationQueue {
public void updateFailedFuture(int coordinationId, RpcFailure failure) {
// logger.debug("Updating failed future.");
- DrillRpcFuture<?> rpc = removeFromMap(coordinationId);
+ DrillRpcFutureImpl<?> rpc = removeFromMap(coordinationId);
rpc.setException(new RemoteRpcException(failure));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
index 9a4a7f7..bae947a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFuture.java
@@ -17,80 +17,10 @@
******************************************************************************/
package org.apache.drill.exec.rpc;
-import java.util.concurrent.ExecutionException;
+import com.google.common.util.concurrent.CheckedFuture;
-import com.google.common.util.concurrent.AbstractCheckedFuture;
-import com.google.common.util.concurrent.AbstractFuture;
-import com.google.common.util.concurrent.ListenableFuture;
-
-public class DrillRpcFuture<V> extends AbstractCheckedFuture<V, RpcException> {
+public interface DrillRpcFuture<T> extends CheckedFuture<T,RpcException> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFuture.class);
- final int coordinationId;
- private final Class<V> clazz;
-
- public DrillRpcFuture(ListenableFuture<V> delegate, int coordinationId, Class<V> clazz) {
- super(delegate);
- this.coordinationId = coordinationId;
- this.clazz = clazz;
- }
-
- public Class<V> getOutcomeClass(){
- return clazz;
- }
-
- /**
- * Drill doesn't currently support rpc cancellations since nearly all requests should be either instance of
- * asynchronous. Business level cancellation is managed a separate call (e.g. canceling a query.). Calling this method
- * will result in an UnsupportedOperationException.
- */
- @Override
- public boolean cancel(boolean mayInterruptIfRunning) {
- throw new UnsupportedOperationException(
- "Drill doesn't currently support rpc cancellations. See javadocs for more detail.");
- }
-
- @Override
- protected RpcException mapException(Exception ex) {
- if (ex instanceof RpcException) return (RpcException) ex;
-
- if (ex instanceof ExecutionException) {
- Throwable e2 = ex.getCause();
-
- if (e2 instanceof RpcException) {
- return (RpcException) e2;
- }
- }
- return new RpcException(ex);
-
- }
-
- @SuppressWarnings("unchecked")
- void setValue(Object value) {
- assert clazz.isAssignableFrom(value.getClass());
- ((InnerFuture<V>) super.delegate()).setValue((V) value);
- }
-
- boolean setException(Throwable t) {
- return ((InnerFuture<V>) super.delegate()).setException(t);
- }
-
- public static class InnerFuture<T> extends AbstractFuture<T> {
- // we rewrite these so that the parent can see them
-
- void setValue(T value) {
- super.set(value);
- }
-
- protected boolean setException(Throwable t) {
- return super.setException(t);
- }
- }
-
- public static <V> DrillRpcFuture<V> getNewFuture(int coordinationId, Class<V> clazz) {
- InnerFuture<V> f = new InnerFuture<V>();
- return new DrillRpcFuture<V>(f, coordinationId, clazz);
- }
-
-
-}
\ No newline at end of file
+ public void addLightListener(RpcOutcomeListener<T> outcomeListener);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
new file mode 100644
index 0000000..ee14eeb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/DrillRpcFutureImpl.java
@@ -0,0 +1,118 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+
+class DrillRpcFutureImpl<V> extends AbstractCheckedFuture<V, RpcException> implements DrillRpcFuture<V>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRpcFutureImpl.class);
+
+ final int coordinationId;
+ private final Class<V> clazz;
+
+ public DrillRpcFutureImpl(ListenableFuture<V> delegate, int coordinationId, Class<V> clazz) {
+ super(delegate);
+ this.coordinationId = coordinationId;
+ this.clazz = clazz;
+ }
+
+ public Class<V> getOutcomeClass(){
+ return clazz;
+ }
+
+ /**
+ * Drill doesn't currently support rpc cancellations since nearly all requests should be either instance of
+ * asynchronous. Business level cancellation is managed a separate call (e.g. canceling a query.). Calling this method
+ * will result in an UnsupportedOperationException.
+ */
+ @Override
+ public boolean cancel(boolean mayInterruptIfRunning) {
+ throw new UnsupportedOperationException(
+ "Drill doesn't currently support rpc cancellations. See javadocs for more detail.");
+ }
+
+ @Override
+ protected RpcException mapException(Exception ex) {
+ Throwable e = ex;
+ while(e instanceof ExecutionException){
+ e = e.getCause();
+ }
+ if (e instanceof RpcException) return (RpcException) e;
+
+ return new RpcException(ex);
+
+ }
+
+ @SuppressWarnings("unchecked")
+ void setValue(Object value) {
+ assert clazz.isAssignableFrom(value.getClass());
+ ((InnerFuture<V>) super.delegate()).setValue((V) value);
+ }
+
+ boolean setException(Throwable t) {
+ return ((InnerFuture<V>) super.delegate()).setException(t);
+ }
+
+ public static class InnerFuture<T> extends AbstractFuture<T> {
+ // we rewrite these so that the parent can see them
+
+ void setValue(T value) {
+ super.set(value);
+ }
+
+ protected boolean setException(Throwable t) {
+ return super.setException(t);
+ }
+ }
+
+ public class RpcOutcomeListenerWrapper implements Runnable{
+ final RpcOutcomeListener<V> inner;
+
+ public RpcOutcomeListenerWrapper(RpcOutcomeListener<V> inner) {
+ super();
+ this.inner = inner;
+ }
+
+ @Override
+ public void run() {
+ try{
+ inner.success(DrillRpcFutureImpl.this.checkedGet());
+ }catch(RpcException e){
+ inner.failed(e);
+ }
+ }
+ }
+
+ public void addLightListener(RpcOutcomeListener<V> outcomeListener){
+ this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
+ }
+
+
+
+ public static <V> DrillRpcFutureImpl<V> getNewFuture(int coordinationId, Class<V> clazz) {
+ InnerFuture<V> f = new InnerFuture<V>();
+ return new DrillRpcFutureImpl<V>(f, coordinationId, clazz);
+ }
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
index ab977db..be1ff6b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/InboundRpcMessage.java
@@ -18,15 +18,20 @@
package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+
+import java.io.InputStream;
import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
public class InboundRpcMessage extends RpcMessage{
public ByteBuf pBody;
+ public ByteBuf dBody;
public InboundRpcMessage(RpcMode mode, int rpcType, int coordinationId, ByteBuf pBody, ByteBuf dBody) {
- super(mode, rpcType, coordinationId, dBody);
+ super(mode, rpcType, coordinationId);
this.pBody = pBody;
+ this.dBody = dBody;
}
public int getBodySize(){
@@ -37,7 +42,7 @@ public class InboundRpcMessage extends RpcMessage{
void release(){
pBody.release();
- super.release();
+ if(dBody != null) dBody.release();
}
@Override
@@ -46,5 +51,7 @@ public class InboundRpcMessage extends RpcMessage{
+ coordinationId + ", dBody=" + dBody + "]";
}
-
+ public InputStream getProtobufBodyAsIS(){
+ return new ByteBufInputStream(pBody);
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
index 91c3d45..e4858c4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/OutboundRpcMessage.java
@@ -17,6 +17,8 @@
******************************************************************************/
package org.apache.drill.exec.rpc;
+import java.util.Arrays;
+
import io.netty.buffer.ByteBuf;
import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
@@ -24,28 +26,49 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
import com.google.protobuf.Internal.EnumLite;
import com.google.protobuf.MessageLite;
-class OutboundRpcMessage extends RpcMessage{
+public class OutboundRpcMessage extends RpcMessage {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OutboundRpcMessage.class);
final MessageLite pBody;
-
- public OutboundRpcMessage(RpcMode mode, EnumLite rpcType, int coordinationId, MessageLite pBody, ByteBuf dBody) {
- super(mode, rpcType.getNumber(), coordinationId, dBody);
+ public ByteBuf[] dBodies;
+
+ public OutboundRpcMessage(RpcMode mode, EnumLite rpcType, int coordinationId, MessageLite pBody, ByteBuf... dBodies) {
+ super(mode, rpcType.getNumber(), coordinationId);
this.pBody = pBody;
+ this.dBodies = dBodies;
}
-
- public int getBodySize(){
+
+ public int getBodySize() {
int len = pBody.getSerializedSize();
len += RpcEncoder.getRawVarintSize(len);
- if(dBody != null) len += dBody.capacity();
+ len += getRawBodySize();
return len;
}
+ public int getRawBodySize(){
+ if(dBodies == null) return 0;
+ int len = 0;
+
+ for (int i = 0; i < dBodies.length; i++) {
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Reader Index {}, Writer Index {}", dBodies[i].readerIndex(), dBodies[i].writerIndex());
+ len += dBodies[i].readableBytes();
+ }
+ return len;
+ }
+
@Override
public String toString() {
return "OutboundRpcMessage [pBody=" + pBody + ", mode=" + mode + ", rpcType=" + rpcType + ", coordinationId="
- + coordinationId + ", dBody=" + dBody + "]";
+ + coordinationId + ", dBodies=" + Arrays.toString(dBodies) + "]";
}
-
+ void release(){
+ if(dBodies != null){
+ for(ByteBuf b : dBodies){
+ b.release();
+ }
+ }
+ }
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
new file mode 100644
index 0000000..cedba10
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RemoteConnection.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import io.netty.channel.Channel;
+
+public class RemoteConnection{
+ private final Channel channel;
+
+ public RemoteConnection(Channel channel) {
+ super();
+ this.channel = channel;
+ }
+
+
+ public final Channel getChannel() {
+ return channel;
+ }
+
+
+ public ConnectionThrottle getConnectionThrottle(){
+ // can't be implemented until we switch to per query sockets.
+ return null;
+ }
+
+ public interface ConnectionThrottle{
+ public void disableReceiving();
+ public void enableReceiving();
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
index 4bd592b..0c4ab7a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Response.java
@@ -17,6 +17,8 @@
******************************************************************************/
package org.apache.drill.exec.rpc;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+
import io.netty.buffer.ByteBuf;
import com.google.protobuf.Internal.EnumLite;
@@ -27,13 +29,13 @@ public class Response {
public EnumLite rpcType;
public MessageLite pBody;
- public ByteBuf dBody;
+ public ByteBuf[] dBodies;
- public Response(EnumLite rpcType, MessageLite pBody, ByteBuf dBody) {
+ public Response(EnumLite rpcType, MessageLite pBody, ByteBuf... dBodies) {
super();
this.rpcType = rpcType;
this.pBody = pBody;
- this.dBody = dBody;
+ this.dBodies = dBodies;
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
index 76300d1..11764db 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcBus.java
@@ -23,113 +23,140 @@ import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundMessageHandlerAdapter;
-import io.netty.channel.socket.SocketChannel;
import io.netty.util.concurrent.GenericFutureListener;
import java.io.Closeable;
+import java.util.Arrays;
import java.util.concurrent.CancellationException;
import org.apache.drill.exec.proto.GeneralRPCProtos.RpcFailure;
import org.apache.drill.exec.proto.GeneralRPCProtos.RpcMode;
+import org.slf4j.Logger;
import com.google.common.base.Preconditions;
import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.MessageLite;
import com.google.protobuf.Parser;
/**
- * The Rpc Bus deals with incoming and outgoing communication and is used on both the server and the client side of a system.
+ * The Rpc Bus deals with incoming and outgoing communication and is used on both the server and the client side of a
+ * system.
+ *
* @param <T>
*/
-public abstract class RpcBus<T extends EnumLite> implements Closeable{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcBus.class);
-
- private CoordinationQueue queue = new CoordinationQueue(16, 16);
- protected Channel channel;
+public abstract class RpcBus<T extends EnumLite, C extends RemoteConnection> implements Closeable {
+ final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(this.getClass());
+
+ protected final CoordinationQueue queue = new CoordinationQueue(16, 16);
protected abstract MessageLite getResponseDefaultInstance(int rpcType) throws RpcException;
- protected abstract Response handle(SocketChannel channel, int RpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException;
- public abstract boolean isClient();
-
- protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
- SEND protobufBody, Class<RECEIVE> clazz, ByteBuf dataBody) throws RpcException {
+ protected abstract Response handle(C connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException;
+
+ public abstract boolean isClient();
+
+ protected final RpcConfig rpcConfig;
+
+ public RpcBus(RpcConfig rpcConfig) {
+ this.rpcConfig = rpcConfig;
+ }
+
+ public <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(C connection, T rpcType,
+ SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) {
+
+ assert !Arrays.asList(dataBodies).contains(null);
+ assert rpcConfig.checkSend(rpcType, protobufBody.getClass(), clazz);
+
ByteBuf pBuffer = null;
boolean completed = false;
try {
-// logger.debug("Seding message");
+ // logger.debug("Seding message");
Preconditions.checkNotNull(protobufBody);
- DrillRpcFuture<RECEIVE> rpcFuture = queue.getNewFuture(clazz);
- OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, rpcFuture.coordinationId, protobufBody, dataBody);
- ChannelFuture channelFuture = channel.write(m);
+ DrillRpcFutureImpl<RECEIVE> rpcFuture = queue.getNewFuture(clazz);
+ OutboundRpcMessage m = new OutboundRpcMessage(RpcMode.REQUEST, rpcType, rpcFuture.coordinationId, protobufBody, dataBodies);
+ ChannelFuture channelFuture = connection.getChannel().write(m);
channelFuture.addListener(new Listener(rpcFuture.coordinationId, clazz));
completed = true;
return rpcFuture;
} finally {
if (!completed) {
if (pBuffer != null) pBuffer.release();
- if (dataBody != null) dataBody.release();
+ if (dataBodies != null) {
+ for (ByteBuf b : dataBodies) {
+ b.release();
+ }
+
+ }
}
;
}
}
-
- public class ChannelClosedHandler implements GenericFutureListener<ChannelFuture>{
+ public abstract C initRemoteConnection(Channel channel);
+
+ public class ChannelClosedHandler implements GenericFutureListener<ChannelFuture> {
@Override
public void operationComplete(ChannelFuture future) throws Exception {
- logger.info("Channel closed between local {} and remote {}", future.channel().localAddress(), future.channel().remoteAddress());
- queue.channelClosed(new ChannelClosedException());
+ logger.info("Channel closed between local {} and remote {}", future.channel().localAddress(), future.channel()
+ .remoteAddress());
+ closeQueueDueToChannelClose();
}
}
-
- protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch){
+
+ protected void closeQueueDueToChannelClose() {
+ if (this.isClient()) queue.channelClosed(new ChannelClosedException());
+ }
+
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(C clientConnection) {
return new ChannelClosedHandler();
}
-
+
protected class InboundHandler extends ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
- private final SocketChannel channel;
-
-
- public InboundHandler(SocketChannel channel) {
+ private final C connection;
+ public InboundHandler(C connection) {
super();
- this.channel = channel;
+ this.connection = connection;
}
-
@Override
public void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage msg) throws Exception {
- if(!ctx.channel().isOpen()) return;
+ if (!ctx.channel().isOpen()) return;
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received message {}", msg);
- switch(msg.mode){
+ if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Received message {}", msg);
+ switch (msg.mode) {
case REQUEST:
// handle message and ack.
- Response r = handle(channel, msg.rpcType, msg.pBody, msg.dBody);
- OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE, r.rpcType, msg.coordinationId, r.pBody, r.dBody);
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Adding message to outbound buffer. {}", outMessage);
+ Response r = handle(connection, msg.rpcType, msg.pBody, msg.dBody);
+ assert rpcConfig.checkResponseSend(r.rpcType, r.pBody.getClass());
+ OutboundRpcMessage outMessage = new OutboundRpcMessage(RpcMode.RESPONSE, r.rpcType, msg.coordinationId,
+ r.pBody, r.dBodies);
+ if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Adding message to outbound buffer. {}", outMessage);
ctx.write(outMessage);
break;
-
+
case RESPONSE:
MessageLite m = getResponseDefaultInstance(msg.rpcType);
- DrillRpcFuture<?> rpcFuture = queue.getFuture(msg.coordinationId, m.getClass());
+ assert rpcConfig.checkReceive(msg.rpcType, m.getClass());
+ DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(msg.rpcType, msg.coordinationId, m.getClass());
Parser<?> parser = m.getParserForType();
Object value = parser.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
rpcFuture.setValue(value);
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
+ if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future {} with value {}", rpcFuture, value);
+
break;
-
+
case RESPONSE_FAILURE:
RpcFailure failure = RpcFailure.parseFrom(new ByteBufInputStream(msg.pBody, msg.pBody.readableBytes()));
queue.updateFailedFuture(msg.coordinationId, failure);
- if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Updated rpc future with coordinationId {} with failure ", msg.coordinationId, failure);
+ if (RpcConstants.EXTRA_DEBUGGING)
+ logger.debug("Updated rpc future with coordinationId {} with failure ", msg.coordinationId, failure);
break;
-
+
default:
- throw new UnsupportedOperationException();
+ throw new UnsupportedOperationException();
}
}
@@ -147,18 +174,18 @@ public abstract class RpcBus<T extends EnumLite> implements Closeable{
@Override
public void operationComplete(ChannelFuture channelFuture) throws Exception {
-// logger.debug("Completed channel write.");
-
+ // logger.debug("Completed channel write.");
+
if (channelFuture.isCancelled()) {
- DrillRpcFuture<?> rpcFuture = queue.getFuture(coordinationId, clazz);
+ DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
rpcFuture.setException(new CancellationException("Socket operation was canceled."));
} else if (!channelFuture.isSuccess()) {
try {
channelFuture.get();
- throw new IllegalStateException(
- "Future was described as completed and not succesful but did not throw an exception.");
+ throw new IllegalStateException("Future was described as completed and not succesful but did not throw an exception.");
} catch (Exception e) {
- DrillRpcFuture<?> rpcFuture = queue.getFuture(coordinationId, clazz);
+ logger.error("Error occurred during Rpc", e);
+ DrillRpcFutureImpl<?> rpcFuture = queue.getFuture(-1, coordinationId, clazz);
rpcFuture.setException(e);
}
} else {
@@ -168,6 +195,13 @@ public abstract class RpcBus<T extends EnumLite> implements Closeable{
}
}
-
-
+
+ public static <T> T get(ByteBuf pBody, Parser<T> parser) throws RpcException{
+ try {
+ ByteBufInputStream is = new ByteBufInputStream(pBody);
+ return parser.parseFrom(is);
+ } catch (InvalidProtocolBufferException e) {
+ throw new RpcException(String.format("Failure while decoding message with parser of type. %s", parser.getClass().getCanonicalName()), e);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
new file mode 100644
index 0000000..c6b4c49
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcConfig.java
@@ -0,0 +1,150 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+
+public class RpcConfig {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcConfig.class);
+
+ private final String name;
+ private final Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap;
+ private final Map<Integer, RpcMessageType<?, ?, ?>> receiveMap;
+
+ private RpcConfig(String name, Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap, Map<Integer, RpcMessageType<?, ?, ?>> receiveMap){
+ this.name = name;
+ this.sendMap = ImmutableMap.copyOf(sendMap);
+ this.receiveMap = ImmutableMap.copyOf(receiveMap);
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public boolean checkReceive(int rpcType, Class<?> receiveClass){
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking reception for rpcType %d and receive class %s.", rpcType, receiveClass));
+ RpcMessageType<?,?,?> type = receiveMap.get(rpcType);
+ if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc receive type number of %s.", name, rpcType));
+
+ if(receiveClass != type.getRet()){
+ throw new IllegalStateException(String.format("%s: The definition for receive doesn't match implementation code. The definition is %s however the current receive for this type was of type %s.", name, type, receiveClass.getCanonicalName()));
+ }
+ return true;
+ }
+
+ public boolean checkSend(EnumLite send, Class<?> sendClass, Class<?> receiveClass){
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking send classes for send RpcType %s. Send Class is %s and Receive class is %s.", send, sendClass, receiveClass));
+ RpcMessageType<?,?,?> type = sendMap.get(send);
+ if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc send type of %s.", name, send));
+
+ if(type.getSend() != sendClass) throw new IllegalStateException(String.format("%s: The definition for send doesn't match implementation code. The definition is %s however the current send is trying to send an object of type %s.", name, type, sendClass.getCanonicalName()));
+ if(type.getRet() != receiveClass) throw new IllegalStateException(String.format("%s: The definition for send doesn't match implementation code. The definition is %s however the current send is trying to setup an expected reception of an object of type %s.", name, type, receiveClass.getCanonicalName()));
+
+ return true;
+ }
+
+ public boolean checkResponseSend(EnumLite responseType, Class<?> responseClass){
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("Checking responce send of type %s with response class of %s.", responseType, responseClass));
+ RpcMessageType<?,?,?> type = receiveMap.get(responseType.getNumber());
+ if(type == null) throw new IllegalStateException(String.format("%s: There is no defined RpcMessage type for a Rpc response of type %s.", name, responseType));
+ if(type.getRet() != responseClass) throw new IllegalStateException(String.format("%s: The definition for the response doesn't match implementation code. The definition is %s however the current response is trying to response with an object of type %s.", name, type, responseClass.getCanonicalName()));
+
+ return true;
+ }
+
+ public static class RpcMessageType<SEND extends MessageLite, RECEIVE extends MessageLite, T extends EnumLite>{
+ private T sendEnum;
+ private Class<SEND> send;
+ private T receiveEnum;
+ private Class<RECEIVE> ret;
+ public RpcMessageType(T sendEnum, Class<SEND> send, T receiveEnum, Class<RECEIVE> ret) {
+ super();
+ this.sendEnum = sendEnum;
+ this.send = send;
+ this.receiveEnum = receiveEnum;
+ this.ret = ret;
+ }
+ public Class<SEND> getSend() {
+ return send;
+ }
+ public void setSend(Class<SEND> send) {
+ this.send = send;
+ }
+ public T getSendEnum() {
+ return sendEnum;
+ }
+ public void setSendEnum(T sendEnum) {
+ this.sendEnum = sendEnum;
+ }
+ public Class<RECEIVE> getRet() {
+ return ret;
+ }
+ public void setRet(Class<RECEIVE> ret) {
+ this.ret = ret;
+ }
+ public T getReceiveEnum() {
+ return receiveEnum;
+ }
+ public void setReceiveEnum(T receiveEnum) {
+ this.receiveEnum = receiveEnum;
+ }
+ @Override
+ public String toString() {
+ return "RpcMessageType [sendEnum=" + sendEnum + ", send=" + send + ", receiveEnum=" + receiveEnum + ", ret="
+ + ret + "]";
+ }
+
+
+ }
+
+ public static RpcConfigBuilder newBuilder(String name){
+ return new RpcConfigBuilder(name);
+ }
+
+ public static class RpcConfigBuilder {
+ private final String name;
+ private Map<EnumLite, RpcMessageType<?, ?, ?>> sendMap = Maps.newHashMap();
+ private Map<Integer, RpcMessageType<?, ?, ?>> receiveMap = Maps.newHashMap();
+
+ private RpcConfigBuilder(String name){
+ this.name = name;
+ }
+
+ public <SEND extends MessageLite, RECEIVE extends MessageLite, T extends EnumLite> RpcConfigBuilder add(T sendEnum, Class<SEND> send, T receiveEnum, Class<RECEIVE> rec){
+ RpcMessageType<SEND, RECEIVE, T> type = new RpcMessageType<SEND, RECEIVE, T>(sendEnum, send, receiveEnum, rec);
+ this.sendMap.put(sendEnum, type);
+ this.receiveMap.put(receiveEnum.getNumber(), type);
+ return this;
+ }
+
+ public RpcConfig build(){
+ return new RpcConfig(name, sendMap, receiveMap);
+
+ }
+ }
+
+
+
+}
+
+
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
index 134e54b..4e9714b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcDecoder.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.CorruptedFrameException;
import io.netty.handler.codec.MessageToMessageDecoder;
@@ -32,14 +33,19 @@ import org.apache.drill.exec.proto.GeneralRPCProtos.RpcHeader;
* Converts a previously length adjusted buffer into an RpcMessage.
*/
class RpcDecoder extends MessageToMessageDecoder<ByteBuf> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcDecoder.class);
+ final org.slf4j.Logger logger;
private final AtomicLong messageCounter = new AtomicLong();
+ public RpcDecoder(String name){
+ this.logger = org.slf4j.LoggerFactory.getLogger(RpcDecoder.class.getCanonicalName() + "." + name);
+ }
+
+
@Override
- protected InboundRpcMessage decode(ChannelHandlerContext ctx, ByteBuf buffer) throws Exception {
+ protected void decode(ChannelHandlerContext ctx, ByteBuf buffer, MessageBuf<Object> out) throws Exception {
if(!ctx.channel().isOpen()){
- return null;
+ return;
}
if (RpcConstants.EXTRA_DEBUGGING) logger.debug("Inbound rpc message received.");
@@ -94,7 +100,7 @@ class RpcDecoder extends MessageToMessageDecoder<ByteBuf> {
buffer.skipBytes(dBodyLength);
messageCounter.incrementAndGet();
if (RpcConstants.EXTRA_DEBUGGING) logger.trace("Inbound Rpc Message Decoded {}.", m);
- return m;
+ out.add(m);
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
index 8d3d97c..f76d648 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcEncoder.java
@@ -34,7 +34,7 @@ import com.google.protobuf.WireFormat;
* Converts an RPCMessage into wire format.
*/
class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcEncoder.class);
+ final org.slf4j.Logger logger;
static final int HEADER_TAG = makeTag(CompleteRpcMessage.HEADER_FIELD_NUMBER, WireFormat.WIRETYPE_LENGTH_DELIMITED);
static final int PROTOBUF_BODY_TAG = makeTag(CompleteRpcMessage.PROTOBUF_BODY_FIELD_NUMBER, WireFormat.WIRETYPE_LENGTH_DELIMITED);
@@ -43,6 +43,9 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
static final int PROTOBUF_BODY_TAG_LENGTH = getRawVarintSize(PROTOBUF_BODY_TAG);
static final int RAW_BODY_TAG_LENGTH = getRawVarintSize(RAW_BODY_TAG);
+ public RpcEncoder(String name){
+ this.logger = org.slf4j.LoggerFactory.getLogger(RpcEncoder.class.getCanonicalName() + "." + name);
+ }
@Override
public void flush(ChannelHandlerContext ctx, OutboundRpcMessage msg) throws Exception {
@@ -61,7 +64,7 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
// figure out the full length
int headerLength = header.getSerializedSize();
int protoBodyLength = msg.pBody.getSerializedSize();
- int rawBodyLength = msg.dBody == null ? 0 : msg.dBody.readableBytes();
+ int rawBodyLength = msg.getRawBodySize();
int fullLength = //
HEADER_TAG_LENGTH + getRawVarintSize(headerLength) + headerLength + //
PROTOBUF_BODY_TAG_LENGTH + getRawVarintSize(protoBodyLength) + protoBodyLength; //
@@ -89,11 +92,15 @@ class RpcEncoder extends ChannelOutboundMessageHandlerAdapter<OutboundRpcMessage
msg.pBody.writeTo(cos);
// if exists, write data body and tag.
- if(msg.dBody != null && msg.dBody.readableBytes() > 0){
+ // TODO: is it possible to avoid this copy, i think so...
+ if(msg.getRawBodySize() > 0){
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Writing raw body of size {}", msg.getRawBodySize());
cos.writeRawVarint32(RAW_BODY_TAG);
cos.writeRawVarint32(rawBodyLength);
cos.flush(); // need to flush so that dbody goes after if cos is caching.
- buf.writeBytes(msg.dBody);
+ for(int i =0; i < msg.dBodies.length; i++){
+ buf.writeBytes(msg.dBodies[i]);
+ }
}else{
cos.flush();
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
index ef1b88f..a0aed94 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcExceptionHandler.java
@@ -25,28 +25,23 @@ public class RpcExceptionHandler implements ChannelHandler{
public RpcExceptionHandler(){
}
-
- @Override
- public void beforeAdd(ChannelHandlerContext ctx) throws Exception {
- }
- @Override
- public void afterAdd(ChannelHandlerContext ctx) throws Exception {
- }
@Override
- public void beforeRemove(ChannelHandlerContext ctx) throws Exception {
+ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
+ if(!ctx.channel().isOpen()) return;
+ logger.info("Exception in pipeline. Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
+ ctx.close();
}
+
@Override
- public void afterRemove(ChannelHandlerContext ctx) throws Exception {
+ public void handlerAdded(ChannelHandlerContext ctx) throws Exception {
}
+
@Override
- public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception {
- if(!ctx.channel().isOpen()) return;
- logger.info("Exception in pipeline. Closing channel between local " + ctx.channel().localAddress() + " and remote " + ctx.channel().remoteAddress(), cause);
- ctx.close();
+ public void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
index fd1938d..08ea150 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcMessage.java
@@ -27,19 +27,14 @@ public abstract class RpcMessage {
public RpcMode mode;
public int rpcType;
public int coordinationId;
- public ByteBuf dBody;
- public RpcMessage(RpcMode mode, int rpcType, int coordinationId, ByteBuf dBody) {
+ public RpcMessage(RpcMode mode, int rpcType, int coordinationId) {
this.mode = mode;
this.rpcType = rpcType;
this.coordinationId = coordinationId;
- this.dBody = dBody;
}
public abstract int getBodySize();
-
- void release(){
- if(dBody != null) dBody.release();
- }
+ abstract void release();
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
new file mode 100644
index 0000000..fac908c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/RpcOutcomeListener.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+public abstract class RpcOutcomeListener<V> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RpcOutcomeListener.class);
+
+ public void failed(RpcException ex){};
+ public void success(V value){};
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
index 462bc52..20a7d7d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/ZeroCopyProtobufLengthDecoder.java
@@ -18,6 +18,7 @@
package org.apache.drill.exec.rpc;
import io.netty.buffer.ByteBuf;
+import io.netty.buffer.MessageBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.handler.codec.ByteToMessageDecoder;
import io.netty.handler.codec.CorruptedFrameException;
@@ -30,12 +31,14 @@ import com.google.protobuf.CodedInputStream;
public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ZeroCopyProtobufLengthDecoder.class);
+
@Override
- protected ByteBuf decode(ChannelHandlerContext ctx, ByteBuf in) throws Exception {
+ protected void decode(ChannelHandlerContext ctx, ByteBuf in, MessageBuf<Object> out) throws Exception {
+
if(!ctx.channel().isOpen()){
logger.info("Channel is closed, discarding remaining {} byte(s) in buffer.", in.readableBytes());
in.skipBytes(in.readableBytes());
- return null;
+ return;
}
in.markReaderIndex();
@@ -43,7 +46,7 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
for (int i = 0; i < buf.length; i ++) {
if (!in.isReadable()) {
in.resetReaderIndex();
- return null;
+ return;
}
buf[i] = in.readByte();
@@ -60,13 +63,14 @@ public class ZeroCopyProtobufLengthDecoder extends ByteToMessageDecoder {
if (in.readableBytes() < length) {
in.resetReaderIndex();
- return null;
+ return;
} else {
- ByteBuf out = in.slice(in.readerIndex(), length);
+ ByteBuf outBuf = in.slice(in.readerIndex(), length);
in.retain();
in.skipBytes(length);
if(RpcConstants.EXTRA_DEBUGGING) logger.debug(String.format("ReaderIndex is %d after length header of %d bytes and frame body of length %d bytes.", in.readerIndex(), i+1, length));
- return out;
+ out.add(outBuf);
+ return;
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
new file mode 100644
index 0000000..ecaf8d3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/AvailabilityListener.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+public interface AvailabilityListener {
+ public void isAvailable(BitConnection connection);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
index b16c6cb..4ba99a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitClient.java
@@ -18,45 +18,88 @@
package org.apache.drill.exec.rpc.bit;
import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
import io.netty.util.concurrent.GenericFutureListener;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
import org.apache.drill.exec.proto.ExecProtos.RpcType;
import org.apache.drill.exec.rpc.BasicClient;
import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcBus;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
import com.google.protobuf.MessageLite;
-public class BitClient extends BasicClient<RpcType>{
-
- private final DrillbitContext context;
+public class BitClient extends BasicClient<RpcType, BitConnection>{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
+
private final BitComHandler handler;
+ private final DrillbitEndpoint endpoint;
+ private BitConnection connection;
+ private final AvailabilityListener openListener;
+ private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+ private final ListenerPool listeners;
- public BitClient(BitComHandler handler, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
- super(alloc, eventLoopGroup);
- this.context = context;
+ public BitClient(DrillbitEndpoint endpoint, AvailabilityListener openListener, BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
+ super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
+
+ this.endpoint = endpoint;
this.handler = handler;
+ this.openListener = openListener;
+ this.registry = registry;
+ this.listeners = listeners;
}
+ public BitHandshake connect() throws RpcException, InterruptedException{
+ BitHandshake bs = connectAsClient(RpcType.HANDSHAKE, BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getBitPort(), BitHandshake.class);
+ connection.setEndpoint(endpoint);
+ return bs;
+ }
+
+ @SuppressWarnings("unchecked")
@Override
- protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- return handler.getResponseDefaultInstance(rpcType);
+ public BitConnection initRemoteConnection(Channel channel) {
+ this.connection = new BitConnection(openListener, channel, (RpcBus<RpcType, BitConnection>) (RpcBus<?, ?>) this, registry, listeners);
+ return connection;
}
@Override
- protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(context, rpcType, pBody, dBody);
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection clientConnection) {
+ return clientConnection.getCloseHandler(super.getCloseHandler(clientConnection));
}
@Override
- protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch) {
- return super.getCloseHandler(ch);
+ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
}
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitClient.class);
+ @Override
+ protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return handler.handle(connection, rpcType, pBody, dBody);
+ }
+
+ @Override
+ protected ClientHandshakeHandler<BitHandshake> getHandshakeHandler() {
+ return new ClientHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.class, BitHandshake.PARSER){
+
+ @Override
+ protected void validateHandshake(BitHandshake inbound) throws Exception {
+ logger.debug("Handling handshake from bit server to bit client. {}", inbound);
+ if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+ }
+
+ };
+ }
+
+ public BitConnection getConnection(){
+ return this.connection;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
index 2349899..c60d36b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitCom.java
@@ -17,73 +17,34 @@
******************************************************************************/
package org.apache.drill.exec.rpc.bit;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.util.concurrent.GenericFutureListener;
-
import java.io.Closeable;
-import java.util.Collection;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
/**
- * Service that allows one Drillbit to communicate with another. Internally manages whether each particular bit is a server
- * or a client depending on who initially made the connection. If no connection exists, BitCom is
- * responsible for making a connection. BitCom should automatically straight route local BitCommunication rather than connecting to its self.
+ * Service that allows one Drillbit to communicate with another. Internally manages whether each particular bit is a
+ * server or a client depending on who initially made the connection. If no connection exists, BitCom is responsible for
+ * making a connection. BitCom should automatically straight route local BitCommunication rather than connecting to its
+ * self.
*/
-public interface BitCom extends Closeable{
+public interface BitCom extends Closeable {
/**
- * Routes the output of a RecordBatch to another node. The record batch
- * @param node The node id to send the record batch to.
- * @param batch The record batch to send.
- * @return A SendProgress object which can be used to monitor the sending of the batch.
- */
- public abstract DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch);
-
-
- /**
- * Requests an iterator to access an incoming record batch.
- * @param fragmentId
- * @return
- */
- public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId);
-
- /**
- * Send a query PlanFragment to another bit.
- * @param context
+ * Get a Bit to Bit communication tunnel. If the BitCom doesn't have a tunnel attached to the node already, it will
+ * start creating one. This create the connection asynchronously.
+ *
* @param node
- * @param fragment
* @return
*/
- public abstract DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node, PlanFragment fragment);
+ public BitTunnel getTunnel(DrillbitEndpoint node) ;
- public abstract void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId);
-
-
- public abstract DrillRpcFuture<Ack> cancelFragment(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle);
-
- public abstract DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle);
-
-
- public interface TunnelListener extends GenericFutureListener<ChannelFuture> {
- public void connectionEstablished(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus);
- }
-
- public interface SendManager{
- /**
- * Sender responsible for regularly checking this value to see whether it should continue to send or yield the process
- * @return
- */
- public boolean canContinue();
- }
+ public int start() throws InterruptedException, DrillbitStartupException;
+ /**
+ * Register an incoming batch handler for a local foreman.
+ * @param handler
+ */
+ public void registerIncomingBatchHandler(IncomingFragmentHandler handler);
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
new file mode 100644
index 0000000..e1d4902
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComDefaultInstanceHandler.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.protobuf.MessageLite;
+
+public class BitComDefaultInstanceHandler {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComDefaultInstanceHandler.class);
+
+
+ public static MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ switch (rpcType) {
+ case RpcType.ACK_VALUE:
+ return Ack.getDefaultInstance();
+ case RpcType.HANDSHAKE_VALUE:
+ return BitHandshake.getDefaultInstance();
+ case RpcType.RESP_FRAGMENT_HANDLE_VALUE:
+ return FragmentHandle.getDefaultInstance();
+ case RpcType.RESP_FRAGMENT_STATUS_VALUE:
+ return FragmentStatus.getDefaultInstance();
+ case RpcType.RESP_BIT_STATUS_VALUE:
+ return BitStatus.getDefaultInstance();
+
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
deleted file mode 100644
index b2c5cbb..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComHandler.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.rpc.bit;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.socket.SocketChannel;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.proto.ExecProtos.BitBatchChunk;
-import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
-import org.apache.drill.exec.proto.ExecProtos.BitStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.RpcType;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.rpc.Response;
-import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.bit.BitCom.TunnelListener;
-import org.apache.drill.exec.rpc.bit.BitComImpl.TunnelModifier;
-import org.apache.drill.exec.server.DrillbitContext;
-
-import com.google.protobuf.MessageLite;
-
-public class BitComHandler {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComHandler.class);
-
- private final TunnelModifier modifier;
-
- public BitComHandler(TunnelModifier modifier){
- this.modifier = modifier;
- }
-
- public TunnelListener getTunnelListener(RpcBus<?>.ChannelClosedHandler internalHandler){
- return new Listener(internalHandler);
- }
-
- public class Listener implements TunnelListener {
- final RpcBus<?>.ChannelClosedHandler internalHandler;
-
- public Listener(RpcBus<?>.ChannelClosedHandler internalHandler) {
- this.internalHandler = internalHandler;
- }
-
- @Override
- public void operationComplete(ChannelFuture future) throws Exception {
- logger.debug("BitTunnel closed, removing from BitCom.");
- internalHandler.operationComplete(future);
- BitTunnel t = modifier.remove(future.channel());
- if(t != null) t.shutdownIfClient();
- }
-
- @Override
- public void connectionEstablished(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus) {
- modifier.create(channel, endpoint, bus);
- }
-
- }
-
-
-
-
- public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- switch (rpcType) {
- case RpcType.ACK_VALUE:
- return Ack.getDefaultInstance();
- case RpcType.HANDSHAKE_VALUE:
- return BitHandshake.getDefaultInstance();
- case RpcType.RESP_FRAGMENT_HANDLE_VALUE:
- return FragmentHandle.getDefaultInstance();
- case RpcType.RESP_FRAGMENT_STATUS_VALUE:
- return FragmentStatus.getDefaultInstance();
- case RpcType.RESP_BIT_STATUS_VALUE:
- return BitStatus.getDefaultInstance();
- case RpcType.RESP_BATCH_CHUNK_VALUE:
- return BitBatchChunk.getDefaultInstance();
-
- default:
- throw new UnsupportedOperationException();
- }
- }
-
- protected Response handle(DrillbitContext context, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- switch (rpcType) {
-
- case RpcType.HANDSHAKE_VALUE:
- // parse incoming handshake.
- // get endpoint information.
- // record endpoint information in registry.
- // respond with our handshake info.
- return new Response(RpcType.HANDSHAKE, BitHandshake.getDefaultInstance(), null);
-
- case RpcType.REQ_BATCH_CHUNK_VALUE:
- return new Response(RpcType.RESP_BATCH_CHUNK, BitBatchChunk.getDefaultInstance(), null);
-
- case RpcType.REQ_BIT_STATUS_VALUE:
- return new Response(RpcType.RESP_BIT_STATUS, BitStatus.getDefaultInstance(), null);
-
- case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
- return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
-
- case RpcType.REQ_FRAGMENT_STATUS_VALUE:
- return new Response(RpcType.RESP_FRAGMENT_STATUS, FragmentStatus.getDefaultInstance(), null);
-
- case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
- return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
-
- case RpcType.REQ_RECORD_BATCH_VALUE:
- return new Response(RpcType.RESP_BATCH_CHUNK, BitBatchChunk.getDefaultInstance(), null);
-
- default:
- throw new UnsupportedOperationException();
- }
-
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
index aada154..c98be44 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitComImpl.java
@@ -17,138 +17,158 @@
******************************************************************************/
package org.apache.drill.exec.rpc.bit;
-import io.netty.channel.Channel;
-import io.netty.channel.socket.SocketChannel;
-import io.netty.util.concurrent.Future;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
-import java.util.Collection;
-import java.util.Map;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
import com.google.common.collect.Maps;
import com.google.common.io.Closeables;
-
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+
+/**
+ * Manages communication tunnels between nodes.
+ */
public class BitComImpl implements BitCom {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComImpl.class);
- private Map<DrillbitEndpoint, BitTunnel> tunnels = Maps.newConcurrentMap();
- private Map<SocketChannel, DrillbitEndpoint> endpoints = Maps.newConcurrentMap();
- private Object lock = new Object();
- private BitServer server;
- private DrillbitContext context;
+ private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
+ private final ListenerPool listeners;
+ private volatile BitServer server;
+ private final BitComHandler handler;
+ private final BootStrapContext context;
+
+ // TODO: this executor should be removed.
+ private final Executor exec = Executors.newCachedThreadPool(new NamedThreadFactory("BitComImpl execution pool: "));
- public BitComImpl(DrillbitContext context) {
+ public BitComImpl(BootStrapContext context, BitComHandler handler) {
+ super();
+ this.handler = handler;
this.context = context;
+ this.listeners = new ListenerPool(8);
}
public int start() throws InterruptedException, DrillbitStartupException {
- server = new BitServer(new BitComHandler(modifier), context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup(), context);
+ server = new BitServer(handler, context, registry, listeners);
int port = context.getConfig().getInt(ExecConstants.INITIAL_BIT_PORT);
return server.bind(port);
}
- private Future<BitTunnel> getNode(DrillbitEndpoint endpoint) {
- return null;
+ private CheckedFuture<BitConnection, RpcException> getNode(final DrillbitEndpoint endpoint, boolean check) {
+
+
+ SettableFuture<BitConnection> future = SettableFuture.create();
+ BitComFuture<BitConnection> checkedFuture = new BitComFuture<BitConnection>(future);
+ BitConnection t = null;
+
+ if (check) {
+ t = registry.get(endpoint);
+
+ if (t != null) {
+ future.set(t);
+ return checkedFuture;
+ }
+ }
-// BitTunnel t = tunnels.get(endpoint);
-// if (t == null) {
-// synchronized (lock) {
-// t = tunnels.get(endpoint);
-// if (t != null) return t;
-// BitClient c = new BitClient(new BitComHandler(modifier), context.getAllocator().getUnderlyingAllocator(),
-// context.getBitLoopGroup(), context);
-//
-// // need to figure what to do here with regards to waiting for handshake before returning. Probably need to add
-// // future registry so that new endpoint registration ping the registry.
-// throw new UnsupportedOperationException();
-// c.connectAsClient(endpoint.getAddress(), endpoint.getBitPort()).await();
-// t = new BitTunnel(c);
-// tunnels.put(endpoint, t);
-//
-// }
-// }
-// return null;
+ try {
+ AvailWatcher watcher = new AvailWatcher(future);
+ BitClient c = new BitClient(endpoint, watcher, handler, context, registry, listeners);
+ c.connect();
+ return checkedFuture;
+ } catch (InterruptedException | RpcException e) {
+ future.setException(new FragmentSetupException("Unable to open connection"));
+ return checkedFuture;
+ }
+
}
-
+ private class AvailWatcher implements AvailabilityListener{
+ final SettableFuture<BitConnection> future;
+
+ public AvailWatcher(SettableFuture<BitConnection> future) {
+ super();
+ this.future = future;
+ }
- @Override
- public DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, DrillbitEndpoint node,
- PlanFragment fragment) {
- return null;
+ @Override
+ public void isAvailable(BitConnection connection) {
+ future.set(connection);
+ }
+
+ }
+
+ BitConnection getConnection(DrillbitEndpoint endpoint) throws RpcException {
+ BitConnection t = registry.get(endpoint);
+ if(t != null) return t;
+ return this.getNode(endpoint, false).checkedGet();
}
- @Override
- public DrillRpcFuture<Ack> cancelFragment(FragmentContext context, DrillbitEndpoint node, FragmentHandle handle) {
- return null;
+
+ CheckedFuture<BitConnection, RpcException> getConnectionAsync(DrillbitEndpoint endpoint) {
+ return this.getNode(endpoint, true);
}
+
@Override
- public DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, DrillbitEndpoint node,
- FragmentHandle handle) {
- return null;
+ public BitTunnel getTunnel(DrillbitEndpoint endpoint){
+ BitConnection t = registry.get(endpoint);
+ if(t == null){
+ return new BitTunnel(exec, endpoint, this, t);
+ }else{
+ return new BitTunnel(exec, endpoint, this, this.getNode(endpoint, false));
+ }
}
- private final TunnelModifier modifier = new TunnelModifier();
/**
- * Fully synchronized modifier. Contention should be low since endpoints shouldn't be constantly changing.
+ * A future which remaps exceptions to a BitComException.
+ * @param <T>
*/
- class TunnelModifier {
- public BitTunnel remove(Channel ch) {
- synchronized (this) {
- DrillbitEndpoint endpoint = endpoints.remove(ch);
- if (endpoint == null) {
- logger
- .warn("We attempted to find a endpoint from a provided channel and found none. This suggests a race condition or memory leak problem.");
- return null;
- }
-
- BitTunnel tunnel = tunnels.remove(endpoint);
- return tunnel;
- }
+ private class BitComFuture<T> extends AbstractCheckedFuture<T, RpcException>{
+
+ protected BitComFuture(ListenableFuture<T> delegate) {
+ super(delegate);
}
- public void create(SocketChannel channel, DrillbitEndpoint endpoint, RpcBus<?> bus) {
- synchronized (this) {
- endpoints.put(channel, endpoint);
- tunnels.put(endpoint, new BitTunnel(bus));
+ @Override
+ protected RpcException mapException(Exception e) {
+ Throwable t = e;
+ if(e instanceof ExecutionException){
+ t = e.getCause();
}
+
+ if(t instanceof RpcException) return (RpcException) t;
+ return new RpcException(t);
}
}
public void close() {
Closeables.closeQuietly(server);
- for (BitTunnel bt : tunnels.values()) {
+ for (BitConnection bt : registry.values()) {
bt.shutdownIfClient();
}
}
-
@Override
- public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, DrillbitEndpoint node, RecordBatch batch) {
- return null;
- }
-
- @Override
- public RecordBatch getReceivingRecordBatchHandle(int majorFragmentId, int minorFragmentId) {
- return null;
- }
-
- @Override
- public void startQuery(Collection<DrillbitEndpoint> firstNodes, long queryId) {
+ public void registerIncomingBatchHandler(IncomingFragmentHandler handler) {
+ this.handler.registerIncomingFragmentHandler(handler);
}
+
+
}
Re: [13/13] git commit: Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.
Posted by Ted Dunning <te...@gmail.com>.
Sent from my iPhone
On May 13, 2013, at 18:52, jacques@apache.org wrote:
> Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.
>
>
> Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
> Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/e57a8d6d
> Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/e57a8d6d
> Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/e57a8d6d
>
> Branch: refs/heads/execwork
> Commit: e57a8d6d4ae282a79bc6e0a7354de992c391300f
> Parents: f0be80d
> Author: Jacques Nadeau <ja...@apache.org>
> Authored: Sun May 5 20:18:32 2013 -0700
> Committer: Jacques Nadeau <ja...@apache.org>
> Committed: Mon May 13 18:50:31 2013 -0700
>
> ----------------------------------------------------------------------
> sandbox/prototype/common/pom.xml | 12 +
> .../apache/drill/common/config/DrillConfig.java | 9 +-
> .../common/exceptions/ExecutionSetupException.java | 44 +++
> .../exceptions/PhysicalOperatorSetupException.java | 45 +++
> .../apache/drill/common/expression/SchemaPath.java | 2 +-
> .../drill/common/expression/types/DataType.java | 11 +
> .../apache/drill/common/optimize/Optimizer.java | 45 ---
> .../drill/common/physical/DataValidationMode.java | 24 --
> .../drill/common/physical/EndpointAffinity.java | 60 ----
> .../org/apache/drill/common/physical/FieldSet.java | 80 -----
> .../apache/drill/common/physical/OperatorCost.java | 61 ----
> .../apache/drill/common/physical/PhysicalPlan.java | 95 -----
> .../apache/drill/common/physical/ReadEntry.java | 25 --
> .../apache/drill/common/physical/RecordField.java | 60 ----
> .../apache/drill/common/physical/WriteEntry.java | 22 --
> .../apache/drill/common/physical/pop/Filter.java | 52 ---
> .../physical/pop/PartitionToRandomExchange.java | 92 -----
> .../apache/drill/common/physical/pop/Project.java | 53 ---
> .../apache/drill/common/physical/pop/Screen.java | 77 ----
> .../org/apache/drill/common/physical/pop/Sort.java | 57 ---
> .../common/physical/pop/base/AbstractBase.java | 56 ---
> .../common/physical/pop/base/AbstractExchange.java | 68 ----
> .../physical/pop/base/AbstractPhysicalVisitor.java | 80 -----
> .../common/physical/pop/base/AbstractReceiver.java | 32 --
> .../common/physical/pop/base/AbstractScan.java | 62 ----
> .../common/physical/pop/base/AbstractSender.java | 29 --
> .../common/physical/pop/base/AbstractSingle.java | 48 ---
> .../common/physical/pop/base/AbstractStore.java | 42 ---
> .../drill/common/physical/pop/base/Exchange.java | 69 ----
> .../common/physical/pop/base/ExchangeCost.java | 55 ---
> .../common/physical/pop/base/FragmentLeaf.java | 25 --
> .../common/physical/pop/base/FragmentRoot.java | 25 --
> .../common/physical/pop/base/HasAffinity.java | 26 --
> .../drill/common/physical/pop/base/Leaf.java | 21 --
> .../common/physical/pop/base/PhysicalOperator.java | 59 ---
> .../physical/pop/base/PhysicalOperatorUtil.java | 34 --
> .../common/physical/pop/base/PhysicalVisitor.java | 43 ---
> .../drill/common/physical/pop/base/Receiver.java | 38 --
> .../drill/common/physical/pop/base/Root.java | 24 --
> .../drill/common/physical/pop/base/Scan.java | 36 --
> .../drill/common/physical/pop/base/Sender.java | 29 --
> .../drill/common/physical/pop/base/Store.java | 30 --
> .../common/src/main/protobuf/Coordination.proto | 26 --
> .../apache/drill/common/physical/MockScanPOP.java | 69 ----
> .../apache/drill/common/physical/MockStorePOP.java | 62 ----
> .../drill/common/physical/ParsePhysicalPlan.java | 41 ---
> .../common/src/test/resources/drill-module.conf | 1 -
> .../common/src/test/resources/physical_test1.json | 33 --
> sandbox/prototype/exec/java-exec/pom.xml | 4 +-
> .../java/org/apache/drill/exec/ByteReorder.java | 54 ---
> .../apache/drill/exec/cache/DistributedCache.java | 14 +-
> .../org/apache/drill/exec/cache/HazelCache.java | 87 ++----
> .../org/apache/drill/exec/cache/LocalCache.java | 55 +++
> .../org/apache/drill/exec/cache/ProtoBufImpl.java | 50 +++
> .../org/apache/drill/exec/cache/ProtoBufWrap.java | 69 ++++
> .../java/org/apache/drill/exec/cache/ProtoMap.java | 52 +++
> .../drill/exec/cache/TemplatizedLogicalPlan.java | 22 --
> .../drill/exec/cache/TemplatizedPhysicalPlan.java | 22 --
> .../org/apache/drill/exec/client/DrillClient.java | 73 ++++-
> .../drill/exec/coord/ClusterCoordinator.java | 9 +-
> .../exec/coord/DrillServiceInstanceHelper.java | 4 +-
> .../drill/exec/coord/DrillbitEndpointSerDe.java | 65 ++++
> .../drill/exec/coord/LocalClusterCoordinator.java | 95 +++++
> .../drill/exec/coord/ZKClusterCoordinator.java | 30 ++-
> .../drill/exec/exception/BitComException.java | 45 +++
> .../exec/exception/ExecutionSetupException.java | 45 ---
> .../exec/exception/FragmentSetupException.java | 2 +
> .../apache/drill/exec/foreman/CancelableQuery.java | 22 --
> .../drill/exec/foreman/ExecutionPlanner.java | 24 --
> .../org/apache/drill/exec/foreman/Foreman.java | 39 --
> .../apache/drill/exec/foreman/QueryWorkUnit.java | 54 ---
> .../apache/drill/exec/foreman/ResourceRequest.java | 30 --
> .../apache/drill/exec/foreman/StatusProvider.java | 24 --
> .../apache/drill/exec/memory/BufferAllocator.java | 5 +-
> .../exec/metrics/SingleThreadNestedCounter.java | 22 +-
> .../exec/ops/FilteringRecordBatchTransformer.java | 58 ---
> .../org/apache/drill/exec/ops/FragmentContext.java | 54 +++-
> .../apache/drill/exec/ops/FragmentConverter.java | 30 --
> .../org/apache/drill/exec/ops/FragmentRoot.java | 37 --
> .../org/apache/drill/exec/ops/OperatorFactory.java | 22 --
> .../org/apache/drill/exec/ops/OutputMutator.java | 28 --
> .../org/apache/drill/exec/ops/QueryContext.java | 27 +-
> .../java/org/apache/drill/exec/ops/ScanBatch.java | 157 ---------
> .../drill/exec/ops/StreamingRecordBatch.java | 25 --
> .../exec/ops/exchange/ExchangeRecordBatch.java | 22 --
> .../drill/exec/ops/exchange/RecordBatchSender.java | 24 --
> .../drill/exec/ops/filter/FilterRecordBatch.java | 109 ------
> .../exec/ops/filter/SelectionVectorUpdater.java | 80 -----
> .../apache/drill/exec/opt/IdentityOptimizer.java | 3 +-
> .../java/org/apache/drill/exec/opt/Optimizer.java | 45 +++
> .../drill/exec/physical/DataValidationMode.java | 24 ++
> .../drill/exec/physical/EndpointAffinity.java | 60 ++++
> .../apache/drill/exec/physical/OperatorCost.java | 66 ++++
> .../apache/drill/exec/physical/PhysicalPlan.java | 94 +++++
> .../org/apache/drill/exec/physical/ReadEntry.java | 33 ++
> .../apache/drill/exec/physical/RecordField.java | 60 ++++
> .../org/apache/drill/exec/physical/WriteEntry.java | 22 ++
> .../drill/exec/physical/base/AbstractBase.java | 43 +++
> .../drill/exec/physical/base/AbstractExchange.java | 90 +++++
> .../physical/base/AbstractPhysicalVisitor.java | 124 +++++++
> .../drill/exec/physical/base/AbstractReceiver.java | 63 ++++
> .../drill/exec/physical/base/AbstractScan.java | 84 +++++
> .../drill/exec/physical/base/AbstractSender.java | 53 +++
> .../drill/exec/physical/base/AbstractSingle.java | 68 ++++
> .../drill/exec/physical/base/AbstractStore.java | 36 ++
> .../apache/drill/exec/physical/base/Exchange.java | 92 +++++
> .../drill/exec/physical/base/ExchangeCost.java | 68 ++++
> .../drill/exec/physical/base/FragmentLeaf.java | 25 ++
> .../drill/exec/physical/base/FragmentRoot.java | 25 ++
> .../drill/exec/physical/base/HasAffinity.java | 37 ++
> .../org/apache/drill/exec/physical/base/Leaf.java | 25 ++
> .../drill/exec/physical/base/PhysicalOperator.java | 80 +++++
> .../exec/physical/base/PhysicalOperatorUtil.java | 34 ++
> .../drill/exec/physical/base/PhysicalVisitor.java | 61 ++++
> .../apache/drill/exec/physical/base/Receiver.java | 51 +++
> .../org/apache/drill/exec/physical/base/Root.java | 24 ++
> .../org/apache/drill/exec/physical/base/Scan.java | 36 ++
> .../apache/drill/exec/physical/base/Sender.java | 44 +++
> .../org/apache/drill/exec/physical/base/Size.java | 48 +++
> .../org/apache/drill/exec/physical/base/Store.java | 74 ++++
> .../apache/drill/exec/physical/config/Filter.java | 75 ++++
> .../exec/physical/config/HashPartitionSender.java | 58 +++
> .../exec/physical/config/HashToRandomExchange.java | 86 +++++
> .../exec/physical/config/MockRecordReader.java | 113 ++++++
> .../exec/physical/config/MockScanBatchCreator.java | 49 +++
> .../drill/exec/physical/config/MockScanPOP.java | 193 ++++++++++
> .../exec/physical/config/MockStorageEngine.java | 58 +++
> .../drill/exec/physical/config/MockStorePOP.java | 75 ++++
> .../drill/exec/physical/config/PartitionRange.java | 47 +++
> .../apache/drill/exec/physical/config/Project.java | 72 ++++
> .../drill/exec/physical/config/RandomReceiver.java | 83 +++++
> .../drill/exec/physical/config/RangeSender.java | 72 ++++
> .../apache/drill/exec/physical/config/Screen.java | 106 ++++++
> .../drill/exec/physical/config/SingleSender.java | 78 ++++
> .../apache/drill/exec/physical/config/Sort.java | 86 +++++
> .../drill/exec/physical/config/UnionExchange.java | 79 +++++
> .../drill/exec/physical/impl/BatchCreator.java | 31 ++
> .../exec/physical/impl/FilterRecordBatch.java | 108 ++++++
> .../impl/FilteringRecordBatchTransformer.java | 58 +++
> .../drill/exec/physical/impl/ImplCreator.java | 102 ++++++
> .../drill/exec/physical/impl/OutputMutator.java | 28 ++
> .../drill/exec/physical/impl/PhysicalConfig.java | 29 ++
> .../exec/physical/impl/RandomReceiverCreator.java | 46 +++
> .../drill/exec/physical/impl/RootCreator.java | 31 ++
> .../apache/drill/exec/physical/impl/RootExec.java | 40 +++
> .../apache/drill/exec/physical/impl/ScanBatch.java | 172 +++++++++
> .../drill/exec/physical/impl/ScreenCreator.java | 90 +++++
> .../exec/physical/impl/SingleSenderCreator.java | 89 +++++
> .../drill/exec/physical/impl/WireRecordBatch.java | 99 ++++++
> .../impl/materialize/QueryWritableBatch.java | 46 +++
> .../impl/materialize/RecordMaterializer.java | 31 ++
> .../impl/materialize/VectorRecordMaterializer.java | 52 +++
> .../exec/planner/AbstractOpWrapperVisitor.java | 45 +++
> .../org/apache/drill/exec/planner/ExecPlanner.java | 8 +-
> .../drill/exec/planner/FragmentMaterializer.java | 86 -----
> .../apache/drill/exec/planner/FragmentNode.java | 138 --------
> .../drill/exec/planner/FragmentPlanningSet.java | 61 ----
> .../drill/exec/planner/FragmentRunnable.java | 124 -------
> .../drill/exec/planner/FragmentScheduler.java | 32 --
> .../apache/drill/exec/planner/FragmentStats.java | 63 ----
> .../drill/exec/planner/FragmentStatsCollector.java | 109 ------
> .../apache/drill/exec/planner/FragmentVisitor.java | 22 --
> .../apache/drill/exec/planner/FragmentWrapper.java | 127 -------
> .../exec/planner/FragmentingPhysicalVisitor.java | 71 ----
> .../drill/exec/planner/MaterializedFragment.java | 69 ----
> .../drill/exec/planner/PhysicalPlanReader.java | 63 +++-
> .../org/apache/drill/exec/planner/ScanFinder.java | 54 ---
> .../drill/exec/planner/SimpleExecPlanner.java | 28 +-
> .../drill/exec/planner/SimpleParallelizer.java | 147 --------
> .../drill/exec/planner/fragment/Fragment.java | 150 ++++++++
> .../exec/planner/fragment/FragmentVisitor.java | 23 ++
> .../planner/fragment/MakeFragmentsVisitor.java | 69 ++++
> .../planner/fragment/MaterializedFragment.java | 69 ++++
> .../drill/exec/planner/fragment/Materializer.java | 107 ++++++
> .../drill/exec/planner/fragment/PlanningSet.java | 66 ++++
> .../exec/planner/fragment/SimpleParallelizer.java | 163 +++++++++
> .../apache/drill/exec/planner/fragment/Stats.java | 70 ++++
> .../exec/planner/fragment/StatsCollector.java | 106 ++++++
> .../drill/exec/planner/fragment/Wrapper.java | 186 ++++++++++
> .../exec/pop/receiver/NWayOrderingReceiver.java | 52 ---
> .../drill/exec/pop/receiver/RandomReceiver.java | 55 ---
> .../drill/exec/pop/sender/HashPartitionSender.java | 49 ---
> .../org/apache/drill/exec/record/BatchSchema.java | 102 +-----
> .../drill/exec/record/FragmentWritableBatch.java | 59 +++
> .../drill/exec/record/InvalidValueAccessor.java | 2 +-
> .../apache/drill/exec/record/MajorTypeSerDe.java | 116 ++++++
> .../drill/exec/record/MaterializedField.java | 167 +++++++---
> .../apache/drill/exec/record/RawFragmentBatch.java | 44 +++
> .../exec/record/RawFragmentBatchProvider.java | 27 ++
> .../org/apache/drill/exec/record/RecordBatch.java | 9 +-
> .../drill/exec/record/RecordBatchLoader.java | 143 ++++++++
> .../apache/drill/exec/record/SchemaBuilder.java | 127 +++++++
> .../apache/drill/exec/record/WritableBatch.java | 108 ++++++
> .../record/vector/AbstractFixedValueVector.java | 21 +-
> .../drill/exec/record/vector/BaseValueVector.java | 86 ++++-
> .../org/apache/drill/exec/record/vector/Bit.java | 168 +++++++++
> .../apache/drill/exec/record/vector/BitVector.java | 166 ---------
> .../drill/exec/record/vector/ByteVector.java | 48 ---
> .../apache/drill/exec/record/vector/Fixed1.java | 43 +++
> .../apache/drill/exec/record/vector/Fixed12.java | 35 ++
> .../apache/drill/exec/record/vector/Fixed16.java | 37 ++
> .../apache/drill/exec/record/vector/Fixed2.java | 53 +++
> .../apache/drill/exec/record/vector/Fixed4.java | 55 +++
> .../apache/drill/exec/record/vector/Fixed8.java | 58 +++
> .../apache/drill/exec/record/vector/FixedLen.java | 45 +++
> .../drill/exec/record/vector/Int16Vector.java | 52 ---
> .../drill/exec/record/vector/Int32Vector.java | 52 ---
> .../drill/exec/record/vector/NullableFixed4.java | 37 ++
> .../exec/record/vector/NullableInt32Vector.java | 47 ---
> .../exec/record/vector/NullableValueVector.java | 36 ++-
> .../apache/drill/exec/record/vector/RepeatMap.java | 57 +++
> .../drill/exec/record/vector/SelectionVector.java | 10 +-
> .../drill/exec/record/vector/TypeHelper.java | 250 +++++++++++++
> .../drill/exec/record/vector/UInt16Vector.java | 51 ---
> .../drill/exec/record/vector/ValueVector.java | 46 +++-
> .../apache/drill/exec/record/vector/VarLen1.java | 36 ++
> .../apache/drill/exec/record/vector/VarLen2.java | 36 ++
> .../apache/drill/exec/record/vector/VarLen4.java | 36 ++
> .../drill/exec/record/vector/VariableVector.java | 35 ++-
> .../drill/exec/rpc/AbstractHandshakeHandler.java | 57 +++
> .../main/java/org/apache/drill/exec/rpc/Acks.java | 27 ++
> .../org/apache/drill/exec/rpc/BasicClient.java | 137 +++++++-
> .../drill/exec/rpc/BasicClientWithConnection.java | 64 ++++
> .../org/apache/drill/exec/rpc/BasicServer.java | 61 +++-
> .../apache/drill/exec/rpc/CoordinationQueue.java | 32 +-
> .../org/apache/drill/exec/rpc/DrillRpcFuture.java | 78 +----
> .../apache/drill/exec/rpc/DrillRpcFutureImpl.java | 118 +++++++
> .../apache/drill/exec/rpc/InboundRpcMessage.java | 13 +-
> .../apache/drill/exec/rpc/OutboundRpcMessage.java | 41 ++-
> .../apache/drill/exec/rpc/RemoteConnection.java | 45 +++
> .../java/org/apache/drill/exec/rpc/Response.java | 8 +-
> .../java/org/apache/drill/exec/rpc/RpcBus.java | 136 +++++---
> .../java/org/apache/drill/exec/rpc/RpcConfig.java | 150 ++++++++
> .../java/org/apache/drill/exec/rpc/RpcDecoder.java | 14 +-
> .../java/org/apache/drill/exec/rpc/RpcEncoder.java | 15 +-
> .../apache/drill/exec/rpc/RpcExceptionHandler.java | 21 +-
> .../java/org/apache/drill/exec/rpc/RpcMessage.java | 9 +-
> .../apache/drill/exec/rpc/RpcOutcomeListener.java | 28 ++
> .../exec/rpc/ZeroCopyProtobufLengthDecoder.java | 16 +-
> .../drill/exec/rpc/bit/AvailabilityListener.java | 22 ++
> .../org/apache/drill/exec/rpc/bit/BitClient.java | 77 ++++-
> .../java/org/apa
[13/13] git commit: Update typing system. Update RPC system. Add
Fragmenting Implementation. Working single node. Distributed failing due to
threading issues.
Posted by ja...@apache.org.
Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.
Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/e57a8d6d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/e57a8d6d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/e57a8d6d
Branch: refs/heads/execwork
Commit: e57a8d6d4ae282a79bc6e0a7354de992c391300f
Parents: f0be80d
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun May 5 20:18:32 2013 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon May 13 18:50:31 2013 -0700
----------------------------------------------------------------------
sandbox/prototype/common/pom.xml | 12 +
.../apache/drill/common/config/DrillConfig.java | 9 +-
.../common/exceptions/ExecutionSetupException.java | 44 +++
.../exceptions/PhysicalOperatorSetupException.java | 45 +++
.../apache/drill/common/expression/SchemaPath.java | 2 +-
.../drill/common/expression/types/DataType.java | 11 +
.../apache/drill/common/optimize/Optimizer.java | 45 ---
.../drill/common/physical/DataValidationMode.java | 24 --
.../drill/common/physical/EndpointAffinity.java | 60 ----
.../org/apache/drill/common/physical/FieldSet.java | 80 -----
.../apache/drill/common/physical/OperatorCost.java | 61 ----
.../apache/drill/common/physical/PhysicalPlan.java | 95 -----
.../apache/drill/common/physical/ReadEntry.java | 25 --
.../apache/drill/common/physical/RecordField.java | 60 ----
.../apache/drill/common/physical/WriteEntry.java | 22 --
.../apache/drill/common/physical/pop/Filter.java | 52 ---
.../physical/pop/PartitionToRandomExchange.java | 92 -----
.../apache/drill/common/physical/pop/Project.java | 53 ---
.../apache/drill/common/physical/pop/Screen.java | 77 ----
.../org/apache/drill/common/physical/pop/Sort.java | 57 ---
.../common/physical/pop/base/AbstractBase.java | 56 ---
.../common/physical/pop/base/AbstractExchange.java | 68 ----
.../physical/pop/base/AbstractPhysicalVisitor.java | 80 -----
.../common/physical/pop/base/AbstractReceiver.java | 32 --
.../common/physical/pop/base/AbstractScan.java | 62 ----
.../common/physical/pop/base/AbstractSender.java | 29 --
.../common/physical/pop/base/AbstractSingle.java | 48 ---
.../common/physical/pop/base/AbstractStore.java | 42 ---
.../drill/common/physical/pop/base/Exchange.java | 69 ----
.../common/physical/pop/base/ExchangeCost.java | 55 ---
.../common/physical/pop/base/FragmentLeaf.java | 25 --
.../common/physical/pop/base/FragmentRoot.java | 25 --
.../common/physical/pop/base/HasAffinity.java | 26 --
.../drill/common/physical/pop/base/Leaf.java | 21 --
.../common/physical/pop/base/PhysicalOperator.java | 59 ---
.../physical/pop/base/PhysicalOperatorUtil.java | 34 --
.../common/physical/pop/base/PhysicalVisitor.java | 43 ---
.../drill/common/physical/pop/base/Receiver.java | 38 --
.../drill/common/physical/pop/base/Root.java | 24 --
.../drill/common/physical/pop/base/Scan.java | 36 --
.../drill/common/physical/pop/base/Sender.java | 29 --
.../drill/common/physical/pop/base/Store.java | 30 --
.../common/src/main/protobuf/Coordination.proto | 26 --
.../apache/drill/common/physical/MockScanPOP.java | 69 ----
.../apache/drill/common/physical/MockStorePOP.java | 62 ----
.../drill/common/physical/ParsePhysicalPlan.java | 41 ---
.../common/src/test/resources/drill-module.conf | 1 -
.../common/src/test/resources/physical_test1.json | 33 --
sandbox/prototype/exec/java-exec/pom.xml | 4 +-
.../java/org/apache/drill/exec/ByteReorder.java | 54 ---
.../apache/drill/exec/cache/DistributedCache.java | 14 +-
.../org/apache/drill/exec/cache/HazelCache.java | 87 ++----
.../org/apache/drill/exec/cache/LocalCache.java | 55 +++
.../org/apache/drill/exec/cache/ProtoBufImpl.java | 50 +++
.../org/apache/drill/exec/cache/ProtoBufWrap.java | 69 ++++
.../java/org/apache/drill/exec/cache/ProtoMap.java | 52 +++
.../drill/exec/cache/TemplatizedLogicalPlan.java | 22 --
.../drill/exec/cache/TemplatizedPhysicalPlan.java | 22 --
.../org/apache/drill/exec/client/DrillClient.java | 73 ++++-
.../drill/exec/coord/ClusterCoordinator.java | 9 +-
.../exec/coord/DrillServiceInstanceHelper.java | 4 +-
.../drill/exec/coord/DrillbitEndpointSerDe.java | 65 ++++
.../drill/exec/coord/LocalClusterCoordinator.java | 95 +++++
.../drill/exec/coord/ZKClusterCoordinator.java | 30 ++-
.../drill/exec/exception/BitComException.java | 45 +++
.../exec/exception/ExecutionSetupException.java | 45 ---
.../exec/exception/FragmentSetupException.java | 2 +
.../apache/drill/exec/foreman/CancelableQuery.java | 22 --
.../drill/exec/foreman/ExecutionPlanner.java | 24 --
.../org/apache/drill/exec/foreman/Foreman.java | 39 --
.../apache/drill/exec/foreman/QueryWorkUnit.java | 54 ---
.../apache/drill/exec/foreman/ResourceRequest.java | 30 --
.../apache/drill/exec/foreman/StatusProvider.java | 24 --
.../apache/drill/exec/memory/BufferAllocator.java | 5 +-
.../exec/metrics/SingleThreadNestedCounter.java | 22 +-
.../exec/ops/FilteringRecordBatchTransformer.java | 58 ---
.../org/apache/drill/exec/ops/FragmentContext.java | 54 +++-
.../apache/drill/exec/ops/FragmentConverter.java | 30 --
.../org/apache/drill/exec/ops/FragmentRoot.java | 37 --
.../org/apache/drill/exec/ops/OperatorFactory.java | 22 --
.../org/apache/drill/exec/ops/OutputMutator.java | 28 --
.../org/apache/drill/exec/ops/QueryContext.java | 27 +-
.../java/org/apache/drill/exec/ops/ScanBatch.java | 157 ---------
.../drill/exec/ops/StreamingRecordBatch.java | 25 --
.../exec/ops/exchange/ExchangeRecordBatch.java | 22 --
.../drill/exec/ops/exchange/RecordBatchSender.java | 24 --
.../drill/exec/ops/filter/FilterRecordBatch.java | 109 ------
.../exec/ops/filter/SelectionVectorUpdater.java | 80 -----
.../apache/drill/exec/opt/IdentityOptimizer.java | 3 +-
.../java/org/apache/drill/exec/opt/Optimizer.java | 45 +++
.../drill/exec/physical/DataValidationMode.java | 24 ++
.../drill/exec/physical/EndpointAffinity.java | 60 ++++
.../apache/drill/exec/physical/OperatorCost.java | 66 ++++
.../apache/drill/exec/physical/PhysicalPlan.java | 94 +++++
.../org/apache/drill/exec/physical/ReadEntry.java | 33 ++
.../apache/drill/exec/physical/RecordField.java | 60 ++++
.../org/apache/drill/exec/physical/WriteEntry.java | 22 ++
.../drill/exec/physical/base/AbstractBase.java | 43 +++
.../drill/exec/physical/base/AbstractExchange.java | 90 +++++
.../physical/base/AbstractPhysicalVisitor.java | 124 +++++++
.../drill/exec/physical/base/AbstractReceiver.java | 63 ++++
.../drill/exec/physical/base/AbstractScan.java | 84 +++++
.../drill/exec/physical/base/AbstractSender.java | 53 +++
.../drill/exec/physical/base/AbstractSingle.java | 68 ++++
.../drill/exec/physical/base/AbstractStore.java | 36 ++
.../apache/drill/exec/physical/base/Exchange.java | 92 +++++
.../drill/exec/physical/base/ExchangeCost.java | 68 ++++
.../drill/exec/physical/base/FragmentLeaf.java | 25 ++
.../drill/exec/physical/base/FragmentRoot.java | 25 ++
.../drill/exec/physical/base/HasAffinity.java | 37 ++
.../org/apache/drill/exec/physical/base/Leaf.java | 25 ++
.../drill/exec/physical/base/PhysicalOperator.java | 80 +++++
.../exec/physical/base/PhysicalOperatorUtil.java | 34 ++
.../drill/exec/physical/base/PhysicalVisitor.java | 61 ++++
.../apache/drill/exec/physical/base/Receiver.java | 51 +++
.../org/apache/drill/exec/physical/base/Root.java | 24 ++
.../org/apache/drill/exec/physical/base/Scan.java | 36 ++
.../apache/drill/exec/physical/base/Sender.java | 44 +++
.../org/apache/drill/exec/physical/base/Size.java | 48 +++
.../org/apache/drill/exec/physical/base/Store.java | 74 ++++
.../apache/drill/exec/physical/config/Filter.java | 75 ++++
.../exec/physical/config/HashPartitionSender.java | 58 +++
.../exec/physical/config/HashToRandomExchange.java | 86 +++++
.../exec/physical/config/MockRecordReader.java | 113 ++++++
.../exec/physical/config/MockScanBatchCreator.java | 49 +++
.../drill/exec/physical/config/MockScanPOP.java | 193 ++++++++++
.../exec/physical/config/MockStorageEngine.java | 58 +++
.../drill/exec/physical/config/MockStorePOP.java | 75 ++++
.../drill/exec/physical/config/PartitionRange.java | 47 +++
.../apache/drill/exec/physical/config/Project.java | 72 ++++
.../drill/exec/physical/config/RandomReceiver.java | 83 +++++
.../drill/exec/physical/config/RangeSender.java | 72 ++++
.../apache/drill/exec/physical/config/Screen.java | 106 ++++++
.../drill/exec/physical/config/SingleSender.java | 78 ++++
.../apache/drill/exec/physical/config/Sort.java | 86 +++++
.../drill/exec/physical/config/UnionExchange.java | 79 +++++
.../drill/exec/physical/impl/BatchCreator.java | 31 ++
.../exec/physical/impl/FilterRecordBatch.java | 108 ++++++
.../impl/FilteringRecordBatchTransformer.java | 58 +++
.../drill/exec/physical/impl/ImplCreator.java | 102 ++++++
.../drill/exec/physical/impl/OutputMutator.java | 28 ++
.../drill/exec/physical/impl/PhysicalConfig.java | 29 ++
.../exec/physical/impl/RandomReceiverCreator.java | 46 +++
.../drill/exec/physical/impl/RootCreator.java | 31 ++
.../apache/drill/exec/physical/impl/RootExec.java | 40 +++
.../apache/drill/exec/physical/impl/ScanBatch.java | 172 +++++++++
.../drill/exec/physical/impl/ScreenCreator.java | 90 +++++
.../exec/physical/impl/SingleSenderCreator.java | 89 +++++
.../drill/exec/physical/impl/WireRecordBatch.java | 99 ++++++
.../impl/materialize/QueryWritableBatch.java | 46 +++
.../impl/materialize/RecordMaterializer.java | 31 ++
.../impl/materialize/VectorRecordMaterializer.java | 52 +++
.../exec/planner/AbstractOpWrapperVisitor.java | 45 +++
.../org/apache/drill/exec/planner/ExecPlanner.java | 8 +-
.../drill/exec/planner/FragmentMaterializer.java | 86 -----
.../apache/drill/exec/planner/FragmentNode.java | 138 --------
.../drill/exec/planner/FragmentPlanningSet.java | 61 ----
.../drill/exec/planner/FragmentRunnable.java | 124 -------
.../drill/exec/planner/FragmentScheduler.java | 32 --
.../apache/drill/exec/planner/FragmentStats.java | 63 ----
.../drill/exec/planner/FragmentStatsCollector.java | 109 ------
.../apache/drill/exec/planner/FragmentVisitor.java | 22 --
.../apache/drill/exec/planner/FragmentWrapper.java | 127 -------
.../exec/planner/FragmentingPhysicalVisitor.java | 71 ----
.../drill/exec/planner/MaterializedFragment.java | 69 ----
.../drill/exec/planner/PhysicalPlanReader.java | 63 +++-
.../org/apache/drill/exec/planner/ScanFinder.java | 54 ---
.../drill/exec/planner/SimpleExecPlanner.java | 28 +-
.../drill/exec/planner/SimpleParallelizer.java | 147 --------
.../drill/exec/planner/fragment/Fragment.java | 150 ++++++++
.../exec/planner/fragment/FragmentVisitor.java | 23 ++
.../planner/fragment/MakeFragmentsVisitor.java | 69 ++++
.../planner/fragment/MaterializedFragment.java | 69 ++++
.../drill/exec/planner/fragment/Materializer.java | 107 ++++++
.../drill/exec/planner/fragment/PlanningSet.java | 66 ++++
.../exec/planner/fragment/SimpleParallelizer.java | 163 +++++++++
.../apache/drill/exec/planner/fragment/Stats.java | 70 ++++
.../exec/planner/fragment/StatsCollector.java | 106 ++++++
.../drill/exec/planner/fragment/Wrapper.java | 186 ++++++++++
.../exec/pop/receiver/NWayOrderingReceiver.java | 52 ---
.../drill/exec/pop/receiver/RandomReceiver.java | 55 ---
.../drill/exec/pop/sender/HashPartitionSender.java | 49 ---
.../org/apache/drill/exec/record/BatchSchema.java | 102 +-----
.../drill/exec/record/FragmentWritableBatch.java | 59 +++
.../drill/exec/record/InvalidValueAccessor.java | 2 +-
.../apache/drill/exec/record/MajorTypeSerDe.java | 116 ++++++
.../drill/exec/record/MaterializedField.java | 167 +++++++---
.../apache/drill/exec/record/RawFragmentBatch.java | 44 +++
.../exec/record/RawFragmentBatchProvider.java | 27 ++
.../org/apache/drill/exec/record/RecordBatch.java | 9 +-
.../drill/exec/record/RecordBatchLoader.java | 143 ++++++++
.../apache/drill/exec/record/SchemaBuilder.java | 127 +++++++
.../apache/drill/exec/record/WritableBatch.java | 108 ++++++
.../record/vector/AbstractFixedValueVector.java | 21 +-
.../drill/exec/record/vector/BaseValueVector.java | 86 ++++-
.../org/apache/drill/exec/record/vector/Bit.java | 168 +++++++++
.../apache/drill/exec/record/vector/BitVector.java | 166 ---------
.../drill/exec/record/vector/ByteVector.java | 48 ---
.../apache/drill/exec/record/vector/Fixed1.java | 43 +++
.../apache/drill/exec/record/vector/Fixed12.java | 35 ++
.../apache/drill/exec/record/vector/Fixed16.java | 37 ++
.../apache/drill/exec/record/vector/Fixed2.java | 53 +++
.../apache/drill/exec/record/vector/Fixed4.java | 55 +++
.../apache/drill/exec/record/vector/Fixed8.java | 58 +++
.../apache/drill/exec/record/vector/FixedLen.java | 45 +++
.../drill/exec/record/vector/Int16Vector.java | 52 ---
.../drill/exec/record/vector/Int32Vector.java | 52 ---
.../drill/exec/record/vector/NullableFixed4.java | 37 ++
.../exec/record/vector/NullableInt32Vector.java | 47 ---
.../exec/record/vector/NullableValueVector.java | 36 ++-
.../apache/drill/exec/record/vector/RepeatMap.java | 57 +++
.../drill/exec/record/vector/SelectionVector.java | 10 +-
.../drill/exec/record/vector/TypeHelper.java | 250 +++++++++++++
.../drill/exec/record/vector/UInt16Vector.java | 51 ---
.../drill/exec/record/vector/ValueVector.java | 46 +++-
.../apache/drill/exec/record/vector/VarLen1.java | 36 ++
.../apache/drill/exec/record/vector/VarLen2.java | 36 ++
.../apache/drill/exec/record/vector/VarLen4.java | 36 ++
.../drill/exec/record/vector/VariableVector.java | 35 ++-
.../drill/exec/rpc/AbstractHandshakeHandler.java | 57 +++
.../main/java/org/apache/drill/exec/rpc/Acks.java | 27 ++
.../org/apache/drill/exec/rpc/BasicClient.java | 137 +++++++-
.../drill/exec/rpc/BasicClientWithConnection.java | 64 ++++
.../org/apache/drill/exec/rpc/BasicServer.java | 61 +++-
.../apache/drill/exec/rpc/CoordinationQueue.java | 32 +-
.../org/apache/drill/exec/rpc/DrillRpcFuture.java | 78 +----
.../apache/drill/exec/rpc/DrillRpcFutureImpl.java | 118 +++++++
.../apache/drill/exec/rpc/InboundRpcMessage.java | 13 +-
.../apache/drill/exec/rpc/OutboundRpcMessage.java | 41 ++-
.../apache/drill/exec/rpc/RemoteConnection.java | 45 +++
.../java/org/apache/drill/exec/rpc/Response.java | 8 +-
.../java/org/apache/drill/exec/rpc/RpcBus.java | 136 +++++---
.../java/org/apache/drill/exec/rpc/RpcConfig.java | 150 ++++++++
.../java/org/apache/drill/exec/rpc/RpcDecoder.java | 14 +-
.../java/org/apache/drill/exec/rpc/RpcEncoder.java | 15 +-
.../apache/drill/exec/rpc/RpcExceptionHandler.java | 21 +-
.../java/org/apache/drill/exec/rpc/RpcMessage.java | 9 +-
.../apache/drill/exec/rpc/RpcOutcomeListener.java | 28 ++
.../exec/rpc/ZeroCopyProtobufLengthDecoder.java | 16 +-
.../drill/exec/rpc/bit/AvailabilityListener.java | 22 ++
.../org/apache/drill/exec/rpc/bit/BitClient.java | 77 ++++-
.../java/org/apache/drill/exec/rpc/bit/BitCom.java | 75 +---
.../exec/rpc/bit/BitComDefaultInstanceHandler.java | 51 +++
.../apache/drill/exec/rpc/bit/BitComHandler.java | 136 -------
.../org/apache/drill/exec/rpc/bit/BitComImpl.java | 194 ++++++-----
.../apache/drill/exec/rpc/bit/BitConnection.java | 168 +++++++++
.../drill/exec/rpc/bit/BitConnectionManager.java | 80 +++++
.../apache/drill/exec/rpc/bit/BitRpcConfig.java | 46 +++
.../org/apache/drill/exec/rpc/bit/BitServer.java | 61 +++-
.../org/apache/drill/exec/rpc/bit/BitTunnel.java | 215 +++++++++++-
.../apache/drill/exec/rpc/bit/ListenerPool.java | 56 +++
.../drill/exec/rpc/user/QueryResultBatch.java | 49 +++
.../org/apache/drill/exec/rpc/user/UserClient.java | 169 ++++++++-
.../drill/exec/rpc/user/UserResultsListener.java | 41 +++
.../apache/drill/exec/rpc/user/UserRpcConfig.java | 39 ++
.../org/apache/drill/exec/rpc/user/UserServer.java | 93 ++++--
.../apache/drill/exec/server/BootStrapContext.java | 68 ++++
.../org/apache/drill/exec/server/Drillbit.java | 65 ++--
.../apache/drill/exec/server/DrillbitContext.java | 58 +++-
.../apache/drill/exec/server/RemoteServiceSet.java | 59 +++
.../apache/drill/exec/service/ServiceEngine.java | 45 ++--
.../drill/exec/store/AbstractStorageEngine.java | 2 +-
.../org/apache/drill/exec/store/RecordReader.java | 7 +-
.../org/apache/drill/exec/store/StorageEngine.java | 2 +-
.../org/apache/drill/exec/util/AtomicState.java | 58 +++
.../exec/work/AbstractFragmentRunnerListener.java | 109 ++++++
.../apache/drill/exec/work/CancelableQuery.java | 22 ++
.../apache/drill/exec/work/EndpointListener.java | 45 +++
.../org/apache/drill/exec/work/FragmentRunner.java | 124 +++++++
.../drill/exec/work/FragmentRunnerListener.java | 28 ++
.../org/apache/drill/exec/work/QueryWorkUnit.java | 64 ++++
.../apache/drill/exec/work/RecordOutputStream.java | 22 ++
.../exec/work/RemotingFragmentRunnerListener.java | 46 +++
.../apache/drill/exec/work/ResourceRequest.java | 30 ++
.../org/apache/drill/exec/work/RootNodeDriver.java | 25 ++
.../org/apache/drill/exec/work/StatusProvider.java | 24 ++
.../org/apache/drill/exec/work/WorkManager.java | 168 +++++++++
.../exec/work/batch/AbstractFragmentCollector.java | 84 +++++
.../drill/exec/work/batch/BatchCollector.java | 32 ++
.../drill/exec/work/batch/BitComHandler.java | 41 +++
.../drill/exec/work/batch/BitComHandlerImpl.java | 205 +++++++++++
.../drill/exec/work/batch/IncomingBuffers.java | 108 ++++++
.../drill/exec/work/batch/MergingCollector.java | 45 +++
.../exec/work/batch/PartitionedCollector.java | 42 +++
.../drill/exec/work/batch/RawBatchBuffer.java | 33 ++
.../exec/work/batch/UnlmitedRawBatchBuffer.java | 73 ++++
.../drill/exec/work/foreman/ErrorHelper.java | 47 +++
.../apache/drill/exec/work/foreman/Foreman.java | 272 +++++++++++++++
.../exec/work/foreman/FragmentStatusListener.java | 26 ++
.../exec/work/foreman/RunningFragmentManager.java | 266 ++++++++++++++
.../drill/exec/work/foreman/TunnelManager.java | 53 +++
.../work/fragment/IncomingFragmentHandler.java | 49 +++
.../exec/work/fragment/LocalFragmentHandler.java | 69 ++++
.../exec/work/fragment/RemoteFragmentHandler.java | 123 +++++++
.../apache/drill/exec/work/user/UserWorker.java | 72 ++++
.../java-exec/src/main/protobuf/Coordination.proto | 26 ++
.../src/main/protobuf/ExecutionProtos.proto | 55 ++--
.../java-exec/src/main/protobuf/GeneralRPC.proto | 6 +-
.../java-exec/src/main/protobuf/SchemaDef.proto | 86 ++++--
.../exec/java-exec/src/main/protobuf/User.proto | 69 ++--
.../src/main/protobuf/UserBitShared.proto | 46 +++
.../org/apache/drill/exec/DrillSystemTestBase.java | 7 +
.../drill/exec/client/DrillClientSystemTest.java | 25 +-
.../exec/compile/TestClassCompilationTypes.java | 5 +-
.../exec/physical/config/ParsePhysicalPlan.java | 42 +++
.../exec/physical/impl/DistributedFragmentRun.java | 53 +++
.../exec/physical/impl/SimpleFragmentRun.java | 100 ++++++
.../org/apache/drill/exec/pop/CheckFragmenter.java | 70 ++--
.../apache/drill/exec/pop/CheckInjectionValue.java | 12 +-
.../org/apache/drill/exec/pop/FragmentChecker.java | 66 ++++
.../org/apache/drill/exec/pop/PopUnitTestBase.java | 71 ++++
.../apache/drill/exec/rpc/user/RunRemoteQuery.java | 41 ---
.../apache/drill/exec/rpc/user/UserRpcTest.java | 107 ------
.../org/apache/drill/exec/server/TestBitRpc.java | 84 +++++
.../apache/drill/exec/store/MockRecordConfig.java | 46 ---
.../apache/drill/exec/store/MockRecordReader.java | 108 ------
.../apache/drill/exec/store/MockStorageEngine.java | 54 ---
.../java-exec/src/test/resources/drill-module.conf | 5 +-
.../exec/java-exec/src/test/resources/logback.xml | 10 +-
.../src/test/resources/physical_screen.json | 5 +-
.../test/resources/physical_simpleexchange.json | 36 ++-
.../test/resources/physical_single_exchange.json | 34 ++
.../src/test/resources/physical_test1.json | 40 +++
.../src/test/resources/physical_test2.json | 34 ++
.../exec/ref/src/test/resources/donuts.json | 2 +-
sandbox/prototype/pom.xml | 2 +-
326 files changed, 13371 insertions(+), 6094 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/pom.xml b/sandbox/prototype/common/pom.xml
index 3a15c05..beb2d28 100644
--- a/sandbox/prototype/common/pom.xml
+++ b/sandbox/prototype/common/pom.xml
@@ -74,6 +74,18 @@
<build>
<plugins>
+ <plugin>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>test-jar</id>
+ <phase>package</phase>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
<plugin>
<groupId>org.antlr</groupId>
<artifactId>antlr3-maven-plugin</artifactId>
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 2b8f45d..18d5e51 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -27,9 +27,9 @@ import org.apache.drill.common.exceptions.DrillConfigurationException;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.common.logical.StorageEngineConfigBase;
import org.apache.drill.common.logical.data.LogicalOperatorBase;
-import org.apache.drill.common.physical.pop.base.PhysicalOperatorUtil;
import org.apache.drill.common.util.PathScanner;
+import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.core.JsonParser.Feature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.SerializationFeature;
@@ -50,15 +50,18 @@ public final class DrillConfig extends NestedConfig{
public DrillConfig(Config config) {
super(config);
mapper = new ObjectMapper();
- SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule").addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+ SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule")
+ .addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
+
mapper.registerModule(deserModule);
mapper.enable(SerializationFeature.INDENT_OUTPUT);
mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+ mapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, false);
mapper.configure(Feature.ALLOW_COMMENTS, true);
mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
- mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(this));
mapper.registerSubtypes(StorageEngineConfigBase.getSubTypes(this));
+
};
/**
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
new file mode 100644
index 0000000..9096d89
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/ExecutionSetupException.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * 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.drill.common.exceptions;
+
+
+public class ExecutionSetupException extends DrillException{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionSetupException.class);
+
+ public ExecutionSetupException() {
+ super();
+ }
+
+ public ExecutionSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+ super(message, cause, enableSuppression, writableStackTrace);
+ }
+
+ public ExecutionSetupException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public ExecutionSetupException(String message) {
+ super(message);
+ }
+
+ public ExecutionSetupException(Throwable cause) {
+ super(cause);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
new file mode 100644
index 0000000..f22cb16
--- /dev/null
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/exceptions/PhysicalOperatorSetupException.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.common.exceptions;
+
+public class PhysicalOperatorSetupException extends ExecutionSetupException{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorSetupException.class);
+
+ public PhysicalOperatorSetupException() {
+ super();
+ }
+
+ public PhysicalOperatorSetupException(String message, Throwable cause, boolean enableSuppression,
+ boolean writableStackTrace) {
+ super(message, cause, enableSuppression, writableStackTrace);
+ }
+
+ public PhysicalOperatorSetupException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public PhysicalOperatorSetupException(String message) {
+ super(message);
+ }
+
+ public PhysicalOperatorSetupException(Throwable cause) {
+ super(cause);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index b3675a8..004d812 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -104,7 +104,7 @@ public class SchemaPath extends LogicalExpressionBase{
}
-
+
@Override
public <T> T accept(ExprVisitor<T> visitor) {
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
index 25b82a7..56e2485 100644
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
+++ b/sandbox/prototype/common/src/main/java/org/apache/drill/common/expression/types/DataType.java
@@ -74,6 +74,17 @@ public abstract class DataType {
public static final DataType ARRAY = new AtomType("ARRAY", Comparability.NONE, false);
public static final DataType NULL = new AtomType("NULL", Comparability.NONE, false);
+ //TODO: Hack to get some size data, needs to be fixed so that each type reveals it's size.
+ public int size(){
+ if(this == BOOLEAN){
+ return 1;
+ }else if(this == INT32){
+ return 4;
+ }else if(this == INT16){
+ return 4;
+ }
+ return 2;
+ }
static final Map<String, DataType> TYPES;
static {
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
deleted file mode 100644
index 4b2037c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/optimize/Optimizer.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.optimize;
-
-import java.io.Closeable;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.exceptions.DrillConfigurationException;
-import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.physical.PhysicalPlan;
-
-public abstract class Optimizer implements Closeable{
-
- public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
-
- public abstract void init(DrillConfig config);
-
- public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
- public abstract void close();
-
- public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
- Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
- o.init(config);
- return o;
- }
-
- public interface OptimizationContext{
- public int getPriority();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
deleted file mode 100644
index 6de2cfd..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/DataValidationMode.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-public enum DataValidationMode {
- TERMINATE, // terminate the query if the data doesn't match expected.
- DROP_RECORD, // drop the record that doesn't match the expected situation.
- SINK_RECORD // record the failed record along with the rule violation in a secondary location.
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
deleted file mode 100644
index 9ccf430..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/EndpointAffinity.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-public class EndpointAffinity implements Comparable<EndpointAffinity>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointAffinity.class);
-
- private DrillbitEndpoint endpoint;
- private float affinity = 0.0f;
-
- public EndpointAffinity(DrillbitEndpoint endpoint) {
- super();
- this.endpoint = endpoint;
- }
-
- public EndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
- super();
- this.endpoint = endpoint;
- this.affinity = affinity;
- }
-
- public DrillbitEndpoint getEndpoint() {
- return endpoint;
- }
- public void setEndpoint(DrillbitEndpoint endpoint) {
- this.endpoint = endpoint;
- }
- public float getAffinity() {
- return affinity;
- }
-
- @Override
- public int compareTo(EndpointAffinity o) {
- return Float.compare(affinity, o.affinity);
- }
-
- public void addAffinity(float f){
- affinity += f;
- }
-
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
deleted file mode 100644
index c76098d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/FieldSet.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.drill.common.physical.FieldSet.De;
-import org.apache.drill.common.physical.FieldSet.Se;
-
-import com.fasterxml.jackson.core.JsonGenerationException;
-import com.fasterxml.jackson.core.JsonGenerator;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.SerializerProvider;
-import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
-import com.fasterxml.jackson.databind.annotation.JsonSerialize;
-import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
-import com.fasterxml.jackson.databind.ser.std.StdSerializer;
-import com.google.common.collect.Lists;
-
-@JsonSerialize(using = Se.class)
-@JsonDeserialize(using = De.class)
-public class FieldSet {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FieldSet.class);
-
- private List<RecordField> fields;
-
- public FieldSet(Iterable<RecordField> fields){
- this.fields = Lists.newArrayList(fields);
- }
-
-
- public static class De extends StdDeserializer<FieldSet> {
-
- public De() {
- super(FieldSet.class);
- }
-
- @Override
- public FieldSet deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
- JsonProcessingException {
- Iterable<RecordField> fields = jp.readValueAs(new TypeReference<List<RecordField>>(){});
- logger.debug("Fields {}", fields);
- return new FieldSet(fields);
- }
-
- }
-
- public static class Se extends StdSerializer<FieldSet> {
-
- public Se() {
- super(FieldSet.class);
- }
-
- @Override
- public void serialize(FieldSet value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
- JsonGenerationException {
- jgen.writeObject(value.fields);
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
deleted file mode 100644
index fadfff0..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/OperatorCost.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class OperatorCost {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCost.class);
-
- private final float network;
- private final float disk;
- private final float memory;
- private final float cpu;
-
- @JsonCreator
- public OperatorCost(@JsonProperty("network") float network, @JsonProperty("disk") float disk, @JsonProperty("memory") float memory, @JsonProperty("cpu") float cpu) {
- super();
- this.network = network;
- this.disk = disk;
- this.memory = memory;
- this.cpu = cpu;
- }
-
- public float getNetwork() {
- return network;
- }
-
- public float getDisk() {
- return disk;
- }
-
- public float getMemory() {
- return memory;
- }
-
- public float getCpu() {
- return cpu;
- }
-
- public static OperatorCost combine(OperatorCost c1, OperatorCost c2){
- return new OperatorCost(c1.network + c2.network, c1.disk + c2.disk, c1.memory + c2.memory, c1.cpu + c2.cpu);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
deleted file mode 100644
index e83dac7..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/PhysicalPlan.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.drill.common.PlanProperties;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.graph.Graph;
-import org.apache.drill.common.graph.GraphAlgos;
-import org.apache.drill.common.physical.pop.base.Leaf;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Root;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.collect.Lists;
-
-@JsonPropertyOrder({ "head", "graph" })
-public class PhysicalPlan {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
-
- PlanProperties properties;
-
- Graph<PhysicalOperator, Root, Leaf> graph;
-
- @JsonCreator
- public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
- this.properties = properties;
- this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
- }
-
- @JsonProperty("graph")
- public List<PhysicalOperator> getSortedOperators(){
- // reverse the list so that nested references are flattened rather than nested.
- return getSortedOperators(true);
- }
-
- public List<PhysicalOperator> getSortedOperators(boolean reverse){
- List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
- if(reverse){
- return Lists.reverse(list);
- }else{
- return list;
- }
-
- }
-
-
- @JsonProperty("head")
- public PlanProperties getProperties() {
- return properties;
- }
-
- /** Parses a physical plan. */
- public static PhysicalPlan parse(ObjectReader reader, String planString) {
- try {
- PhysicalPlan plan = reader.readValue(planString);
- return plan;
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
- public String unparse(ObjectWriter writer) {
- try {
- return writer.writeValueAsString(this);
- } catch (JsonProcessingException e) {
- throw new RuntimeException(e);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
deleted file mode 100644
index 7c23cf5..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/ReadEntry.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-/**
- * Describes a chunk of read work that will be done.
- */
-public interface ReadEntry {
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
deleted file mode 100644
index 8d0072a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/RecordField.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import org.apache.drill.common.expression.types.DataType;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class RecordField {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
-
-
- private DataType type;
- private ValueMode mode;
-
- @JsonCreator
- public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
- super();
- this.type = type;
- this.mode = mode;
- }
-
- public DataType getType() {
- return type;
- }
-
- public ValueMode getMode() {
- return mode;
- }
-
- public static enum ValueMode {
- VECTOR,
- DICT,
- RLE
- }
-
- public static enum ValueType {
- OPTIONAL,
- REQUIRED,
- REPEATED
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/WriteEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/WriteEntry.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/WriteEntry.java
deleted file mode 100644
index 7440ce2..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/WriteEntry.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-public interface WriteEntry {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WriteEntry.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
deleted file mode 100644
index 2c86d99..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Filter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("filter")
-public class Filter extends AbstractSingle {
-
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
-
- private final LogicalExpression expr;
-
- @JsonCreator
- public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
- super(child);
- this.expr = expr;
- }
-
- public LogicalExpression getExpr() {
- return expr;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitFilter(this, value);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
deleted file mode 100644
index 0289780..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/PartitionToRandomExchange.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop;
-
-import java.util.List;
-
-import org.apache.drill.common.defs.PartitionDef;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.physical.pop.base.AbstractExchange;
-import org.apache.drill.common.physical.pop.base.ExchangeCost;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Receiver;
-import org.apache.drill.common.physical.pop.base.Sender;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("partition-to-random-exchange")
-public class PartitionToRandomExchange extends AbstractExchange{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionToRandomExchange.class);
-
- private final PartitionDef partition;
- private final int maxWidth;
-
- @JsonCreator
- public PartitionToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("partition") PartitionDef partition, @JsonProperty("cost") ExchangeCost cost) {
- super(child, cost);
- this.partition = partition;
-
- LogicalExpression[] parts = partition.getStarts();
- if(parts != null && parts.length > 0){
- this.maxWidth = parts.length+1;
- }else{
- this.maxWidth = Integer.MAX_VALUE;
- }
- }
-
- public PartitionDef getPartition() {
- return partition;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitExchange(this, value);
- }
-
- @Override
- public int getMaxSendWidth() {
- return maxWidth;
- }
-
- @Override
- public void setupSenders(List<DrillbitEndpoint> senderLocations) {
- }
-
- @Override
- public void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
- }
-
- @Override
- public Sender getSender(int minorFragmentId, PhysicalOperator child) {
- return null;
- }
-
- @Override
- public Receiver getReceiver(int minorFragmentId) {
- return null;
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
deleted file mode 100644
index 7cff28d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Project.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("project")
-public class Project extends AbstractSingle{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
-
- private final List<NamedExpression> exprs;
-
- @JsonCreator
- public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
- super(child);
- this.exprs = exprs;
- }
-
- public List<NamedExpression> getExprs() {
- return exprs;
- }
-
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitProject(this, value);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
deleted file mode 100644
index fdbd8f1..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Screen.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("screen")
-public class Screen extends AbstractStore {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
-
- private final DrillbitEndpoint endpoint;
-
- public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
- super(child);
- this.endpoint = endpoint;
- }
-
- @Override
- public List<EndpointAffinity> getOperatorAffinity() {
- return Collections.singletonList(new EndpointAffinity(endpoint, 1000));
- }
-
- @Override
- public int getMaxWidth() {
- return 1;
- }
-
- @Override
- public void applyAssignments(List<DrillbitEndpoint> endpoints) {
- // we actually don't have to do anything since nothing should have changed. we'll check just check that things
- // didn't get screwed up.
- if (endpoints.size() != 1)
- throw new UnsupportedOperationException("A Screen operator can only be assigned to a single node.");
- DrillbitEndpoint endpoint = endpoints.iterator().next();
- if (this.endpoint != endpoint)
- throw new UnsupportedOperationException("A Screen operator can only be assigned to its home node.");
-
- }
-
- @Override
- public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
- return new Screen(child, endpoint);
- }
-
- @JsonIgnore
- public DrillbitEndpoint getEndpoint() {
- return endpoint;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
deleted file mode 100644
index b4d802d..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/Sort.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.physical.pop.base.AbstractSingle;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("sort")
-public class Sort extends AbstractSingle{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
-
- private final LogicalExpression expr;
- private boolean reverse = false;
-
- @JsonCreator
- public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("reverse") boolean reverse) {
- super(child);
- this.expr = expr;
- this.reverse = reverse;
- }
-
- public LogicalExpression getExpr() {
- return expr;
- }
-
- public boolean getReverse() {
- return reverse;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitSort(this, value);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
deleted file mode 100644
index 5d3584c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractBase.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.graph.GraphVisitor;
-import org.apache.drill.common.physical.OperatorCost;
-
-public abstract class AbstractBase implements PhysicalOperator{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
-
- private OperatorCost cost;
-
- @Override
- public void accept(GraphVisitor<PhysicalOperator> visitor) {
- visitor.enter(this);
- if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
- for(PhysicalOperator o : this){
- o.accept(visitor);
- }
- visitor.leave(this);
- }
-
- @Override
- public boolean isExecutable() {
- return true;
- }
-
- @Override
- public OperatorCost getCost() {
- return cost;
- }
-
- // should be used only for the purposes of json...
- void setCost(OperatorCost cost){
- this.cost = cost;
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
deleted file mode 100644
index 1f60c53..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractExchange.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public abstract class AbstractExchange extends AbstractSingle implements Exchange {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
-
- private final ExchangeCost cost;
-
- public AbstractExchange(PhysicalOperator child, ExchangeCost cost) {
- super(child);
- this.cost = cost;
- }
-
- /**
- * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
- * something executable
- */
- @Override
- public boolean isExecutable() {
- return false;
- }
-
- @Override
- public OperatorCost getAggregateSendCost() {
- return cost.getSend();
- }
-
- @Override
- public OperatorCost getAggregateReceiveCost() {
- return cost.getReceive();
- }
-
- @Override
- public ExchangeCost getExchangeCost() {
- return cost;
- }
-
- @JsonIgnore
- @Override
- public OperatorCost getCost() {
- return cost.getCombinedCost();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
deleted file mode 100644
index 49f7bda..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractPhysicalVisitor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.pop.Filter;
-import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
-import org.apache.drill.common.physical.pop.Project;
-import org.apache.drill.common.physical.pop.Sort;
-
-public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
-
- @Override
- public T visitExchange(Exchange exchange, X value) throws E{
- return visitUnknown(exchange, value);
- }
-
- @Override
- public T visitFilter(Filter filter, X value) throws E{
- return visitUnknown(filter, value);
- }
-
- @Override
- public T visitProject(Project project, X value) throws E{
- return visitUnknown(project, value);
- }
-
- @Override
- public T visitSort(Sort sort, X value) throws E{
- return visitUnknown(sort, value);
- }
-
- @Override
- public T visitSender(Sender sender, X value) throws E {
- return visitUnknown(sender, value);
- }
-
- @Override
- public T visitReceiver(Receiver receiver, X value) throws E {
- return visitUnknown(receiver, value);
- }
-
- @Override
- public T visitScan(Scan<?> scan, X value) throws E{
- return visitUnknown(scan, value);
- }
-
- @Override
- public T visitStore(Store store, X value) throws E{
- return visitUnknown(store, value);
- }
-
- @Override
- public T visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, X value) throws E{
- return visitExchange(partitionToRandom, value);
- }
-
- @Override
- public T visitUnknown(PhysicalOperator op, X value) throws E{
- throw new UnsupportedOperationException(String.format(
- "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
- .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
deleted file mode 100644
index fd9d93c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractReceiver.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-
-import com.google.common.collect.Iterators;
-
-public abstract class AbstractReceiver extends AbstractBase implements Receiver{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
-
- @Override
- public Iterator<PhysicalOperator> iterator() {
- return Iterators.emptyIterator();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
deleted file mode 100644
index 3727139..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractScan.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.ReadEntry;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.collect.Iterators;
-
-public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
-
- private List<R> readEntries;
-
- public AbstractScan(List<R> readEntries) {
- this.readEntries = readEntries;
- }
-
- @Override
- @JsonProperty("entries")
- public List<R> getReadEntries() {
- return readEntries;
- }
-
- @Override
- public Iterator<PhysicalOperator> iterator() {
- return Iterators.emptyIterator();
- }
-
- @Override
- public boolean isExecutable() {
- return true;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitScan(this, value);
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
deleted file mode 100644
index 8b0608a..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSender.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-
-
-public abstract class AbstractSender extends AbstractSingle implements Sender {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSender.class);
-
- public AbstractSender(PhysicalOperator child) {
- super(child);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
deleted file mode 100644
index 49358df..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractSingle.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.Iterator;
-
-
-import com.google.common.collect.Iterators;
-
-/**
- * Describes an operator that expects a single child operator as its input.
- * @param <T> The type of Exec model supported.
- */
-public abstract class AbstractSingle extends AbstractBase{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
-
- private final PhysicalOperator child;
-
- public AbstractSingle(PhysicalOperator child) {
- super();
- this.child = child;
- }
-
- @Override
- public Iterator<PhysicalOperator> iterator() {
- return Iterators.singletonIterator(child);
- }
-
- public PhysicalOperator getChild(){
- return child;
- }
-
-}
[11/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
index 85c573d..75dce2c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ZKClusterCoordinator.java
@@ -23,16 +23,19 @@ import static com.google.common.collect.Collections2.transform;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import com.google.common.base.Function;
import com.netflix.curator.RetryPolicy;
import com.netflix.curator.framework.CuratorFramework;
import com.netflix.curator.framework.CuratorFrameworkFactory;
import com.netflix.curator.framework.state.ConnectionState;
+import com.netflix.curator.framework.state.ConnectionStateListener;
import com.netflix.curator.retry.RetryNTimes;
import com.netflix.curator.x.discovery.ServiceDiscovery;
import com.netflix.curator.x.discovery.ServiceDiscoveryBuilder;
@@ -52,6 +55,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
private ServiceCache<DrillbitEndpoint> serviceCache;
private volatile Collection<DrillbitEndpoint> endpoints = Collections.emptyList();
private final String serviceName;
+ private final CountDownLatch initialConnection = new CountDownLatch(1);
public ZKClusterCoordinator(DrillConfig config) throws IOException {
@@ -64,6 +68,7 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
.retryPolicy(rp)
.connectString(config.getString(ExecConstants.ZK_CONNECTION))
.build();
+ curator.getConnectionStateListenable().addListener(new InitialConnectionListener());
discovery = getDiscovery();
serviceCache = discovery.
serviceCacheBuilder()
@@ -72,15 +77,36 @@ public class ZKClusterCoordinator extends ClusterCoordinator {
.build();
}
- public void start() throws Exception {
+ public void start(long millisToWait) throws Exception {
logger.debug("Starting ZKClusterCoordination.");
curator.start();
discovery.start();
serviceCache.start();
serviceCache.addListener(new ZKListener());
+
+ if(millisToWait != 0){
+ boolean success = this.initialConnection.await(millisToWait, TimeUnit.MILLISECONDS);
+ if(!success) throw new IOException(String.format("Failure to connect to the zookeeper cluster service within the allotted time of %d milliseconds.", millisToWait));
+ }else{
+ this.initialConnection.await();
+ }
+
+
updateEndpoints();
}
+
+ private class InitialConnectionListener implements ConnectionStateListener{
+ @Override
+ public void stateChanged(CuratorFramework client, ConnectionState newState) {
+ if(newState == ConnectionState.CONNECTED){
+ ZKClusterCoordinator.this.initialConnection.countDown();
+ client.getConnectionStateListenable().removeListener(this);
+ }
+ }
+
+ }
+
private class ZKListener implements ServiceCacheListener {
@Override
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
new file mode 100644
index 0000000..9c18e51
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/BitComException.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.exception;
+
+import org.apache.drill.common.exceptions.DrillException;
+
+public class BitComException extends DrillException{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComException.class);
+
+ public BitComException() {
+ super();
+ }
+
+ public BitComException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
+ super(message, cause, enableSuppression, writableStackTrace);
+ }
+
+ public BitComException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public BitComException(String message) {
+ super(message);
+ }
+
+ public BitComException(Throwable cause) {
+ super(cause);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
deleted file mode 100644
index a4899bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/ExecutionSetupException.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.exception;
-
-import org.apache.drill.common.exceptions.DrillException;
-
-public class ExecutionSetupException extends DrillException{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionSetupException.class);
-
- public ExecutionSetupException() {
- super();
- }
-
- public ExecutionSetupException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
- super(message, cause, enableSuppression, writableStackTrace);
- }
-
- public ExecutionSetupException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public ExecutionSetupException(String message) {
- super(message);
- }
-
- public ExecutionSetupException(Throwable cause) {
- super(cause);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
index c273463..dbd66b9 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/exception/FragmentSetupException.java
@@ -17,6 +17,8 @@
******************************************************************************/
package org.apache.drill.exec.exception;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+
public class FragmentSetupException extends ExecutionSetupException{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentSetupException.class);
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
deleted file mode 100644
index 30e7a63..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/CancelableQuery.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-public interface CancelableQuery {
- public boolean cancel(long queryid);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
deleted file mode 100644
index 4e4ec77..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ExecutionPlanner.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-public class ExecutionPlanner {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecutionPlanner.class);
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
deleted file mode 100644
index f138171..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/Foreman.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-
-public class Foreman extends Thread{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
-
- public Foreman(){
-
- }
-
- public void doWork(QueryWorkUnit work){
- // generate fragment structure.
- // store fragments in distributed grid.
- // generate any codegen required and store in grid.
- // drop
- // do get on the result set you're looking for. Do the initial get on the result node you're looking for. This will return either data or a metadata record set
- }
-
- public boolean checkStatus(long queryId){
- return false;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
deleted file mode 100644
index bdf4a1e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/QueryWorkUnit.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-import com.google.common.base.Preconditions;
-
-public class QueryWorkUnit {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
-
- private PlanFragment rootFragment; // for local
- private List<PlanFragment> fragments;
-
- public QueryWorkUnit(PlanFragment rootFragment, List<PlanFragment> fragments) {
- super();
- Preconditions.checkNotNull(rootFragment);
- Preconditions.checkNotNull(fragments);
- this.rootFragment = rootFragment;
- this.fragments = fragments;
- }
-
- public PlanFragment getRootFragment() {
- return rootFragment;
- }
-
- public List<PlanFragment> getFragments() {
- return fragments;
- }
-
-
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
deleted file mode 100644
index 96d7d1e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/ResourceRequest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-public class ResourceRequest {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
-
- public long memoryMin;
- public long memoryDesired;
-
-
- public static class ResourceAllocation {
- public long memory;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
deleted file mode 100644
index fee6172..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/foreman/StatusProvider.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.foreman;
-
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-
-public interface StatusProvider {
- public FragmentStatus getStatus();
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
index 2b3f574..6bddab7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/memory/BufferAllocator.java
@@ -19,11 +19,10 @@ package org.apache.drill.exec.memory;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
import java.io.Closeable;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.common.config.DrillConfig;
/**
* Wrapper class to deal with byte buffer allocation. Ensures users only use designated methods. Also allows inser
@@ -48,7 +47,7 @@ public abstract class BufferAllocator implements Closeable{
@Override
public abstract void close();
- public static BufferAllocator getAllocator(DrillbitContext context){
+ public static BufferAllocator getAllocator(DrillConfig config){
// TODO: support alternative allocators (including a debugging allocator that records all allocation locations for each buffer).
return new DirectBufferAllocator();
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
index 6b89c12..1f47041 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/metrics/SingleThreadNestedCounter.java
@@ -22,34 +22,34 @@ import org.apache.drill.exec.server.DrillbitContext;
import com.yammer.metrics.Counter;
/**
- * Wraps a parent counter so that local in thread metrics can be collected while collecting for a global counter.
+ * Wraps a parent counter so that local in-thread metrics can be collected while collecting for a global counter. Note
+ * that this one writer, many reader safe.
*/
public class SingleThreadNestedCounter {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleThreadNestedCounter.class);
-
+
private volatile long count;
private final Counter counter;
-
-
+
public SingleThreadNestedCounter(DrillbitContext context, String name) {
super();
this.counter = context.getMetrics().counter(name);
}
- public long inc(long n){
+ public long inc(long n) {
counter.inc(n);
- count+= n;
+ count += n;
return count;
}
-
- public long dec(long n){
+
+ public long dec(long n) {
counter.dec(n);
count -= n;
return count;
}
-
- public long get(){
+
+ public long get() {
return count;
}
-
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
deleted file mode 100644
index f626cea..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FilteringRecordBatchTransformer.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
-
-public abstract class FilteringRecordBatchTransformer {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
-
- final RecordBatch incoming;
- final SelectionVector selectionVector;
- final BatchSchema schema;
-
- public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
- super();
- this.incoming = incoming;
- this.selectionVector = selectionVector;
- this.schema = innerSetup();
- }
-
- public abstract BatchSchema innerSetup();
-
- /**
- * Applies the filter to the selection index. Ignores any values in the selection vector, instead creating a.
- * @return
- */
- public abstract int apply();
-
- /**
- * Applies the filter to the selection index. Utilizes the existing selection index and only evaluates on those records.
- * @return
- */
- public abstract int applyWithSelection();
-
- public BatchSchema getSchema() {
- return schema;
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index 0cf17e9..e64453c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -20,10 +20,15 @@ package org.apache.drill.exec.ops;
import org.apache.drill.common.expression.LogicalExpression;
import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.planner.FragmentRunnable;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.physical.impl.FilteringRecordBatchTransformer;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
import com.yammer.metrics.MetricRegistry;
import com.yammer.metrics.Timer;
@@ -34,51 +39,72 @@ import com.yammer.metrics.Timer;
public class FragmentContext {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentContext.class);
- private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunnable.class, "completionTimes");
- private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "batchesCompleted");
- private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunnable.class, "recordsCompleted");
- private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunnable.class, "dataProcessed");
+ private final static String METRIC_TIMER_FRAGMENT_TIME = MetricRegistry.name(FragmentRunner.class, "completionTimes");
+ private final static String METRIC_BATCHES_COMPLETED = MetricRegistry.name(FragmentRunner.class, "batchesCompleted");
+ private final static String METRIC_RECORDS_COMPLETED = MetricRegistry.name(FragmentRunner.class, "recordsCompleted");
+ private final static String METRIC_DATA_PROCESSED = MetricRegistry.name(FragmentRunner.class, "dataProcessed");
private final DrillbitContext context;
- private final PlanFragment fragment;
public final SingleThreadNestedCounter batchesCompleted;
public final SingleThreadNestedCounter recordsCompleted;
public final SingleThreadNestedCounter dataProcessed;
public final Timer fragmentTime;
+ private final FragmentHandle handle;
+ private final UserClientConnection connection;
+ private final IncomingBuffers buffers;
- public FragmentContext(DrillbitContext dbContext, PlanFragment fragment) {
+ public FragmentContext(DrillbitContext dbContext, FragmentHandle handle, UserClientConnection connection, IncomingBuffers buffers) {
this.fragmentTime = dbContext.getMetrics().timer(METRIC_TIMER_FRAGMENT_TIME);
this.batchesCompleted = new SingleThreadNestedCounter(dbContext, METRIC_BATCHES_COMPLETED);
this.recordsCompleted = new SingleThreadNestedCounter(dbContext, METRIC_RECORDS_COMPLETED);
this.dataProcessed = new SingleThreadNestedCounter(dbContext, METRIC_DATA_PROCESSED);
this.context = dbContext;
- this.fragment = fragment;
+ this.connection = connection;
+ this.handle = handle;
+ this.buffers = buffers;
}
public void fail(Throwable cause) {
}
+
public DrillbitContext getDrillbitContext(){
return context;
}
-
- public PlanFragment getFragment() {
- return fragment;
+
+ public DrillbitEndpoint getIdentity(){
+ return context.getEndpoint();
}
+ public FragmentHandle getHandle() {
+ return handle;
+ }
+
public BufferAllocator getAllocator(){
// TODO: A local query allocator to ensure memory limits and accurately gauge memory usage.
return context.getAllocator();
}
-
public FilteringRecordBatchTransformer getFilteringExpression(LogicalExpression expr){
return null;
}
+ public void addMetricsToStatus(FragmentStatus.Builder stats){
+ stats.setBatchesCompleted(batchesCompleted.get());
+ stats.setDataProcessed(dataProcessed.get());
+ stats.setRecordsCompleted(recordsCompleted.get());
+ }
+ public UserClientConnection getConnection() {
+ return connection;
+ }
+
public BitCom getCommunicator(){
- return null;
+ return context.getBitCom();
+ }
+
+ public IncomingBuffers getBuffers(){
+ return buffers;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
deleted file mode 100644
index 3c75648..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentConverter.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-public class FragmentConverter {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentConverter.class);
-
- public static FragmentRoot getFragment(FragmentContext context){
- PlanFragment m = context.getFragment();
-
- return null;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
deleted file mode 100644
index ddacb41..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentRoot.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-/**
- * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
- * output nodes and storage nodes. They are there driving force behind the completion of a query.
- */
-public interface FragmentRoot {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRoot.class);
-
- /**
- * Do the next batch of work.
- * @return Whether or not additional batches of work are necessary.
- */
- public boolean next();
-
-
- public void setup() throws FragmentSetupException;
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
deleted file mode 100644
index 8d4e807..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorFactory.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-public class OperatorFactory {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorFactory.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
deleted file mode 100644
index 59abdc4..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/OutputMutator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public interface OutputMutator {
- public void removeField(int fieldId) throws SchemaChangeException;
- public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
- public void setNewSchema(BatchSchema schema) throws SchemaChangeException ;
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fe37e70..fd24deb 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -19,33 +19,42 @@ package org.apache.drill.exec.ops;
import java.util.Collection;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.server.DrillbitContext;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
public class QueryContext {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
- private long queryId;
+ private QueryId queryId;
private DrillbitContext drillbitContext;
- public QueryContext(long queryId, DrillbitContext drllbitContext) {
+ public QueryContext(QueryId queryId, DrillbitContext drllbitContext) {
super();
this.queryId = queryId;
this.drillbitContext = drllbitContext;
}
- public long getQueryId() {
- return queryId;
+ public DrillbitEndpoint getCurrentEndpoint(){
+ return drillbitContext.getEndpoint();
}
- public ObjectMapper getMapper(){
- return drillbitContext.getConfig().getMapper();
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public DistributedCache getCache(){
+ return drillbitContext.getCache();
}
public Collection<DrillbitEndpoint> getActiveEndpoints(){
return drillbitContext.getBits();
}
+ public PhysicalPlanReader getPlanReader(){
+ return drillbitContext.getPlanReader();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
deleted file mode 100644
index b46804f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ScanBatch.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-import java.util.Iterator;
-
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.ValueVector;
-import org.apache.drill.exec.store.RecordReader;
-
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.carrotsearch.hppc.procedures.IntObjectProcedure;
-
-/**
- * Record batch used for a particular scan. Operators against one or more
- */
-public abstract class ScanBatch implements RecordBatch {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
-
- private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
- private BatchSchema schema;
- private int recordCount;
- private boolean schemaChanged = true;
- private final FragmentContext context;
- private Iterator<RecordReader> readers;
- private RecordReader currentReader;
- private final BatchSchema expectedSchema;
- private final Mutator mutator = new Mutator();
-
- public ScanBatch(BatchSchema expectedSchema, Iterator<RecordReader> readers, FragmentContext context)
- throws ExecutionSetupException {
- this.expectedSchema = expectedSchema;
- this.context = context;
- this.readers = readers;
- if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
- this.currentReader = readers.next();
- this.currentReader.setup(expectedSchema, mutator);
- }
-
- private void schemaChanged() {
- schema = null;
- schemaChanged = true;
- }
-
- @Override
- public FragmentContext getContext() {
- return context;
- }
-
- @Override
- public BatchSchema getSchema() {
- return schema;
- }
-
- @Override
- public int getRecordCount() {
- return recordCount;
- }
-
- @Override
- public void kill() {
- releaseAssets();
- }
-
- private void releaseAssets() {
- fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
- @Override
- public void apply(int key, ValueVector<?> value) {
- value.close();
- }
- });
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
- if (fields.containsKey(fieldId))
- throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
- ValueVector<?> vector = this.fields.lget();
- if (vector.getClass().isAssignableFrom(clazz)) {
- return (T) vector;
- } else {
- throw new InvalidValueAccessor(String.format(
- "You requested a field accessor of type %s for field id %d but the actual type was %s.",
- clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
- }
- }
-
- @Override
- public IterOutcome next() {
- while ((recordCount = currentReader.next()) == 0) {
- try {
- if (!readers.hasNext()) {
- currentReader.cleanup();
- releaseAssets();
- return IterOutcome.NONE;
- }
- currentReader.cleanup();
- currentReader = readers.next();
- currentReader.setup(expectedSchema, mutator);
- } catch (ExecutionSetupException e) {
- this.context.fail(e);
- releaseAssets();
- return IterOutcome.STOP;
- }
- }
-
- if (schemaChanged) {
- schemaChanged = false;
- return IterOutcome.OK_NEW_SCHEMA;
- } else {
- return IterOutcome.OK;
- }
- }
-
- private class Mutator implements OutputMutator {
-
- public void removeField(int fieldId) throws SchemaChangeException {
- schemaChanged();
- ValueVector<?> v = fields.remove(fieldId);
- if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
- v.close();
- }
-
- public void addField(int fieldId, ValueVector<?> vector) {
- schemaChanged();
- ValueVector<?> v = fields.put(fieldId, vector);
- if (v != null) v.close();
- }
-
- @Override
- public void setNewSchema(BatchSchema schema) {
- ScanBatch.this.schema = schema;
- ScanBatch.this.schemaChanged = true;
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
deleted file mode 100644
index 0fc7a1f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/StreamingRecordBatch.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops;
-
-/**
- * Works on one incoming batch at a time. Creates one output batch for each input batch.
- */
-public class StreamingRecordBatch {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamingRecordBatch.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
deleted file mode 100644
index 07d7099..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/ExchangeRecordBatch.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops.exchange;
-
-public class ExchangeRecordBatch {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeRecordBatch.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/RecordBatchSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/RecordBatchSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/RecordBatchSender.java
deleted file mode 100644
index 0e35932..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/exchange/RecordBatchSender.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops.exchange;
-
-public class RecordBatchSender {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchSender.class);
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
deleted file mode 100644
index 5bef612..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/FilterRecordBatch.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops.filter;
-
-import org.apache.drill.exec.ops.FilteringRecordBatchTransformer;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.InvalidValueAccessor;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.record.vector.SelectionVector;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public abstract class FilterRecordBatch implements RecordBatch {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
-
- private RecordBatch incoming;
- private SelectionVector selectionVector;
- private BatchSchema schema;
- private FilteringRecordBatchTransformer transformer;
- private int outstanding;
-
- public FilterRecordBatch(RecordBatch batch) {
- this.incoming = batch;
- }
-
- @Override
- public FragmentContext getContext() {
- return incoming.getContext();
- }
-
- @Override
- public BatchSchema getSchema() {
- return schema;
- }
-
- @Override
- public int getRecordCount() {
- return 0;
- }
-
- @Override
- public void kill() {
- incoming.kill();
- }
-
- @Override
- public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
- return null;
- }
-
- abstract int applyFilter(SelectionVector vector, int count);
-
- /**
- * Release all assets.
- */
- private void close() {
-
- }
-
- @Override
- public IterOutcome next() {
- while (true) {
- IterOutcome o = incoming.next();
- switch (o) {
- case OK_NEW_SCHEMA:
- transformer = incoming.getContext().getFilteringExpression(null);
- schema = transformer.getSchema();
- // fall through to ok.
- case OK:
-
- case NONE:
- case STOP:
- close();
- return IterOutcome.STOP;
- }
-
- if (outstanding > 0) {
- // move data to output location.
-
- for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
-
- }
- }
-
- // make sure the bit vector is as large as the current record batch.
- if (selectionVector.size() < incoming.getRecordCount()) {
- selectionVector.allocateNew(incoming.getRecordCount());
- }
-
- return null;
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
deleted file mode 100644
index 218a19a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ops/filter/SelectionVectorUpdater.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.ops.filter;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.memory.DirectBufferAllocator;
-import org.apache.drill.exec.record.vector.NullableInt32Vector;
-import org.apache.drill.exec.record.vector.UInt16Vector;
-import org.codehaus.janino.ExpressionEvaluator;
-
-public class SelectionVectorUpdater {
- //static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVectorUpdater.class);
-
- // Add a selection vector to a record batch.
- /**
- * where a + b < 10
- */
-
- public static int applyToBatch(final int recordCount, final NullableInt32Vector a, final NullableInt32Vector b,
- final UInt16Vector selectionVector) {
- int selectionIndex = 0;
- for (int i = 0; i < recordCount; i++) {
- int isNotNull = a.isNull(i) * b.isNull(i);
- if (isNotNull > 0 && a.get(i) + b.get(i) < 10) {
- selectionVector.set(selectionIndex, (char) i);
- selectionIndex++;
- }
- }
- return selectionIndex;
- }
-
- public static void mai2n(String[] args) {
- int size = 1024;
- BufferAllocator allocator = new DirectBufferAllocator();
- NullableInt32Vector a = new NullableInt32Vector(0, allocator);
- NullableInt32Vector b = new NullableInt32Vector(1, allocator);
- UInt16Vector select = new UInt16Vector(2, allocator);
- a.allocateNew(size);
- b.allocateNew(size);
- select.allocateNew(size);
- int r = 0;
- for (int i = 0; i < 1500; i++) {
- r += applyToBatch(size, a, b, select);
- }
-
- System.out.println(r);
- }
-
-public static void main(String[] args) throws Exception{
- ExpressionEvaluator ee = new ExpressionEvaluator(
- "c > d ? c : d", // expression
- int.class, // expressionType
- new String[] { "c", "d" }, // parameterNames
- new Class[] { int.class, int.class } // parameterTypes
- );
-
- Integer res = (Integer) ee.evaluate(
- new Object[] { // parameterValues
- new Integer(10),
- new Integer(11),
- }
- );
- System.out.println("res = " + res);
-}
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
index 70a42be..d2aaca3 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/IdentityOptimizer.java
@@ -19,8 +19,7 @@ package org.apache.drill.exec.opt;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.optimize.Optimizer;
-import org.apache.drill.common.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
public class IdentityOptimizer extends Optimizer {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IdentityOptimizer.class);
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
new file mode 100644
index 0000000..9f506c1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/opt/Optimizer.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.opt;
+
+import java.io.Closeable;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillConfigurationException;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.physical.PhysicalPlan;
+
+public abstract class Optimizer implements Closeable{
+
+ public static String OPTIMIZER_IMPL_KEY = "drill.exec.optimizer.implementation";
+
+ public abstract void init(DrillConfig config);
+
+ public abstract PhysicalPlan optimize(OptimizationContext context, LogicalPlan plan);
+ public abstract void close();
+
+ public static Optimizer getOptimizer(DrillConfig config) throws DrillConfigurationException{
+ Optimizer o = config.getInstanceOf(OPTIMIZER_IMPL_KEY, Optimizer.class);
+ o.init(config);
+ return o;
+ }
+
+ public interface OptimizationContext{
+ public int getPriority();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
new file mode 100644
index 0000000..334119d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/DataValidationMode.java
@@ -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.drill.exec.physical;
+
+public enum DataValidationMode {
+ TERMINATE, // terminate the query if the data doesn't match expected.
+ DROP_RECORD, // drop the record that doesn't match the expected situation.
+ SINK_RECORD // record the failed record along with the rule violation in a secondary location.
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
new file mode 100644
index 0000000..d7b21db
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/EndpointAffinity.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+
+public class EndpointAffinity implements Comparable<EndpointAffinity>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointAffinity.class);
+
+ private DrillbitEndpoint endpoint;
+ private float affinity = 0.0f;
+
+ public EndpointAffinity(DrillbitEndpoint endpoint) {
+ super();
+ this.endpoint = endpoint;
+ }
+
+ public EndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+ super();
+ this.endpoint = endpoint;
+ this.affinity = affinity;
+ }
+
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+ public void setEndpoint(DrillbitEndpoint endpoint) {
+ this.endpoint = endpoint;
+ }
+ public float getAffinity() {
+ return affinity;
+ }
+
+ @Override
+ public int compareTo(EndpointAffinity o) {
+ return Float.compare(affinity, o.affinity);
+ }
+
+ public void addAffinity(float f){
+ affinity += f;
+ }
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
new file mode 100644
index 0000000..ebe6446
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/OperatorCost.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class OperatorCost {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OperatorCost.class);
+
+ private final float network;
+ private final float disk;
+ private final float memory;
+ private final float cpu;
+
+
+
+ @JsonCreator
+ public OperatorCost(@JsonProperty("network") float network, @JsonProperty("disk") float disk, @JsonProperty("memory") float memory, @JsonProperty("cpu") float cpu) {
+ super();
+ this.network = network;
+ this.disk = disk;
+ this.memory = memory;
+ this.cpu = cpu;
+ }
+
+ public float getNetwork() {
+ return network;
+ }
+
+ public float getDisk() {
+ return disk;
+ }
+
+ public float getMemory() {
+ return memory;
+ }
+
+ public float getCpu() {
+ return cpu;
+ }
+
+ public static OperatorCost combine(OperatorCost c1, OperatorCost c2){
+ return new OperatorCost(c1.network + c2.network, c1.disk + c2.disk, c1.memory + c2.memory, c1.cpu + c2.cpu);
+ }
+
+ public OperatorCost add(OperatorCost c2){
+ return combine(this, c2);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
new file mode 100644
index 0000000..84bfc87
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/PhysicalPlan.java
@@ -0,0 +1,94 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.PlanProperties;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.graph.Graph;
+import org.apache.drill.common.graph.GraphAlgos;
+import org.apache.drill.exec.physical.base.Leaf;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Root;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.collect.Lists;
+
+@JsonPropertyOrder({ "head", "graph" })
+public class PhysicalPlan {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlan.class);
+
+ PlanProperties properties;
+
+ Graph<PhysicalOperator, Root, Leaf> graph;
+
+ @JsonCreator
+ public PhysicalPlan(@JsonProperty("head") PlanProperties properties, @JsonProperty("graph") List<PhysicalOperator> operators){
+ this.properties = properties;
+ this.graph = Graph.newGraph(operators, Root.class, Leaf.class);
+ }
+
+ @JsonProperty("graph")
+ public List<PhysicalOperator> getSortedOperators(){
+ // reverse the list so that nested references are flattened rather than nested.
+ return getSortedOperators(true);
+ }
+
+ public List<PhysicalOperator> getSortedOperators(boolean reverse){
+ List<PhysicalOperator> list = GraphAlgos.TopoSorter.sort(graph);
+ if(reverse){
+ return Lists.reverse(list);
+ }else{
+ return list;
+ }
+
+ }
+
+
+ @JsonProperty("head")
+ public PlanProperties getProperties() {
+ return properties;
+ }
+
+ /** Parses a physical plan. */
+ public static PhysicalPlan parse(ObjectReader reader, String planString) {
+ try {
+ PhysicalPlan plan = reader.readValue(planString);
+ return plan;
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /** Converts a physical plan to a string. (Opposite of {@link #parse}.) */
+ public String unparse(ObjectWriter writer) {
+ try {
+ return writer.writeValueAsString(this);
+ } catch (JsonProcessingException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
new file mode 100644
index 0000000..02fe025
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/ReadEntry.java
@@ -0,0 +1,33 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * Describes a chunk of read work that will be done.
+ */
+public interface ReadEntry {
+ @JsonIgnore
+ public OperatorCost getCost();
+
+ @JsonIgnore
+ public Size getSize();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
new file mode 100644
index 0000000..db3390a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/RecordField.java
@@ -0,0 +1,60 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+import org.apache.drill.common.expression.types.DataType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class RecordField {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordField.class);
+
+
+ private DataType type;
+ private ValueMode mode;
+
+ @JsonCreator
+ public RecordField(@JsonProperty("type") DataType type, @JsonProperty("mode") ValueMode mode) {
+ super();
+ this.type = type;
+ this.mode = mode;
+ }
+
+ public DataType getType() {
+ return type;
+ }
+
+ public ValueMode getMode() {
+ return mode;
+ }
+
+ public static enum ValueMode {
+ VECTOR,
+ DICT,
+ RLE
+ }
+
+ public static enum ValueType {
+ OPTIONAL,
+ REQUIRED,
+ REPEATED
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/WriteEntry.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/WriteEntry.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/WriteEntry.java
new file mode 100644
index 0000000..96bd996
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/WriteEntry.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * 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.drill.exec.physical;
+
+public interface WriteEntry {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WriteEntry.class);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
new file mode 100644
index 0000000..e91257e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
@@ -0,0 +1,43 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.exec.physical.OperatorCost;
+
+public abstract class AbstractBase implements PhysicalOperator{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractBase.class);
+
+
+
+ @Override
+ public void accept(GraphVisitor<PhysicalOperator> visitor) {
+ visitor.enter(this);
+ if(this.iterator() == null) throw new IllegalArgumentException("Null iterator for pop." + this);
+ for(PhysicalOperator o : this){
+ o.accept(visitor);
+ }
+ visitor.leave(this);
+ }
+
+ @Override
+ public boolean isExecutable() {
+ return true;
+ }
+
+}
Re: [10/13] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.
Posted by Ted Dunning <te...@gmail.com>.
Sent from my iPhone
On May 13, 2013, at 18:52, jacques@apache.org wrote:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> new file mode 100644
> index 0000000..42a15ae
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
> @@ -0,0 +1,90 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.base;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +
> +public abstract class AbstractExchange extends AbstractSingle implements Exchange {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
> +
> + protected int senderMajorFragmentId;
> + protected int receiverMajorFragmentId;
> +
> + public AbstractExchange(PhysicalOperator child) {
> + super(child);
> + }
> +
> + /**
> + * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
> + * something executable
> + */
> + @Override
> + public boolean isExecutable() {
> + return false;
> + }
> +
> + protected abstract void setupSenders(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
> + protected abstract void setupReceivers(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
> +
> + @Override
> + public final void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException {
> + this.senderMajorFragmentId = majorFragmentId;
> + setupSenders(senderLocations);
> + }
> +
> +
> + @Override
> + public final void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
> + this.receiverMajorFragmentId = majorFragmentId;
> + setupReceivers(receiverLocations);
> + }
> +
> + @Override
> + public OperatorCost getAggregateSendCost() {
> + return getExchangeCost().getSendCost();
> + }
> +
> + @Override
> + public OperatorCost getAggregateReceiveCost() {
> + return getExchangeCost().getReceiveCost();
> + }
> +
> + @Override
> + public final <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> + return physicalVisitor.visitExchange(this, value);
> + }
> +
> + @Override
> + public ExchangeCost getExchangeCost(){
> + return ExchangeCost.getSimpleEstimate(getSize());
> + }
> +
> + @JsonIgnore
> + @Override
> + public OperatorCost getCost() {
> + return getExchangeCost().getCombinedCost();
> + }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> new file mode 100644
> index 0000000..f782325
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
> @@ -0,0 +1,124 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.base;
> +
> +import org.apache.drill.exec.physical.config.Filter;
> +import org.apache.drill.exec.physical.config.HashPartitionSender;
> +import org.apache.drill.exec.physical.config.HashToRandomExchange;
> +import org.apache.drill.exec.physical.config.Project;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.physical.config.RangeSender;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.physical.config.Sort;
> +import org.apache.drill.exec.physical.config.UnionExchange;
> +
> +public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
> +
> + @Override
> + public T visitExchange(Exchange exchange, X value) throws E{
> + return visitOp(exchange, value);
> + }
> +
> + @Override
> + public T visitFilter(Filter filter, X value) throws E{
> + return visitOp(filter, value);
> + }
> +
> + @Override
> + public T visitProject(Project project, X value) throws E{
> + return visitOp(project, value);
> + }
> +
> + @Override
> + public T visitSort(Sort sort, X value) throws E{
> + return visitOp(sort, value);
> + }
> +
> + @Override
> + public T visitSender(Sender sender, X value) throws E {
> + return visitOp(sender, value);
> + }
> +
> + @Override
> + public T visitReceiver(Receiver receiver, X value) throws E {
> + return visitOp(receiver, value);
> + }
> +
> + @Override
> + public T visitScan(Scan<?> scan, X value) throws E{
> + return visitOp(scan, value);
> + }
> +
> + @Override
> + public T visitStore(Store store, X value) throws E{
> + return visitOp(store, value);
> + }
> +
> +
> + public T visitChildren(PhysicalOperator op, X value) throws E{
> + for(PhysicalOperator child : op){
> + child.accept(this, value);
> + }
> + return null;
> + }
> +
> + @Override
> + public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
> + return visitSender(op, value);
> + }
> +
> + @Override
> + public T visitRandomReceiver(RandomReceiver op, X value) throws E {
> + return visitReceiver(op, value);
> + }
> +
> + @Override
> + public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
> + return visitExchange(op, value);
> + }
> +
> + @Override
> + public T visitRangeSender(RangeSender op, X value) throws E {
> + return visitSender(op, value);
> + }
> +
> + @Override
> + public T visitScreen(Screen op, X value) throws E {
> + return visitStore(op, value);
> + }
> +
> + @Override
> + public T visitSingleSender(SingleSender op, X value) throws E {
> + return visitSender(op, value);
> + }
> +
> + @Override
> + public T visitUnionExchange(UnionExchange op, X value) throws E {
> + return visitExchange(op, value);
> + }
> +
> + @Override
> + public T visitOp(PhysicalOperator op, X value) throws E{
> + throw new UnsupportedOperationException(String.format(
> + "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
> + .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
> + }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> new file mode 100644
> index 0000000..e8ba19c
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
> @@ -0,0 +1,63 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.base;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Iterators;
> +
> +public abstract class AbstractReceiver extends AbstractBase implements Receiver{
> +
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
> +
> + private final int oppositeMajorFragmentId;
> +
> + public AbstractReceiver(int oppositeMajorFragmentId){
> + this.oppositeMajorFragmentId = oppositeMajorFragmentId;
> + }
> +
> + @Override
> + public Iterator<PhysicalOperator> iterator() {
> + return Iterators.emptyIterator();
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> + return physicalVisitor.visitReceiver(this, value);
> + }
> +
> + @Override
> + public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
> + Preconditions.checkArgument(children.isEmpty());
> + //rewriting is unnecessary since the inputs haven't changed.
> + return this;
> + }
> +
> + @JsonProperty("sender-major-fragment")
> + public int getOppositeMajorFragmentId() {
> + return oppositeMajorFragmentId;
> + }
> +
> +}
> +
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> new file mode 100644
> index 0000000..dbde9c5
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
> @@ -0,0 +1,84 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.base;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.ReadEntry;
> +
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.google.common.collect.Iterators;
> +
> +public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
> +
> + protected final List<R> readEntries;
> + private final OperatorCost cost;
> + private final Size size;
> +
> + public AbstractScan(List<R> readEntries) {
> + this.readEntries = readEntries;
> + OperatorCost cost = new OperatorCost(0,0,0,0);
> + Size size = new Size(0,0);
> + for(R r : readEntries){
> + cost = cost.add(r.getCost());
> + size = size.add(r.getSize());
> + }
> + this.cost = cost;
> + this.size = size;
> + }
> +
> + @Override
> + @JsonProperty("entries")
> + public List<R> getReadEntries() {
> + return readEntries;
> + }
> +
> + @Override
> + public Iterator<PhysicalOperator> iterator() {
> + return Iterators.emptyIterator();
> + }
> +
> + @Override
> + public boolean isExecutable() {
> + return true;
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> + return physicalVisitor.visitScan(this, value);
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + return cost;
> + }
> +
> + @Override
> + public Size getSize() {
> + return size;
> + }
> +
> +
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> new file mode 100644
> index 0000000..f8c22b3
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
> @@ -0,0 +1,53 @@
> +/*******************************************************************************
> + * Licensed to the Apache Software Foundation (ASF) under one
> + * or more contributor license agreements. See
[10/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
new file mode 100644
index 0000000..42a15ae
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractExchange.java
@@ -0,0 +1,90 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public abstract class AbstractExchange extends AbstractSingle implements Exchange {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractExchange.class);
+
+ protected int senderMajorFragmentId;
+ protected int receiverMajorFragmentId;
+
+ public AbstractExchange(PhysicalOperator child) {
+ super(child);
+ }
+
+ /**
+ * Exchanges are not executable. The Execution layer first has to set their parallelization and convert them into
+ * something executable
+ */
+ @Override
+ public boolean isExecutable() {
+ return false;
+ }
+
+ protected abstract void setupSenders(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
+ protected abstract void setupReceivers(List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException ;
+
+ @Override
+ public final void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException {
+ this.senderMajorFragmentId = majorFragmentId;
+ setupSenders(senderLocations);
+ }
+
+
+ @Override
+ public final void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
+ this.receiverMajorFragmentId = majorFragmentId;
+ setupReceivers(receiverLocations);
+ }
+
+ @Override
+ public OperatorCost getAggregateSendCost() {
+ return getExchangeCost().getSendCost();
+ }
+
+ @Override
+ public OperatorCost getAggregateReceiveCost() {
+ return getExchangeCost().getReceiveCost();
+ }
+
+ @Override
+ public final <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitExchange(this, value);
+ }
+
+ @Override
+ public ExchangeCost getExchangeCost(){
+ return ExchangeCost.getSimpleEstimate(getSize());
+ }
+
+ @JsonIgnore
+ @Override
+ public OperatorCost getCost() {
+ return getExchangeCost().getCombinedCost();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
new file mode 100644
index 0000000..f782325
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractPhysicalVisitor.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.RangeSender;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.UnionExchange;
+
+public abstract class AbstractPhysicalVisitor<T, X, E extends Throwable> implements PhysicalVisitor<T, X, E> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractPhysicalVisitor.class);
+
+ @Override
+ public T visitExchange(Exchange exchange, X value) throws E{
+ return visitOp(exchange, value);
+ }
+
+ @Override
+ public T visitFilter(Filter filter, X value) throws E{
+ return visitOp(filter, value);
+ }
+
+ @Override
+ public T visitProject(Project project, X value) throws E{
+ return visitOp(project, value);
+ }
+
+ @Override
+ public T visitSort(Sort sort, X value) throws E{
+ return visitOp(sort, value);
+ }
+
+ @Override
+ public T visitSender(Sender sender, X value) throws E {
+ return visitOp(sender, value);
+ }
+
+ @Override
+ public T visitReceiver(Receiver receiver, X value) throws E {
+ return visitOp(receiver, value);
+ }
+
+ @Override
+ public T visitScan(Scan<?> scan, X value) throws E{
+ return visitOp(scan, value);
+ }
+
+ @Override
+ public T visitStore(Store store, X value) throws E{
+ return visitOp(store, value);
+ }
+
+
+ public T visitChildren(PhysicalOperator op, X value) throws E{
+ for(PhysicalOperator child : op){
+ child.accept(this, value);
+ }
+ return null;
+ }
+
+ @Override
+ public T visitHashPartitionSender(HashPartitionSender op, X value) throws E {
+ return visitSender(op, value);
+ }
+
+ @Override
+ public T visitRandomReceiver(RandomReceiver op, X value) throws E {
+ return visitReceiver(op, value);
+ }
+
+ @Override
+ public T visitHashPartitionSender(HashToRandomExchange op, X value) throws E {
+ return visitExchange(op, value);
+ }
+
+ @Override
+ public T visitRangeSender(RangeSender op, X value) throws E {
+ return visitSender(op, value);
+ }
+
+ @Override
+ public T visitScreen(Screen op, X value) throws E {
+ return visitStore(op, value);
+ }
+
+ @Override
+ public T visitSingleSender(SingleSender op, X value) throws E {
+ return visitSender(op, value);
+ }
+
+ @Override
+ public T visitUnionExchange(UnionExchange op, X value) throws E {
+ return visitExchange(op, value);
+ }
+
+ @Override
+ public T visitOp(PhysicalOperator op, X value) throws E{
+ throw new UnsupportedOperationException(String.format(
+ "The PhysicalVisitor of type %s does not currently support visiting the PhysicalOperator type %s.", this
+ .getClass().getCanonicalName(), op.getClass().getCanonicalName()));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
new file mode 100644
index 0000000..e8ba19c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractReceiver.java
@@ -0,0 +1,63 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractReceiver extends AbstractBase implements Receiver{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractReceiver.class);
+
+ private final int oppositeMajorFragmentId;
+
+ public AbstractReceiver(int oppositeMajorFragmentId){
+ this.oppositeMajorFragmentId = oppositeMajorFragmentId;
+ }
+
+ @Override
+ public Iterator<PhysicalOperator> iterator() {
+ return Iterators.emptyIterator();
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitReceiver(this, value);
+ }
+
+ @Override
+ public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+ Preconditions.checkArgument(children.isEmpty());
+ //rewriting is unnecessary since the inputs haven't changed.
+ return this;
+ }
+
+ @JsonProperty("sender-major-fragment")
+ public int getOppositeMajorFragmentId() {
+ return oppositeMajorFragmentId;
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
new file mode 100644
index 0000000..dbde9c5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractScan.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.collect.Iterators;
+
+public abstract class AbstractScan<R extends ReadEntry> extends AbstractBase implements Scan<R>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractScan.class);
+
+ protected final List<R> readEntries;
+ private final OperatorCost cost;
+ private final Size size;
+
+ public AbstractScan(List<R> readEntries) {
+ this.readEntries = readEntries;
+ OperatorCost cost = new OperatorCost(0,0,0,0);
+ Size size = new Size(0,0);
+ for(R r : readEntries){
+ cost = cost.add(r.getCost());
+ size = size.add(r.getSize());
+ }
+ this.cost = cost;
+ this.size = size;
+ }
+
+ @Override
+ @JsonProperty("entries")
+ public List<R> getReadEntries() {
+ return readEntries;
+ }
+
+ @Override
+ public Iterator<PhysicalOperator> iterator() {
+ return Iterators.emptyIterator();
+ }
+
+ @Override
+ public boolean isExecutable() {
+ return true;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+ return physicalVisitor.visitScan(this, value);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return cost;
+ }
+
+ @Override
+ public Size getSize() {
+ return size;
+ }
+
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
new file mode 100644
index 0000000..f8c22b3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSender.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+
+
+public abstract class AbstractSender extends AbstractSingle implements Sender {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSender.class);
+
+ protected final int oppositeMajorFragmentId;
+
+ public AbstractSender(int oppositeMajorFragmentId, PhysicalOperator child) {
+ super(child);
+ this.oppositeMajorFragmentId = oppositeMajorFragmentId;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitSender(this, value);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ float network = child.getSize().getAggSize();
+ return new OperatorCost(network, 0, 1000, child.getSize().getRecordCount());
+ }
+
+ @Override
+ public int getOppositeMajorFragmentId() {
+ return oppositeMajorFragmentId;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
new file mode 100644
index 0000000..264ee91
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSingle.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.Iterator;
+import java.util.List;
+
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+/**
+ * Describes an operator that expects a single child operator as its input.
+ * @param <T> The type of Exec model supported.
+ */
+public abstract class AbstractSingle extends AbstractBase{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSingle.class);
+
+ protected final PhysicalOperator child;
+
+ public AbstractSingle(PhysicalOperator child) {
+ super();
+ this.child = child;
+ }
+
+ @Override
+ public Iterator<PhysicalOperator> iterator() {
+ return Iterators.singletonIterator(child);
+ }
+
+ public PhysicalOperator getChild(){
+ return child;
+ }
+
+ @Override
+ public Size getSize() {
+ return child.getSize();
+ }
+
+ @Override
+ @JsonIgnore
+ public final PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+ Preconditions.checkArgument(children.size() == 1);
+ return getNewWithChild(children.iterator().next());
+ }
+
+ protected abstract PhysicalOperator getNewWithChild(PhysicalOperator child);
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
new file mode 100644
index 0000000..a833a4e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractStore.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+
+
+
+public abstract class AbstractStore extends AbstractSingle implements Store{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
+
+ public AbstractStore(PhysicalOperator child) {
+ super(child);
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+ return physicalVisitor.visitStore(this, value);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
new file mode 100644
index 0000000..c8bc3a8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Exchange.java
@@ -0,0 +1,92 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Exchange extends PhysicalOperator {
+
+ @JsonIgnore
+ public abstract OperatorCost getAggregateSendCost();
+
+ @JsonIgnore
+ public abstract OperatorCost getAggregateReceiveCost();
+
+ @JsonIgnore
+ public abstract ExchangeCost getExchangeCost();
+
+ /**
+ * Inform this Exchange node about its sender locations. This list should be index-ordered the same as the expected
+ * minorFragmentIds for each sender.
+ *
+ * @param senderLocations
+ */
+ public abstract void setupSenders(int majorFragmentId, List<DrillbitEndpoint> senderLocations) throws PhysicalOperatorSetupException;
+
+ /**
+ * Inform this Exchange node about its receiver locations. This list should be index-ordered the same as the expected
+ * minorFragmentIds for each receiver.
+ *
+ * @param receiverLocations
+ */
+ public abstract void setupReceivers(int majorFragmentId, List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException;
+
+ /**
+ * Get the Sender associated with the given minorFragmentId. Cannot be called until after setupSenders() and
+ * setupReceivers() have been called.
+ *
+ * @param minorFragmentId
+ * The minor fragment id, must be in the range [0, fragment.width).
+ * @param child
+ * The feeding node for the requested sender.
+ * @return The materialized sender for the given arguments.
+ */
+ public abstract Sender getSender(int minorFragmentId, PhysicalOperator child) throws PhysicalOperatorSetupException;
+
+ /**
+ * Get the Receiver associated with the given minorFragmentId. Cannot be called until after setupSenders() and
+ * setupReceivers() have been called.
+ *
+ * @param minorFragmentId
+ * The minor fragment id, must be in the range [0, fragment.width).
+ * @return The materialized recevier for the given arguments.
+ */
+ public abstract Receiver getReceiver(int minorFragmentId);
+
+ /**
+ * The widest width this sender can send (max sending parallelization). Typically Integer.MAX_VALUE.
+ *
+ * @return
+ */
+ @JsonIgnore
+ public abstract int getMaxSendWidth();
+
+ /**
+ * Return the feeding child of this operator node.
+ *
+ * @return
+ */
+ public PhysicalOperator getChild();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
new file mode 100644
index 0000000..9e94f06
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/ExchangeCost.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A container class that holds both send and receive costs for an exchange node.
+ */
+public class ExchangeCost {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeCost.class);
+
+ private final OperatorCost send;
+ private final OperatorCost receive;
+ private final OperatorCost combined;
+
+ @JsonCreator
+ public ExchangeCost(@JsonProperty("send") OperatorCost send, @JsonProperty("receive") OperatorCost receive) {
+ this.send = send;
+ this.receive = receive;
+ this.combined = OperatorCost.combine(send, receive);
+ }
+
+
+
+ @JsonIgnore
+ public OperatorCost getCombinedCost(){
+ return combined;
+ }
+
+ @JsonProperty("send")
+ public OperatorCost getSendCost() {
+ return send;
+ }
+
+ @JsonProperty("receive")
+ public OperatorCost getReceiveCost() {
+ return receive;
+ }
+
+ public static ExchangeCost getSimpleEstimate(Size s){
+ long cnt = s.getRecordCount();
+ int sz = s.getRecordSize();
+ OperatorCost send = new OperatorCost(cnt*sz, 0, 0, cnt);
+ OperatorCost receive = new OperatorCost(cnt*sz, 0, 0, cnt);
+ return new ExchangeCost(send, receive);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
new file mode 100644
index 0000000..522ef7b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentLeaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+/**
+ * A Physical Operator that can be the leaf node of one particular execution fragment. Typically includes Receivers and
+ * Scans
+ */
+public interface FragmentLeaf extends PhysicalOperator {
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
new file mode 100644
index 0000000..66147cc
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/FragmentRoot.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+
+/**
+ * Describes the root operation within a particular Fragment. This includes things Sender nodes.
+ */
+public interface FragmentRoot extends FragmentLeaf{
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
new file mode 100644
index 0000000..1a9a3a9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/HasAffinity.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * Describes a physical operator that has affinity to particular nodes. Used for assignment decisions.
+ */
+public interface HasAffinity extends PhysicalOperator {
+
+ /**
+ * Get the list of Endpoints with associated affinities that this operator has preference for.
+ * @return List of EndpointAffinity objects.
+ */
+ @JsonIgnore
+ public List<EndpointAffinity> getOperatorAffinity();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
new file mode 100644
index 0000000..d4ed456
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Leaf.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+/**
+ * An operator which specifically is a lowest level leaf node of a query plan across all possible fragments. Currently, the only operator that is a Leaf
+ * node are Scan nodes. Ultimately this could include use of Cache scans and other types of atypical data production systems.
+ */
+public interface Leaf extends FragmentLeaf {
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
new file mode 100644
index 0000000..d412c2d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.graph.GraphValue;
+import org.apache.drill.exec.physical.OperatorCost;
+
+import com.fasterxml.jackson.annotation.JsonIdentityInfo;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.ObjectIdGenerators;
+
+@JsonInclude(Include.NON_NULL)
+@JsonPropertyOrder({ "@id" })
+@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
+public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
+
+ /**
+ * Get the cost of execution of this particular operator.
+ *
+ * @return
+ */
+ @JsonIgnore
+ public OperatorCost getCost();
+
+ /**
+ * Get the estimated size of this particular operator.
+ * @return
+ */
+ @JsonIgnore
+ public Size getSize();
+
+ /**
+ * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
+ * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
+ * Exec sub components.
+ *
+ * @return
+ */
+ @JsonIgnore
+ public boolean isExecutable();
+
+ /**
+ * Provides capability to build a set of output based on traversing a query graph tree.
+ *
+ * @param physicalVisitor
+ * @return
+ */
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
+
+ /**
+ * Regenerate with this node with a new set of children. This is used in the case of materialization or optimization.
+ * @param children
+ * @return
+ */
+ @JsonIgnore
+ public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
new file mode 100644
index 0000000..d6e2fc4
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperatorUtil.java
@@ -0,0 +1,34 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.common.config.CommonConstants;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.PathScanner;
+
+public class PhysicalOperatorUtil {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorUtil.class);
+
+ private PhysicalOperatorUtil(){}
+
+ public synchronized static Class<?>[] getSubTypes(DrillConfig config){
+ Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
+ logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
+ return ops;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
new file mode 100644
index 0000000..f36633f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalVisitor.java
@@ -0,0 +1,61 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.HashPartitionSender;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.RangeSender;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.physical.config.UnionExchange;
+
+/**
+ * Visitor class designed to traversal of a operator tree. Basis for a number of operator manipulations including fragmentation and materialization.
+ * @param <RETURN> The class associated with the return of each visit method.
+ * @param <EXTRA> The class object associated with additional data required for a particular operator modification.
+ * @param <EXCEP> An optional exception class that can be thrown when a portion of a modification or traversal fails. Must extend Throwable. In the case where the visitor does not throw any caught exception, this can be set as RuntimeException.
+ */
+public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
+
+
+ public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
+ public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
+ public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
+
+ public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
+ public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
+ public RETURN visitSort(Sort sort, EXTRA value) throws EXCEP;
+ public RETURN visitSender(Sender sender, EXTRA value) throws EXCEP;
+ public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
+
+ public RETURN visitOp(PhysicalOperator op, EXTRA value) throws EXCEP;
+
+
+ public RETURN visitHashPartitionSender(HashPartitionSender op, EXTRA value) throws EXCEP;
+ public RETURN visitRandomReceiver(RandomReceiver op, EXTRA value) throws EXCEP;
+ public RETURN visitHashPartitionSender(HashToRandomExchange op, EXTRA value) throws EXCEP;
+ public RETURN visitRangeSender(RangeSender op, EXTRA value) throws EXCEP;
+ public RETURN visitScreen(Screen op, EXTRA value) throws EXCEP;
+ public RETURN visitSingleSender(SingleSender op, EXTRA value) throws EXCEP;
+ public RETURN visitUnionExchange(UnionExchange op, EXTRA value) throws EXCEP;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
new file mode 100644
index 0000000..cd4cb4a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Receiver.java
@@ -0,0 +1,51 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A receiver is one half of an exchange operator. The receiver is responsible for taking in one or more streams from
+ * corresponding Senders. Receivers are a special type of Physical Operator that are typically only expressed within the execution plan.
+ */
+public interface Receiver extends FragmentLeaf {
+
+ /**
+ * A receiver is expecting streams from one or more providing endpoints. This method should return a list of the expected sending endpoints.
+ * @return List of counterpart sending DrillbitEndpoints.
+ */
+ public abstract List<DrillbitEndpoint> getProvidingEndpoints();
+
+ /**
+ * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
+ * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
+ * receiver would supports this form of operation. A NWAY receiver would not.
+ *
+ * @return True if this receiver supports working on a streaming/out of order input.
+ */
+ @JsonIgnore
+ public abstract boolean supportsOutOfOrderExchange();
+
+ @JsonProperty("sender-major-fragment")
+ public int getOppositeMajorFragmentId();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
new file mode 100644
index 0000000..7adef63
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Root.java
@@ -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.drill.exec.physical.base;
+
+/**
+ * Marker interface describe the root of a query plan. Currently, this is constrained to Screen.
+ */
+public interface Root extends FragmentRoot{
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
new file mode 100644
index 0000000..2207f79
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Scan.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface Scan<R extends ReadEntry> extends Leaf, HasAffinity{
+
+ @JsonProperty("entries")
+ public abstract List<R> getReadEntries();
+
+ public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
+
+ public abstract Scan<?> getSpecificScan(int minorFragmentId);
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
new file mode 100644
index 0000000..71513c7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Sender.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A sender is one half of an exchange node operations. It is responsible for subdividing/cloning and sending a local
+ * record set to a set of destination locations. This is typically only utilized at the level of the execution plan.
+ */
+public interface Sender extends FragmentRoot {
+
+ /**
+ * Get the list of destination endpoints that this Sender will be communicating with.
+ * @return List of DrillbitEndpoints.
+ */
+ public abstract List<DrillbitEndpoint> getDestinations();
+
+ /**
+ * Get the receiver major fragment id that is opposite this sender.
+ * @return
+ */
+ @JsonProperty("receiver-major-fragment")
+ public int getOppositeMajorFragmentId();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
new file mode 100644
index 0000000..7bc6f93
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Size.java
@@ -0,0 +1,48 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+public class Size {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Size.class);
+
+ private final long rowCount;
+ private final int rowSize;
+
+ public Size(long rowCount, int rowSize) {
+ super();
+ this.rowCount = rowCount;
+ this.rowSize = rowSize;
+ }
+
+ public long getRecordCount() {
+ return rowCount;
+ }
+
+ public int getRecordSize() {
+ return rowSize;
+ }
+
+ public Size add(Size s){
+ return new Size(rowCount + s.rowCount, Math.max(rowSize, s.rowSize));
+ }
+
+ public long getAggSize(){
+ return rowCount * rowSize;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
new file mode 100644
index 0000000..9b3a812
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/Store.java
@@ -0,0 +1,74 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.base;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+/**
+ * An interface which supports storing a record stream. In contrast to the logical layer, in the physical/execution
+ * layers, a Store node is actually an outputting node (rather than a root node) that provides returns one or more
+ * records regarding the completion of the query.
+ */
+public interface Store extends HasAffinity {
+
+ /**
+ * Inform the Store node about the actual decided DrillbitEndpoint assignments desired for storage purposes. This is a
+ * precursor to the execution planner running a set of getSpecificStore() method calls for full Store node
+ * materialization.
+ *
+ * @param endpoints
+ * The list of endpoints that this Store node are going to be executed on.
+ * @throws PhysicalOperatorSetupException
+ */
+ public abstract void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException;
+
+ /**
+ * Provides full materialized Store operators for execution purposes.
+ *
+ * @param child
+ * The child operator that this operator will consume from.
+ * @param minorFragmentId
+ * The particular minor fragment id associated with this particular fragment materialization.
+ * @return A materialized Store Operator.
+ * @throws PhysicalOperatorSetupException
+ */
+ public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId)
+ throws PhysicalOperatorSetupException;
+
+ /**
+ * The maximum allowable width for the Store operation. In some cases, a store operation has a limited number of
+ * parallelizations that it can support. For example, a Screen return cannot be parallelized at all. In this case, a
+ * maxWidth value of 1 will be returned. In the case that there is no limit for parallelization, this method should
+ * return Integer.MAX_VALUE.
+ *
+ * @return
+ */
+ @JsonIgnore
+ public abstract int getMaxWidth();
+
+ /**
+ * Get the child of this store operator as this will be needed for parallelization materialization purposes.
+ * @return
+ */
+ public abstract PhysicalOperator getChild();
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
new file mode 100644
index 0000000..4af4243
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
@@ -0,0 +1,75 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("filter")
+public class Filter extends AbstractSingle {
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Filter.class);
+
+ private final LogicalExpression expr;
+ private final float selectivity;
+
+ @JsonCreator
+ public Filter(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("selectivity") float selectivity) {
+ super(child);
+ this.expr = expr;
+ this.selectivity = selectivity;
+ }
+
+ public LogicalExpression getExpr() {
+ return expr;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+ return physicalVisitor.visitFilter(this, value);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return child.getCost();
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new Filter(child, expr, selectivity);
+ }
+
+ @Override
+ public Size getSize() {
+ return new Size( (long) (child.getSize().getRecordCount()*selectivity), child.getSize().getRecordSize());
+ }
+
+
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
new file mode 100644
index 0000000..84994f6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashPartitionSender.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.collect.Lists;
+
+@JsonTypeName("hash-partition-sender")
+public class HashPartitionSender extends AbstractSender {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
+
+ private final List<DrillbitEndpoint> endpoints;
+ private final LogicalExpression expr;
+
+ @JsonCreator
+ public HashPartitionSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr, @JsonProperty("destinations") List<DrillbitEndpoint> endpoints) {
+ super(oppositeMajorFragmentId, child);
+ this.expr = expr;
+ this.endpoints = endpoints;
+ }
+
+ @Override
+ public List<DrillbitEndpoint> getDestinations() {
+ return endpoints;
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new HashPartitionSender(oppositeMajorFragmentId, child, expr, endpoints);
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
new file mode 100644
index 0000000..1f158ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.beans.Transient;
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("hash-to-random-exchange")
+public class HashToRandomExchange extends AbstractExchange{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashToRandomExchange.class);
+
+
+ private final LogicalExpression expr;
+
+ //ephemeral for setup tasks.
+ private List<DrillbitEndpoint> senderLocations;
+ private List<DrillbitEndpoint> receiverLocations;
+
+ @JsonCreator
+ public HashToRandomExchange(@JsonProperty("child") PhysicalOperator child, @JsonProperty("expr") LogicalExpression expr) {
+ super(child);
+ this.expr = expr;
+ }
+
+ @Override
+ public int getMaxSendWidth() {
+ return Integer.MAX_VALUE;
+ }
+
+
+ @Override
+ protected void setupSenders(List<DrillbitEndpoint> senderLocations) {
+ this.senderLocations = senderLocations;
+ }
+
+ @Override
+ protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) {
+ this.receiverLocations = receiverLocations;
+ }
+
+ @Override
+ public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+ return new HashPartitionSender(receiverMajorFragmentId, child, expr, receiverLocations);
+ }
+
+ @Override
+ public Receiver getReceiver(int minorFragmentId) {
+ return new RandomReceiver(senderMajorFragmentId, senderLocations);
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new HashToRandomExchange(child, expr);
+ }
+
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
new file mode 100644
index 0000000..eaaeaa3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockRecordReader.java
@@ -0,0 +1,113 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockColumn;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+public class MockRecordReader implements RecordReader {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
+
+ private OutputMutator output;
+ private MockScanEntry config;
+ private FragmentContext context;
+ private ValueVector<?>[] valueVectors;
+ private int recordsRead;
+
+ public MockRecordReader(FragmentContext context, MockScanEntry config) {
+ this.context = context;
+ this.config = config;
+ }
+
+ private int getEstimatedRecordSize(MockColumn[] types) {
+ int x = 0;
+ for (int i = 0; i < types.length; i++) {
+ x += TypeHelper.getSize(types[i].getMajorType());
+ }
+ return x;
+ }
+
+ private ValueVector<?> getVector(int fieldId, String name, MajorType type, int length) {
+ assert context != null : "Context shouldn't be null.";
+
+ if(type.getMode() != DataMode.REQUIRED) throw new UnsupportedOperationException();
+
+ MaterializedField f = MaterializedField.create(new SchemaPath(name), fieldId, 0, type);
+ ValueVector<?> v;
+ v = TypeHelper.getNewVector(f, context.getAllocator());
+ v.allocateNew(length);
+
+ return v;
+
+ }
+
+ @Override
+ public void setup(OutputMutator output) throws ExecutionSetupException {
+ try {
+ this.output = output;
+ int estimateRowSize = getEstimatedRecordSize(config.getTypes());
+ valueVectors = new ValueVector<?>[config.getTypes().length];
+ int batchRecordCount = 250000 / estimateRowSize;
+
+ for (int i = 0; i < config.getTypes().length; i++) {
+ logger.debug("Adding field {} of type {}", i, config.getTypes()[i]);
+ valueVectors[i] = getVector(i, config.getTypes()[i].getName(), config.getTypes()[i].getMajorType(), batchRecordCount);
+ output.addField(i, valueVectors[i]);
+ }
+ output.setNewSchema();
+ } catch (SchemaChangeException e) {
+ throw new ExecutionSetupException("Failure while setting up fields", e);
+ }
+
+ }
+
+ @Override
+ public int next() {
+ int recordSetSize = Math.min(valueVectors[0].capacity(), this.config.getRecords()- recordsRead);
+ recordsRead += recordSetSize;
+ for(ValueVector<?> v : valueVectors){
+ v.randomizeData();
+ v.setRecordCount(recordSetSize);
+ }
+ return recordSetSize;
+ }
+
+ @Override
+ public void cleanup() {
+ for (int i = 0; i < valueVectors.length; i++) {
+ try {
+ output.removeField(valueVectors[i].getField().getFieldId());
+ } catch (SchemaChangeException e) {
+ logger.warn("Failure while trying tremove field.", e);
+ }
+ valueVectors[i].close();
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
new file mode 100644
index 0000000..b821d6e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanBatchCreator.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.MockScanPOP.MockScanEntry;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class MockScanBatchCreator implements BatchCreator<MockScanPOP>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanBatchCreator.class);
+
+ @Override
+ public RecordBatch getBatch(FragmentContext context, MockScanPOP config, List<RecordBatch> children) throws ExecutionSetupException {
+ Preconditions.checkArgument(children.isEmpty());
+ List<MockScanEntry> entries = config.getReadEntries();
+ List<RecordReader> readers = Lists.newArrayList();
+ for(MockScanEntry e : entries){
+ readers.add(new MockRecordReader(context, e));
+ }
+ return new ScanBatch(context, readers.iterator());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
new file mode 100644
index 0000000..4a3a606
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockScanPOP.java
@@ -0,0 +1,193 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.ReadEntry;
+import org.apache.drill.exec.physical.base.AbstractScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("mock-scan")
+public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
+
+ private final String url;
+ private LinkedList<MockScanEntry>[] mappings;
+
+ @JsonCreator
+ public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
+ super(readEntries);
+ this.url = url;
+ }
+
+ public String getUrl() {
+ return url;
+ }
+
+
+ public static class MockScanEntry implements ReadEntry {
+
+ private final int records;
+ private final MockColumn[] types;
+ private final int recordSize;
+
+
+ @JsonCreator
+ public MockScanEntry(@JsonProperty("records") int records, @JsonProperty("types") MockColumn[] types) {
+ this.records = records;
+ this.types = types;
+ int size = 0;
+ for(MockColumn dt : types){
+ size += TypeHelper.getSize(dt.getMajorType());
+ }
+ this.recordSize = size;
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return new OperatorCost(1, 2, 1, 1);
+ }
+
+
+ public int getRecords() {
+ return records;
+ }
+
+ public MockColumn[] getTypes() {
+ return types;
+ }
+
+ @Override
+ public Size getSize() {
+ return new Size(records, recordSize);
+ }
+ }
+
+ @JsonInclude(Include.NON_NULL)
+ public static class MockColumn{
+ @JsonProperty("type") public MinorType minorType;
+ public String name;
+ public DataMode mode;
+ public Integer width;
+ public Integer precision;
+ public Integer scale;
+
+
+ @JsonCreator
+ public MockColumn(@JsonProperty("name") String name, @JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) {
+ this.name = name;
+ this.minorType = minorType;
+ this.mode = mode;
+ this.width = width;
+ this.precision = precision;
+ this.scale = scale;
+ }
+
+ @JsonProperty("type")
+ public MinorType getMinorType() {
+ return minorType;
+ }
+ public String getName() {
+ return name;
+ }
+ public DataMode getMode() {
+ return mode;
+ }
+ public Integer getWidth() {
+ return width;
+ }
+ public Integer getPrecision() {
+ return precision;
+ }
+ public Integer getScale() {
+ return scale;
+ }
+
+ @JsonIgnore
+ public MajorType getMajorType(){
+ MajorType.Builder b = MajorType.newBuilder();
+ b.setMode(mode);
+ b.setMinorType(minorType);
+ if(precision != null) b.setPrecision(precision);
+ if(width != null) b.setWidth(width);
+ if(scale != null) b.setScale(scale);
+ return b.build();
+ }
+
+ }
+
+ @Override
+ public List<EndpointAffinity> getOperatorAffinity() {
+ return Collections.emptyList();
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+ Preconditions.checkArgument(endpoints.size() <= getReadEntries().size());
+
+ mappings = new LinkedList[endpoints.size()];
+
+ int i =0;
+ for(MockScanEntry e : this.getReadEntries()){
+ if(i == endpoints.size()) i -= endpoints.size();
+ LinkedList<MockScanEntry> entries = mappings[i];
+ if(entries == null){
+ entries = new LinkedList<MockScanEntry>();
+ mappings[i] = entries;
+ }
+ entries.add(e);
+ i++;
+ }
+ }
+
+ @Override
+ public Scan<?> getSpecificScan(int minorFragmentId) {
+ assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId);
+ return new MockScanPOP(url, mappings[minorFragmentId]);
+ }
+
+ @Override
+ @JsonIgnore
+ public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+ Preconditions.checkArgument(children.isEmpty());
+ return new MockScanPOP(url, readEntries);
+
+ }
+
+}
[06/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
deleted file mode 100644
index 83ad599..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BitVector.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-/**
- * Describes a vector which holds a number of true/false values.
- */
-public class BitVector extends AbstractFixedValueVector<BitVector> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitVector.class);
-
- private final MaterializedField field;
-
- public BitVector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, 1);
- this.field = new MaterializedField(fieldId, DataType.BOOLEAN, false, ValueMode.VECTOR, this.getClass());
- }
-
- @Override
- public MaterializedField getField() {
- return field;
- }
-
-// /** Returns true or false for the specified bit index.
-// * The index should be less than the OpenBitSet size
-// */
-// public boolean get(int index) {
-// assert index >= 0 && index < this.valueCount;
-// int i = index >> 3; // div 8
-// // signed shift will keep a negative index and force an
-// // array-index-out-of-bounds-exception, removing the need for an explicit check.
-// int bit = index & 0x3f; // mod 64
-// long bitmask = 1L << bit;
-// return (data.getLong(i) & bitmask) != 0;
-// }
-
- public int getBit(int index) {
-
- assert index >= 0 && index < this.valueCount;
- int i = 8*(index >> 6); // div 8
- int bit = index & 0x3f; // mod 64
- return ((int) (data.getLong(i) >>> bit)) & 0x01;
- }
-
- /** Sets the bit at the specified index.
- * The index should be less than the OpenBitSet size.
- */
- public void set(int index) {
- assert index >= 0 && index < this.valueCount;
- int wordNum = index >> 3;
- int bit = index & 0x3f;
- long bitmask = 1L << bit;
- data.setLong(wordNum, data.getLong(wordNum) | bitmask);
- }
-
- public void clear(int index) {
- assert index >= 0 && index < this.valueCount;
- int wordNum = index >> 3;
- int bit = index & 0x03f;
- long bitmask = 1L << bit;
- data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
- }
-
-
-
- /** Clears a range of bits. Clearing past the end does not change the size of the set.
- *
- * @param startBitIndex lower index
- * @param lastBitIndex one-past the last bit to clear
- */
- private void clear2(int startBitIndex, int lastBitIndex) {
- if (lastBitIndex <= startBitIndex) return;
-
- int firstWordStart = (startBitIndex>>3);
- if (firstWordStart >= this.longWords) return;
-
- // since endIndex is one past the end, this is index of the last
- // word to be changed.
- int lastWordStart = ((lastBitIndex-1)>>3);
-
- long startmask = -1L << startBitIndex;
- long endmask = -1L >>> -lastBitIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
- // invert masks since we are clearing
- startmask = ~startmask;
- endmask = ~endmask;
-
- if (firstWordStart == lastWordStart) {
- data.setLong(firstWordStart, data.getLong(firstWordStart) & (startmask | endmask));
- return;
- }
- data.setLong(firstWordStart, data.getLong(firstWordStart) & startmask);
-
- int middle = Math.min(this.longWords, lastWordStart);
-
- for(int i =firstWordStart+8; i < middle; i += 8){
- data.setLong(i, 0L);
- }
- if (lastWordStart < this.longWords) {
- data.setLong(lastWordStart, data.getLong(lastWordStart) & endmask);
- }
- }
-
- public void setAllFalse(){
- clear(0, valueCount);
- }
-
-
- public void clear(int startIndex, int endIndex) {
- if (endIndex <= startIndex) return;
-
- int startWord = (startIndex >> 6);
- if (startWord >= longWords) return;
-
- // since endIndex is one past the end, this is index of the last
- // word to be changed.
- int endWord = ((endIndex - 1) >> 6);
-
- long startmask = -1L << startIndex;
- long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
-
- // invert masks since we are clearing
- startmask = ~startmask;
- endmask = ~endmask;
-
- int startWordPos = startWord * 8;
- if (startWord == endWord) {
- data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
- return;
- }
-
- int endWordPos = endWord * 8;
-
- data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
-
- int middle = Math.min(longWords, endWord)*8;
-
-
- for(int i =startWordPos+8; i < middle; i += 8){
- data.setLong(i, 0L);
- }
-
- if (endWordPos < startWordPos) {
- data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
deleted file mode 100644
index d8e1c80..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ByteVector.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-
-public class ByteVector extends AbstractFixedValueVector<ByteVector>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteVector.class);
-
- private final MaterializedField field;
-
- public ByteVector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, 8);
- this.field = new MaterializedField(fieldId, DataType.SIGNED_BYTE, false, ValueMode.VECTOR, this.getClass());
- }
-
- @Override
- public MaterializedField getField() {
- return field;
- }
-
- public void setByte(int index, byte b){
- data.setByte(index, b);
- }
-
- public byte getByte(int index){
- return data.getByte(index);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
new file mode 100644
index 0000000..82c86d1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed1.java
@@ -0,0 +1,43 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed1 extends AbstractFixedValueVector<Fixed1>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed1.class);
+
+ public Fixed1(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 8);
+ }
+
+ public void setByte(int index, byte b){
+ data.setByte(index, b);
+ }
+
+ public byte getByte(int index){
+ return data.getByte(index);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return getByte(index);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
new file mode 100644
index 0000000..c5f641a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed12.java
@@ -0,0 +1,35 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed12 extends AbstractFixedValueVector<Fixed12>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed12.class);
+
+ public Fixed12(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 12*8);
+ }
+
+
+ @Override
+ public Object getObject(int index) {
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
new file mode 100644
index 0000000..649832b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed16.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed16 extends AbstractFixedValueVector<Fixed16>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed16.class);
+
+ public Fixed16(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 16*8);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return null;
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
new file mode 100644
index 0000000..bd0e313
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed2.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed2 extends AbstractFixedValueVector<Fixed2>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed2.class);
+
+ public Fixed2(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 2*8);
+ }
+
+ public final void setSmallInt(int index, short value){
+ index*=2;
+ data.setShort(index, value);
+ }
+
+ public final short getSmallInt(int index){
+ index*=2;
+ return data.getShort(index);
+ }
+
+ public final void setUInt2(int index, short value){
+ setSmallInt(index, value);
+ }
+
+ public final short getUInt2(int index){
+ return getSmallInt(index);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return getSmallInt(index);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
new file mode 100644
index 0000000..650029d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed4.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed4 extends AbstractFixedValueVector<Fixed4>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed4.class);
+
+ public Fixed4(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 4*8);
+ }
+
+ public final void setInt(int index, int value){
+ index*=4;
+ data.setInt(index, value);
+ }
+
+ public final int getInt(int index){
+ index*=4;
+ return data.getInt(index);
+ }
+
+ public final void setFloat4(int index, float value){
+ index*=8;
+ data.setFloat(index, value);
+ }
+
+ public final float getFloat4(int index){
+ index*=8;
+ return data.getFloat(index);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return getInt(index);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
new file mode 100644
index 0000000..3629f5c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Fixed8.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class Fixed8 extends AbstractFixedValueVector<Fixed8>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fixed8.class);
+
+ public Fixed8(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 8*8);
+ }
+
+ public final void setBigInt(int index, long value){
+ index*=8;
+ data.setLong(index, value);
+ }
+
+ public final long getBigInt(int index){
+ index*=8;
+ return data.getLong(index);
+ }
+
+ public final void setFloat8(int index, double value){
+ index*=8;
+ data.setDouble(index, value);
+ }
+
+ public final double getFloat8(int index){
+ index*=8;
+ return data.getDouble(index);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return getBigInt(index);
+ }
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
new file mode 100644
index 0000000..594af23
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/FixedLen.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class FixedLen extends AbstractFixedValueVector<FixedLen>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FixedLen.class);
+
+
+ public FixedLen(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, field.getWidth());
+ }
+
+ public void set(ByteBuf b){
+
+ }
+
+ public void get(ByteBuf b){
+
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
deleted file mode 100644
index 779b01b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int16Vector.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class Int16Vector extends AbstractFixedValueVector<Int16Vector>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int16Vector.class);
-
- private final MaterializedField field;
-
- public Int16Vector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, 32);
- this.field = new MaterializedField(fieldId, DataType.INT16, false, ValueMode.VECTOR, this.getClass());
- }
-
- @Override
- public MaterializedField getField() {
- return field;
- }
-
- public final void set(int index, short value){
- index*=2;
- data.setShort(index, value);
- }
-
- public final short get(int index){
- index*=2;
- return data.getShort(index);
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
deleted file mode 100644
index d142367..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Int32Vector.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class Int32Vector extends AbstractFixedValueVector<Int32Vector>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Int32Vector.class);
-
- private final MaterializedField field;
-
- public Int32Vector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, 32);
- this.field = new MaterializedField(fieldId, DataType.INT32, false, ValueMode.VECTOR, this.getClass());
- }
-
- @Override
- public MaterializedField getField() {
- return field;
- }
-
- public final void set(int index, int value){
- index*=4;
- data.setInt(index, value);
- }
-
- public final int get(int index){
- index*=4;
- return data.getInt(index);
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
new file mode 100644
index 0000000..cc18538
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableFixed4.java
@@ -0,0 +1,37 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public final class NullableFixed4 extends NullableValueVector<NullableFixed4, Fixed4>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableFixed4.class);
+
+ public NullableFixed4(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, NullableFixed4.class);
+ }
+
+ @Override
+ protected Fixed4 getNewValueVector(BufferAllocator allocator) {
+ return new Fixed4(null, allocator);
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
deleted file mode 100644
index 372de13..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableInt32Vector.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public final class NullableInt32Vector extends NullableValueVector<NullableInt32Vector, Int32Vector>{
-
- public NullableInt32Vector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, NullableInt32Vector.class);
- }
-
-
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableInt32Vector.class);
-
-
- public int get(int index){
- return this.value.get(index);
- }
-
- public void set(int index, int value){
- this.value.set(index, value);
- }
-
-
- @Override
- protected Int32Vector getNewValueVector(int fieldId, BufferAllocator allocator) {
- return new Int32Vector(fieldId, allocator);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
index 8e714ed..692ab87 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/NullableValueVector.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.record.vector;
import io.netty.buffer.ByteBuf;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
import org.apache.drill.exec.record.MaterializedField;
/**
@@ -28,18 +29,16 @@ import org.apache.drill.exec.record.MaterializedField;
abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extends BaseValueVector<E>> extends BaseValueVector<T> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NullableValueVector.class);
- protected BitVector bits;
+ protected Bit bits;
protected E value;
- private final MaterializedField field;
- public NullableValueVector(int fieldId, BufferAllocator allocator, Class<T> valueClass) {
- super(fieldId, allocator);
- bits = new BitVector(fieldId, allocator);
- value = getNewValueVector(fieldId, allocator);
- this.field = value.getField().getNullableVersion(valueClass);
+ public NullableValueVector(MaterializedField field, BufferAllocator allocator, Class<T> valueClass) {
+ super(field, allocator);
+ bits = new Bit(null, allocator);
+ value = getNewValueVector(allocator);
}
- protected abstract E getNewValueVector(int fieldId, BufferAllocator allocator);
+ protected abstract E getNewValueVector(BufferAllocator allocator);
public int isNull(int index){
return bits.getBit(index);
@@ -76,5 +75,26 @@ abstract class NullableValueVector<T extends NullableValueVector<T, E>, E extend
}
+ @Override
+ public ByteBuf[] getBuffers() {
+ return new ByteBuf[]{bits.data, value.data};
+ }
+
+ @Override
+ public void setRecordCount(int recordCount) {
+ super.setRecordCount(recordCount);
+ bits.setRecordCount(recordCount);
+ value.setRecordCount(recordCount);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ if(isNull(index) == 0){
+ return null;
+ }else{
+ return value.getObject(index);
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
new file mode 100644
index 0000000..2c08551
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/RepeatMap.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class RepeatMap extends BaseValueVector<RepeatMap>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RepeatMap.class);
+
+
+ public RepeatMap(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ }
+
+ @Override
+ protected int getAllocationSize(int valueCount) {
+ return 4 * valueCount;
+ }
+
+ @Override
+ protected void childResetAllocation(int valueCount, ByteBuf buf) {
+ }
+
+ @Override
+ protected void childCloneMetadata(RepeatMap other) {
+ }
+
+ @Override
+ protected void childClear() {
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return null;
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
index e9faa93..323b55f 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/SelectionVector.java
@@ -20,12 +20,16 @@ package org.apache.drill.exec.record.vector;
import io.netty.buffer.ByteBufAllocator;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
-public class SelectionVector extends UInt16Vector{
+/**
+ * Convenience/Clarification Fixed2 wrapper.
+ */
+public class SelectionVector extends Fixed2{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SelectionVector.class);
- public SelectionVector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator);
+ public SelectionVector(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
new file mode 100644
index 0000000..8e89c41
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/TypeHelper.java
@@ -0,0 +1,250 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class TypeHelper {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeHelper.class);
+
+ private static final int WIDTH_ESTIMATE_1 = 10;
+ private static final int WIDTH_ESTIMATE_2 = 50000;
+ private static final int WIDTH_ESTIMATE_4 = 1024*1024;
+
+ public static int getSize(MajorType major){
+ switch(major.getMinorType()){
+ case TINYINT: return 1;
+ case SMALLINT: return 2;
+ case INT: return 4;
+ case BIGINT: return 8;
+ case DECIMAL4: return 4;
+ case DECIMAL8: return 8;
+ case DECIMAL12: return 12;
+ case DECIMAL16: return 16;
+ case MONEY: return 8;
+ case DATE: return 4;
+ case TIME: return 8;
+ case TIMETZ: return 12;
+ case TIMESTAMP: return 8;
+ case DATETIME: return 8;
+ case INTERVAL: return 12;
+ case FLOAT4: return 4;
+ case FLOAT8: return 8;
+ case BOOLEAN: return 1/8;
+ case FIXEDCHAR: return major.getWidth();
+ case VARCHAR1: return 1 + WIDTH_ESTIMATE_1;
+ case VARCHAR2: return 2 + WIDTH_ESTIMATE_2;
+ case VARCHAR4: return 4 + WIDTH_ESTIMATE_4;
+ case FIXEDBINARY: return major.getWidth();
+ case VARBINARY1: return 1 + WIDTH_ESTIMATE_1;
+ case VARBINARY2: return 2 + WIDTH_ESTIMATE_2;
+ case VARBINARY4: return 4 + WIDTH_ESTIMATE_4;
+ case UINT1: return 1;
+ case UINT2: return 2;
+ case UINT4: return 4;
+ case UINT8: return 8;
+ case PROTO2: return 2 + WIDTH_ESTIMATE_2;
+ case PROTO4: return 4 + WIDTH_ESTIMATE_4;
+ case MSGPACK2: return 2 + WIDTH_ESTIMATE_2;
+ case MSGPACK4: return 4 + WIDTH_ESTIMATE_4;
+ }
+ return 4;
+ }
+
+ public static Class<?> getValueVectorClass(MinorType type, DataMode mode){
+ switch(mode){
+ case OPTIONAL:
+ switch(type){
+ case REPEATMAP: return RepeatMap.class;
+ case TINYINT: return Fixed1.class;
+ case SMALLINT: return Fixed2.class;
+ case INT: return Fixed4.class;
+ case BIGINT: return Fixed8.class;
+ case DECIMAL4: return Fixed4.class;
+ case DECIMAL8: return Fixed8.class;
+ case DECIMAL12: return Fixed12.class;
+ case DECIMAL16: return Fixed16.class;
+ case MONEY: return Fixed8.class;
+ case DATE: return Fixed4.class;
+ case TIME: return Fixed8.class;
+ case TIMETZ: return Fixed12.class;
+ case TIMESTAMP: return Fixed8.class;
+ case DATETIME: return Fixed8.class;
+ case INTERVAL: return Fixed12.class;
+ case FLOAT4: return Fixed4.class;
+ case FLOAT8: return Fixed8.class;
+ case BOOLEAN: return Bit.class;
+ case FIXEDCHAR: return FixedLen.class;
+ case VARCHAR1: return VarLen1.class;
+ case VARCHAR2: return VarLen2.class;
+ case VARCHAR4: return VarLen4.class;
+ case FIXEDBINARY: return FixedLen.class;
+ case VARBINARY1: return VarLen1.class;
+ case VARBINARY2: return VarLen2.class;
+ case VARBINARY4: return VarLen4.class;
+ case UINT1: return Fixed1.class;
+ case UINT2: return Fixed2.class;
+ case UINT4: return Fixed4.class;
+ case UINT8: return Fixed8.class;
+ case PROTO2: return VarLen2.class;
+ case PROTO4: return VarLen4.class;
+ case MSGPACK2: return VarLen2.class;
+ case MSGPACK4: return VarLen4.class;
+ }
+ break;
+ case REQUIRED:
+ switch(type){
+// case TINYINT: return NullableFixed1.class;
+// case SMALLINT: return NullableFixed2.class;
+// case INT: return NullableFixed4.class;
+// case BIGINT: return NullableFixed8.class;
+// case DECIMAL4: return NullableFixed4.class;
+// case DECIMAL8: return NullableFixed8.class;
+// case DECIMAL12: return NullableFixed12.class;
+// case DECIMAL16: return NullableFixed16.class;
+// case MONEY: return NullableFixed8.class;
+// case DATE: return NullableFixed4.class;
+// case TIME: return NullableFixed8.class;
+// case TIMETZ: return NullableFixed12.class;
+// case TIMESTAMP: return NullableFixed8.class;
+// case DATETIME: return NullableFixed8.class;
+// case INTERVAL: return NullableFixed12.class;
+// case FLOAT4: return NullableFixed4.class;
+// case FLOAT8: return NullableFixed8.class;
+// case BOOLEAN: return NullableBit.class;
+// case FIXEDCHAR: return NullableFixedLen.class;
+// case VARCHAR1: return NullableVarLen1.class;
+// case VARCHAR2: return NullableVarLen2.class;
+// case VARCHAR4: return NullableVarLen4.class;
+// case FIXEDBINARY: return NullableFixedLen.class;
+// case VARBINARY1: return NullableVarLen1.class;
+// case VARBINARY2: return NullableVarLen2.class;
+// case VARBINARY4: return NullableVarLen4.class;
+// case UINT1: return NullableFixed1.class;
+// case UINT2: return NullableFixed2.class;
+// case UINT4: return NullableFixed4.class;
+// case UINT8: return NullableFixed8.class;
+// case PROTO2: return NullableVarLen2.class;
+// case PROTO4: return NullableVarLen4.class;
+// case MSGPACK2: return NullableVarLen2.class;
+// case MSGPACK4: return NullableVarLen4.class;
+ }
+ break;
+ case REPEATED:
+ switch(type){
+// case TINYINT: return RepeatedFixed1.class;
+// case SMALLINT: return RepeatedFixed2.class;
+// case INT: return RepeatedFixed4.class;
+// case BIGINT: return RepeatedFixed8.class;
+// case DECIMAL4: return RepeatedFixed4.class;
+// case DECIMAL8: return RepeatedFixed8.class;
+// case DECIMAL12: return RepeatedFixed12.class;
+// case DECIMAL16: return RepeatedFixed16.class;
+// case MONEY: return RepeatedFixed8.class;
+// case DATE: return RepeatedFixed4.class;
+// case TIME: return RepeatedFixed8.class;
+// case TIMETZ: return RepeatedFixed12.class;
+// case TIMESTAMP: return RepeatedFixed8.class;
+// case DATETIME: return RepeatedFixed8.class;
+// case INTERVAL: return RepeatedFixed12.class;
+// case FLOAT4: return RepeatedFixed4.class;
+// case FLOAT8: return RepeatedFixed8.class;
+// case BOOLEAN: return RepeatedBit.class;
+// case FIXEDCHAR: return RepeatedFixedLen.class;
+// case VARCHAR1: return RepeatedVarLen1.class;
+// case VARCHAR2: return RepeatedVarLen2.class;
+// case VARCHAR4: return RepeatedVarLen4.class;
+// case FIXEDBINARY: return RepeatedFixedLen.class;
+// case VARBINARY1: return RepeatedVarLen1.class;
+// case VARBINARY2: return RepeatedVarLen2.class;
+// case VARBINARY4: return RepeatedVarLen4.class;
+// case UINT1: return RepeatedFixed1.class;
+// case UINT2: return RepeatedFixed2.class;
+// case UINT4: return RepeatedFixed4.class;
+// case UINT8: return RepeatedFixed8.class;
+// case PROTO2: return RepeatedVarLen2.class;
+// case PROTO4: return RepeatedVarLen4.class;
+// case MSGPACK2: return RepeatedVarLen2.class;
+// case MSGPACK4: return RepeatedVarLen4.class;
+ }
+ break;
+ default:
+ break;
+
+ }
+ throw new UnsupportedOperationException();
+ }
+
+
+ public static ValueVector<?> getNewVector(MaterializedField field, BufferAllocator allocator){
+ MajorType type = field.getType();
+ switch(type.getMode()){
+ case REQUIRED:
+ switch(type.getMinorType()){
+ case TINYINT: return new Fixed1(field, allocator);
+ case SMALLINT: return new Fixed2(field, allocator);
+ case INT: return new Fixed4(field, allocator);
+ case BIGINT: return new Fixed8(field, allocator);
+ case DECIMAL4: return new Fixed4(field, allocator);
+ case DECIMAL8: return new Fixed8(field, allocator);
+ case DECIMAL12: return new Fixed12(field, allocator);
+ case DECIMAL16: return new Fixed16(field, allocator);
+ case MONEY: return new Fixed8(field, allocator);
+ case DATE: return new Fixed4(field, allocator);
+ case TIME: return new Fixed8(field, allocator);
+ case TIMETZ: return new Fixed12(field, allocator);
+ case TIMESTAMP: return new Fixed8(field, allocator);
+ case DATETIME: return new Fixed8(field, allocator);
+ case INTERVAL: return new Fixed12(field, allocator);
+ case FLOAT4: return new Fixed4(field, allocator);
+ case FLOAT8: return new Fixed8(field, allocator);
+ case BOOLEAN: return new Bit(field, allocator);
+ case FIXEDCHAR: return new FixedLen(field, allocator);
+ case VARCHAR1: return new VarLen1(field, allocator);
+ case VARCHAR2: return new VarLen2(field, allocator);
+ case VARCHAR4: return new VarLen4(field, allocator);
+ case FIXEDBINARY: return new FixedLen(field, allocator);
+ case VARBINARY1: return new VarLen1(field, allocator);
+ case VARBINARY2: return new VarLen2(field, allocator);
+ case VARBINARY4: return new VarLen4(field, allocator);
+ case UINT1: return new Fixed1(field, allocator);
+ case UINT2: return new Fixed2(field, allocator);
+ case UINT4: return new Fixed4(field, allocator);
+ case UINT8: return new Fixed8(field, allocator);
+ case PROTO2: return new VarLen2(field, allocator);
+ case PROTO4: return new VarLen4(field, allocator);
+ case MSGPACK2: return new VarLen2(field, allocator);
+ case MSGPACK4: return new VarLen4(field, allocator);
+ }
+ break;
+ case REPEATED:
+ break;
+ case OPTIONAL:
+ break;
+ default:
+ break;
+
+ }
+ throw new UnsupportedOperationException();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
deleted file mode 100644
index 87c306b..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/UInt16Vector.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.record.vector;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.memory.BufferAllocator;
-import org.apache.drill.exec.record.MaterializedField;
-
-public class UInt16Vector extends AbstractFixedValueVector<Int32Vector>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UInt16Vector.class);
-
- private final MaterializedField field;
-
- public UInt16Vector(int fieldId, BufferAllocator allocator) {
- super(fieldId, allocator, 16);
- this.field = new MaterializedField(fieldId, DataType.UINT16, false, ValueMode.VECTOR, this.getClass());
- }
-
- @Override
- public MaterializedField getField() {
- return field;
- }
-
- public final void set(int index, char value){
- index*=2;
- data.setChar(index, value);
- }
-
- public final char get(int index){
- index*=2;
- return data.getChar(index);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
index 76b0e90..8a5a822 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/ValueVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
import java.io.Closeable;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
import org.apache.drill.exec.record.MaterializedField;
/**
@@ -44,6 +45,13 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
public abstract void allocateNew(int valueCount);
/**
+ * Update the value vector to the provided record information.
+ * @param metadata
+ * @param data
+ */
+ public abstract void setTo(FieldMetadata metadata, ByteBuf data);
+
+ /**
* Zero copy move of data from this vector to the target vector. Any future access to this vector without being
* populated by a new vector will cause problems.
*
@@ -52,19 +60,19 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
public abstract void transferTo(T vector);
/**
- * Return the underlying buffer. Note that this doesn't impact the reference counts for this buffer so it only should be
+ * Return the underlying buffers associated with this vector. Note that this doesn't impact the reference counts for this buffer so it only should be
* used for in context access. Also note that this buffer changes regularly thus external classes shouldn't hold a
- * reference to it.
+ * reference to it (unless they change it).
*
* @return The underlying ByteBuf.
*/
- public abstract ByteBuf getBuffer();
+ public abstract ByteBuf[] getBuffers();
/**
- * Returns the number of value contained within this vector.
+ * Returns the maximum number of values contained within this vector.
* @return Vector size
*/
- public abstract int size();
+ public abstract int capacity();
/**
@@ -79,4 +87,32 @@ public interface ValueVector<T extends ValueVector<T>> extends Closeable {
*/
public abstract MaterializedField getField();
+ /**
+ * Define the number of records that are in this value vector.
+ * @param recordCount Number of records active in this vector. Used for purposes such as getting a writable range of the data.
+ */
+ public abstract void setRecordCount(int recordCount);
+ public abstract int getRecordCount();
+
+
+ /**
+ * Get the metadata for this field.
+ * @return
+ */
+ public abstract FieldMetadata getMetadata();
+
+ /**
+ * Debug interface to get values per record.
+ * @param index The record index.
+ * @return The value in the vector.
+ */
+ public Object getObject(int index);
+
+
+ /**
+ * Useful for generating random data.
+ */
+ public void randomizeData();
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
new file mode 100644
index 0000000..d87029d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen1.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen1 extends VariableVector<VarLen1, Fixed1>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen1.class);
+
+ public VarLen1(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ }
+
+ @Override
+ protected Fixed1 getNewLengthVector(BufferAllocator allocator) {
+ return new Fixed1(null, allocator);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
new file mode 100644
index 0000000..ebd440a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen2.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen2 extends VariableVector<VarLen2, Fixed2>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen2.class);
+
+ public VarLen2(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ }
+
+ @Override
+ protected Fixed2 getNewLengthVector(BufferAllocator allocator) {
+ return new Fixed2(null, allocator);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
new file mode 100644
index 0000000..b3cd712
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VarLen4.java
@@ -0,0 +1,36 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.record.MaterializedField;
+
+public class VarLen4 extends VariableVector<VarLen4, Fixed4>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VarLen4.class);
+
+ public VarLen4(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ }
+
+ @Override
+ protected Fixed4 getNewLengthVector(BufferAllocator allocator) {
+ return new Fixed4(null, allocator);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
index dd84c94..4247f14 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/VariableVector.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
import org.apache.drill.exec.memory.BufferAllocator;
import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
/**
* A vector of variable length bytes. Constructed as a vector of lengths or positions and a vector of values. Random access is only possible if the variable vector stores positions as opposed to lengths.
@@ -29,18 +30,16 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VariableVector.class);
- protected E lengthVector;
+ protected final E lengthVector;
private ByteBuf values = DeadBuf.DEAD_BUFFER;
protected int expectedValueLength;
- private final boolean hasPositions;
- public VariableVector(int fieldId, BufferAllocator allocator, boolean hasPositions) {
- super(fieldId, allocator);
- this.lengthVector = getNewLengthVector(fieldId, allocator);
- this.hasPositions = hasPositions;
+ public VariableVector(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator);
+ this.lengthVector = getNewLengthVector(allocator);
}
- protected abstract E getNewLengthVector(int fieldId, BufferAllocator allocator);
+ protected abstract E getNewLengthVector(BufferAllocator allocator);
@Override
protected int getAllocationSize(int valueCount) {
@@ -67,12 +66,28 @@ public abstract class VariableVector<T extends VariableVector<T, E>, E extends B
values.release();
values = DeadBuf.DEAD_BUFFER;
}
- }
+ }
+
- public boolean hasPositions(){
- return hasPositions;
+ @Override
+ public ByteBuf[] getBuffers() {
+ return new ByteBuf[]{lengthVector.data, values};
}
+
+ @Override
+ public void setRecordCount(int recordCount) {
+ super.setRecordCount(recordCount);
+ lengthVector.setRecordCount(recordCount);
+ }
+ public void setTotalBytes(int totalBytes){
+ values.writerIndex(totalBytes);
+ }
+
+ @Override
+ public Object getObject(int index) {
+ return null;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
new file mode 100644
index 0000000..859d385
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/AbstractHandshakeHandler.java
@@ -0,0 +1,57 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+
+import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+
+public abstract class AbstractHandshakeHandler<T extends MessageLite> extends
+ ChannelInboundMessageHandlerAdapter<InboundRpcMessage> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractHandshakeHandler.class);
+
+ protected final EnumLite handshakeType;
+ protected final Parser<T> parser;
+ protected int coordinationId;
+
+ public AbstractHandshakeHandler(EnumLite handshakeType, Parser<T> parser) {
+ super();
+ this.handshakeType = handshakeType;
+ this.parser = parser;
+ }
+
+ @Override
+ public final void messageReceived(ChannelHandlerContext ctx, InboundRpcMessage inbound) throws Exception {
+ coordinationId = inbound.coordinationId;
+ ctx.channel().pipeline().remove(this);
+ if (inbound.rpcType != handshakeType.getNumber())
+ throw new RpcException(String.format("Handshake failure. Expected %s[%d] but received number [%d]",
+ handshakeType, handshakeType.getNumber(), inbound.rpcType));
+
+ T msg = parser.parseFrom(inbound.getProtobufBodyAsIS());
+ consumeHandshake(ctx.channel(), msg);
+
+ }
+
+ protected abstract void consumeHandshake(Channel c, T msg) throws Exception;
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
new file mode 100644
index 0000000..a241880
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/Acks.java
@@ -0,0 +1,27 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+
+public class Acks {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Acks.class);
+
+ public static final Ack OK = Ack.newBuilder().setOk(true).build();
+ public static final Ack FAIL = Ack.newBuilder().setOk(false).build();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
index c62d445..0ff2b9d 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClient.java
@@ -18,23 +18,34 @@
package org.apache.drill.exec.rpc;
import io.netty.bootstrap.Bootstrap;
+import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelOption;
import io.netty.channel.EventLoopGroup;
import io.netty.channel.socket.SocketChannel;
import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.GenericFutureListener;
+import com.google.common.util.concurrent.SettableFuture;
import com.google.protobuf.Internal.EnumLite;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
-public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
+public abstract class BasicClient<T extends EnumLite, R extends RemoteConnection> extends RpcBus<T, R> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClient.class);
private Bootstrap b;
private volatile boolean connect = false;
+ protected R connection;
+ private EventLoopGroup eventLoop;
- public BasicClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+ public BasicClient(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+ super(rpcMapping);
+ this.eventLoop = eventLoopGroup;
+
b = new Bootstrap() //
.group(eventLoopGroup) //
.channel(NioSocketChannel.class) //
@@ -42,40 +53,132 @@ public abstract class BasicClient<T extends EnumLite> extends RpcBus<T> {
.option(ChannelOption.SO_RCVBUF, 1 << 17) //
.option(ChannelOption.SO_SNDBUF, 1 << 17) //
.handler(new ChannelInitializer<SocketChannel>() {
-
+
@Override
protected void initChannel(SocketChannel ch) throws Exception {
- ch.closeFuture().addListener(getCloseHandler(ch));
-
+ logger.debug("initializing client connection.");
+ connection = initRemoteConnection(ch);
+ ch.closeFuture().addListener(getCloseHandler(connection));
+
ch.pipeline().addLast( //
new ZeroCopyProtobufLengthDecoder(), //
- new RpcDecoder(), //
- new RpcEncoder(), //
- new InboundHandler(ch), //
+ new RpcDecoder(rpcConfig.getName()), //
+ new RpcEncoder(rpcConfig.getName()), //
+ getHandshakeHandler(), //
+ new InboundHandler(connection), //
new RpcExceptionHandler() //
);
- channel = ch;
connect = true;
}
}) //
-
- ;
+
+ ;
+ }
+
+ protected abstract ClientHandshakeHandler<?> getHandshakeHandler();
+
+ protected abstract class ClientHandshakeHandler<T extends MessageLite> extends AbstractHandshakeHandler<T> {
+ private Class<T> responseType;
+
+ public ClientHandshakeHandler(EnumLite handshakeType, Class<T> responseType, Parser<T> parser) {
+ super(handshakeType, parser);
+ this.responseType = responseType;
+ }
+
+ @Override
+ protected final void consumeHandshake(Channel c, T msg) throws Exception {
+ validateHandshake(msg);
+ queue.getFuture(handshakeType.getNumber(), coordinationId, responseType).setValue(msg);
+ }
+
+ protected abstract void validateHandshake(T msg) throws Exception;
+
+ }
+
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(Channel channel) {
+ return new ChannelClosedHandler();
+ }
+
+ protected final <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFutureImpl<RECEIVE> send(
+ T connection, T rpcType, SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+ throw new UnsupportedOperationException(
+ "This shouldn't be used in client mode as a client only has a single connection.");
+ }
+
+ protected <SEND extends MessageLite, RECEIVE extends MessageLite> DrillRpcFuture<RECEIVE> send(T rpcType,
+ SEND protobufBody, Class<RECEIVE> clazz, ByteBuf... dataBodies) throws RpcException {
+ return super.send(connection, rpcType, protobufBody, clazz, dataBodies);
}
@Override
public boolean isClient() {
return true;
}
-
- public ChannelFuture connectAsClient(String host, int port) throws InterruptedException {
- ChannelFuture f = b.connect(host, port).sync();
- connect = !connect;
- return f;
+
+ /**
+ * TODO: This is a horrible hack to manage deadlock caused by creation of BitClient within BitCom. Should be cleaned up.
+ */
+ private class HandshakeThread<SEND extends MessageLite, RECEIVE extends MessageLite> extends Thread {
+ final SettableFuture<RECEIVE> future;
+ T handshakeType;
+ SEND handshakeValue;
+ String host;
+ int port;
+ Class<RECEIVE> responseClass;
+
+ public HandshakeThread(T handshakeType, SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) {
+ super();
+ assert host != null && !host.isEmpty();
+ assert port > 0;
+ logger.debug("Creating new handshake thread to connec to {}:{}", host, port);
+ this.setName(String.format("handshake thread for %s", handshakeType.getClass().getCanonicalName()));
+ future = SettableFuture.create();
+ this.handshakeType = handshakeType;
+ this.handshakeValue = handshakeValue;
+ this.host = host;
+ this.port = port;
+ this.responseClass = responseClass;
+ }
+
+ @Override
+ public void run() {
+ try {
+ logger.debug("Starting to get client connection on host {}, port {}.", host, port);
+
+ ChannelFuture f = b.connect(host, port);
+ f.sync();
+ if (connection == null) throw new RpcException("Failure while attempting to connect to server.");
+ connect = !connect;
+ logger.debug("Client connected, sending handshake.");
+ DrillRpcFuture<RECEIVE> fut = send(handshakeType, handshakeValue, responseClass);
+ future.set(fut.checkedGet());
+ logger.debug("Got bit client connection.");
+ } catch (Exception e) {
+ logger.debug("Failed to get client connection.", e);
+ future.setException(e);
+ }
+ }
+
+ }
+
+ protected <SEND extends MessageLite, RECEIVE extends MessageLite> RECEIVE connectAsClient(T handshakeType,
+ SEND handshakeValue, String host, int port, Class<RECEIVE> responseClass) throws InterruptedException,
+ RpcException {
+
+
+ HandshakeThread<SEND, RECEIVE> ht = new HandshakeThread<SEND, RECEIVE>(handshakeType, handshakeValue, host, port, responseClass);
+ ht.start();
+ try{
+ return ht.future.get();
+ }catch(Exception e){
+ throw new RpcException(e);
+ }
+
}
public void close() {
logger.debug("Closing client");
- b.shutdown();
+ connection.getChannel().close();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
new file mode 100644
index 0000000..0e62f14
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicClientWithConnection.java
@@ -0,0 +1,64 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.EventLoopGroup;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import org.apache.drill.exec.rpc.BasicClientWithConnection.ServerConnection;
+
+import com.google.protobuf.Internal.EnumLite;
+
+public abstract class BasicClientWithConnection<T extends EnumLite> extends BasicClient<T, ServerConnection>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BasicClientWithConnection.class);
+
+ public BasicClientWithConnection(RpcConfig rpcMapping, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
+ super(rpcMapping, alloc, eventLoopGroup);
+ }
+
+ @Override
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(ServerConnection clientConnection) {
+ return getCloseHandler(clientConnection.getChannel());
+ }
+
+ @Override
+ protected Response handle(ServerConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return handle(rpcType, pBody, dBody);
+ }
+
+ protected abstract Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException ;
+
+ @Override
+ public ServerConnection initRemoteConnection(Channel channel) {
+ return new ServerConnection(channel);
+ }
+
+ public static class ServerConnection extends RemoteConnection{
+
+ public ServerConnection(Channel channel) {
+ super(channel);
+ }
+
+ }
+
+
+}
Re: [09/13] Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.
Posted by Ted Dunning <te...@gmail.com>.
Sent from my iPhone
On May 13, 2013, at 18:52, jacques@apache.org wrote:
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> new file mode 100644
> index 0000000..0044628
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
> @@ -0,0 +1,58 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.io.IOException;
> +import java.util.Collection;
> +
> +import org.apache.drill.common.logical.data.Scan;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +import org.apache.drill.exec.store.AbstractStorageEngine;
> +import org.apache.drill.exec.store.RecordReader;
> +import org.apache.drill.exec.store.StorageEngine;
> +import org.apache.drill.exec.store.StorageEngine.ReadEntry;
> +
> +import com.google.common.collect.ListMultimap;
> +
> +public class MockStorageEngine extends AbstractStorageEngine{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
> +
> + @Override
> + public boolean supportsRead() {
> + return true;
> + }
> +
> + @Override
> + public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
> + return null;
> + }
> +
> + @Override
> + public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
> + return null;
> + }
> +
> + @Override
> + public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
> + return null;
> + }
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> new file mode 100644
> index 0000000..639d0d2
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
> @@ -0,0 +1,75 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.EndpointAffinity;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractStore;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.Store;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("mock-store")
> +public class MockStorePOP extends AbstractStore {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
> +
> + @JsonCreator
> + public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
> + super(child);
> + }
> +
> + public int getMaxWidth() {
> + return 1;
> + }
> +
> + @Override
> + public List<EndpointAffinity> getOperatorAffinity() {
> + return Collections.emptyList();
> + }
> +
> + @Override
> + public void applyAssignments(List<DrillbitEndpoint> endpoints) {
> +
> + }
> +
> + @Override
> + public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
> + return new MockStorePOP(child);
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + return new OperatorCost(1,getSize().getRecordCount()*getSize().getRecordSize(),1,1);
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new MockStorePOP(child);
> + }
> +
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> new file mode 100644
> index 0000000..eb77eeb
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
> @@ -0,0 +1,47 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import org.apache.drill.common.expression.LogicalExpression;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +
> +public class PartitionRange {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionRange.class);
> +
> + private LogicalExpression start;
> + private LogicalExpression finish;
> +
> + @JsonCreator
> + public PartitionRange(@JsonProperty("start") LogicalExpression start, @JsonProperty("finish") LogicalExpression finish) {
> + super();
> + this.start = start;
> + this.finish = finish;
> + }
> +
> + public LogicalExpression getStart() {
> + return start;
> + }
> +
> + public LogicalExpression getFinish() {
> + return finish;
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> new file mode 100644
> index 0000000..e869393
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
> @@ -0,0 +1,72 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.logical.data.NamedExpression;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSingle;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("project")
> +public class Project extends AbstractSingle{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
> +
> + private final List<NamedExpression> exprs;
> +
> + @JsonCreator
> + public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
> + super(child);
> + this.exprs = exprs;
> + }
> +
> + public List<NamedExpression> getExprs() {
> + return exprs;
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> + return physicalVisitor.visitProject(this, value);
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
> + }
> +
> + @Override
> + public Size getSize() {
> + //TODO: This should really change the row width...
> + return child.getSize();
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new Project(exprs, child);
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> new file mode 100644
> index 0000000..ed41586
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
> @@ -0,0 +1,83 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractReceiver;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +import com.google.common.base.Preconditions;
> +
> +@JsonTypeName("random-receiver")
> +public class RandomReceiver extends AbstractReceiver{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
> +
> + private List<DrillbitEndpoint> senders;
> +
> + @JsonCreator
> + public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId, @JsonProperty("senders") List<DrillbitEndpoint> senders){
> + super(oppositeMajorFragmentId);
> + this.senders = senders;
> + }
> +
> + @Override
> + public List<DrillbitEndpoint> getProvidingEndpoints() {
> + return senders;
> + }
> +
> + @Override
> + public boolean supportsOutOfOrderExchange() {
> + return true;
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + //TODO: deal with receiver cost through exchange.
> + return new OperatorCost(1,1,1,1);
> + }
> +
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> + return physicalVisitor.visitRandomReceiver(this, value);
> + }
> +
> + @Override
> + public Size getSize() {
> + //TODO: deal with size info through exchange.
> + return new Size(1,1);
> + }
> +
> + @Override
> + public int getOppositeMajorFragmentId() {
> + return 0;
> + }
> +
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> new file mode 100644
> index 0000000..7d64dba
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
> @@ -0,0 +1,72 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.physical.base.AbstractSender;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("range-sender")
> +public class RangeSender extends AbstractSender{
> +
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
> +
> + List<EndpointPartition> partitions;
> +
> + @JsonCreator
> + public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
> + super(oppositeMajorFragmentId, child);
> + this.partitions = partitions;
> + }
> +
> + @Override
> + public List<DrillbitEndpoint> getDestinations() {
> + return null;
> + }
> +
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new RangeSender(oppositeMajorFragmentId, child, partitions);
> + }
> +
> +
> + public static class EndpointPartition{
> + private final PartitionRange range;
> + private final DrillbitEndpoint endpoint;
> +
> + @JsonCreator
> + public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
> + super();
> + this.range = range;
> + this.endpoint = endpoint;
> + }
> + public PartitionRange getRange() {
> + return range;
> + }
> + public DrillbitEndpoint getEndpoint() {
> + return endpoint;
> + }
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> new file mode 100644
> index 0000000..86a201d
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
> @@ -0,0 +1,106 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.EndpointAffinity;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractStore;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Root;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.physical.base.Store;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JacksonInject;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +import com.google.common.base.Preconditions;
> +
> +@JsonTypeName("screen")
> +public class Screen extends AbstractStore implements Root{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
> +
> + private final DrillbitEndpoint endpoint;
> +
> + public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
> + super(child);
> + this.endpoint = endpoint;
> + }
> +
> + @Override
> + public List<EndpointAffinity> getOperatorAffinity() {
> + return Collections.singletonList(new EndpointAffinity(endpoint, 1000000000000l));
> + }
> +
> + @Override
> + public int getMaxWidth() {
> + return 1;
> + }
> +
> + @Override
> + public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
> + // we actually don't have to do anything since nothing should have changed. we'll check just check that things
> + // didn't get screwed up.
> + if (endpoints.size() != 1) throw new PhysicalOperatorSetupException("A Screen operator can only be assigned to a single node.");
> + DrillbitEndpoint endpoint = endpoints.iterator().next();
> + logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
> + if (!endpoint.equals(this.endpoint)) {
> + throw new PhysicalOperatorSetupException(String.format(
> + "A Screen operator can only be assigned to its home node. Expected endpoint %s, Actual endpoint: %s",
> + this.endpoint, endpoint));
> + }
> + }
> +
> + @Override
> + public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
> + return new Screen(child, endpoint);
> + }
> +
> + @JsonIgnore
> + public DrillbitEndpoint getEndpoint() {
> + return endpoint;
> + }
> +
> + @Override
> + public String toString() {
> + return "Screen [endpoint=" + endpoint + ", getChild()=" + getChild() + "]";
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new Screen(child, endpoint);
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> + return physicalVisitor.visitScreen(this, value);
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> new file mode 100644
> index 0000000..79d937a
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
> @@ -0,0 +1,78 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.graph.GraphVisitor;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSender;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +/**
> + * Sender that pushes all data to a single destination node.
> + */
> +@JsonTypeName("single-sender")
> +public class SingleSender extends AbstractSender {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
> +
> + private final DrillbitEndpoint destination;
> +
> + @JsonCreator
> + public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
> + super(oppositeMajorFragmentId, child);
> + this.destination = destination;
> + }
> +
> + @Override
> + @JsonIgnore
> + public List<DrillbitEndpoint> getDestinations() {
> + return Collections.singletonList(destination);
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + long recordSize = child.getSize().getRecordSize() * child.getSize().getRecordCount();
> + return new OperatorCost((float) recordSize, recordSize, 0, child.getSize().getRecordCount()/(1<<16));
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new SingleSender(oppositeMajorFragmentId, child, destination);
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
> + return physicalVisitor.visitSingleSender(this, value);
> + }
> +
> +
> + public DrillbitEndpoint getDestination() {
> + return destination;
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> new file mode 100644
> index 0000000..e4ece6b
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
> @@ -0,0 +1,86 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.defs.OrderDef;
> +import org.apache.drill.exec.physical.OperatorCost;
> +import org.apache.drill.exec.physical.base.AbstractSingle;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Size;
> +
> +import com.fasterxml.jackson.annotation.JsonCreator;
> +import com.fasterxml.jackson.annotation.JsonIgnore;
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("sort")
> +public class Sort extends AbstractSingle{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
> +
> + private final List<OrderDef> orderings;
> + private boolean reverse = false;
> +
> + @JsonCreator
> + public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<OrderDef> orderings, @JsonProperty("reverse") boolean reverse) {
> + super(child);
> + this.orderings = orderings;
> + this.reverse = reverse;
> + }
> +
> + public List<OrderDef> getOrderings() {
> + return orderings;
> + }
> +
> + public boolean getReverse() {
> + return reverse;
> + }
> +
> + @Override
> + public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
> + return physicalVisitor.visitSort(this, value);
> + }
> +
> + @Override
> + public OperatorCost getCost() {
> + Size childSize = child.getSize();
> + long n = childSize.getRecordCount();
> + long width = childSize.getRecordSize();
> +
> + //TODO: Magic Number, let's assume 1/10 of data can fit in memory.
> + int k = 10;
> + long n2 = n/k;
> + double cpuCost =
> + k * n2 * (Math.log(n2)/Math.log(2)) + //
> + n * (Math.log(k)/Math.log(2));
> + double diskCost = n*width*2;
> +
> + return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new Sort(child, orderings, reverse);
> + }
> +
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> new file mode 100644
> index 0000000..56467ce
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
> @@ -0,0 +1,79 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.config;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
> +import org.apache.drill.exec.physical.base.AbstractExchange;
> +import org.apache.drill.exec.physical.base.ExchangeCost;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.PhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Receiver;
> +import org.apache.drill.exec.physical.base.Sender;
> +import org.apache.drill.exec.physical.base.Size;
> +import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
> +
> +import com.fasterxml.jackson.annotation.JsonProperty;
> +import com.fasterxml.jackson.annotation.JsonTypeName;
> +
> +@JsonTypeName("union-exchange")
> +public class UnionExchange extends AbstractExchange{
> +
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionExchange.class);
> +
> + private List<DrillbitEndpoint> senderLocations;
> + private DrillbitEndpoint destinationLocation;
> +
> + public UnionExchange(@JsonProperty("child") PhysicalOperator child) {
> + super(child);
> + }
> +
> + @Override
> + public void setupSenders(List<DrillbitEndpoint> senderLocations) {
> + this.senderLocations = senderLocations;
> + }
> +
> + @Override
> + protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
> + if(receiverLocations.size() != 1) throw new PhysicalOperatorSetupException("A Union Exchange only supports a single receiver endpoint.");
> + this.destinationLocation = receiverLocations.iterator().next();
> + }
> +
> + @Override
> + public Sender getSender(int minorFragmentId, PhysicalOperator child) {
> + return new SingleSender(this.receiverMajorFragmentId, child, destinationLocation);
> + }
> +
> + @Override
> + public Receiver getReceiver(int minorFragmentId) {
> + return new RandomReceiver(this.senderMajorFragmentId, senderLocations);
> + }
> +
> + @Override
> + public int getMaxSendWidth() {
> + return Integer.MAX_VALUE;
> + }
> +
> + @Override
> + protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
> + return new UnionExchange(child);
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> new file mode 100644
> index 0000000..9a7df56
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface BatchCreator<T extends PhysicalOperator> {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCreator.class);
> +
> + public RecordBatch getBatch(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> new file mode 100644
> index 0000000..6592ca1
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
> @@ -0,0 +1,108 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public abstract class FilterRecordBatch implements RecordBatch {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
> +
> + private RecordBatch incoming;
> + private SelectionVector selectionVector;
> + private BatchSchema schema;
> + private FilteringRecordBatchTransformer transformer;
> + private int outstanding;
> +
> + public FilterRecordBatch(RecordBatch batch) {
> + this.incoming = batch;
> + }
> +
> + @Override
> + public FragmentContext getContext() {
> + return incoming.getContext();
> + }
> +
> + @Override
> + public BatchSchema getSchema() {
> + return schema;
> + }
> +
> + @Override
> + public int getRecordCount() {
> + return 0;
> + }
> +
> + @Override
> + public void kill() {
> + incoming.kill();
> + }
> +
> + @Override
> + public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> + return null;
> + }
> +
> + abstract int applyFilter(SelectionVector vector, int count);
> +
> + /**
> + * Release all assets.
> + */
> + private void close() {
> +
> + }
> +
> + @Override
> + public IterOutcome next() {
> + while (true) {
> + IterOutcome o = incoming.next();
> + switch (o) {
> + case OK_NEW_SCHEMA:
> + transformer = incoming.getContext().getFilteringExpression(null);
> + schema = transformer.getSchema();
> + // fall through to ok.
> + case OK:
> +
> + case NONE:
> + case STOP:
> + close();
> + return IterOutcome.STOP;
> + }
> +
> + if (outstanding > 0) {
> + // move data to output location.
> +
> + for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
> +
> + }
> + }
> +
> + // make sure the bit vector is as large as the current record batch.
> + if (selectionVector.capacity() < incoming.getRecordCount()) {
> + selectionVector.allocateNew(incoming.getRecordCount());
> + }
> +
> + return null;
> + }
> +
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> new file mode 100644
> index 0000000..191521a
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
> @@ -0,0 +1,58 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.vector.SelectionVector;
> +
> +public abstract class FilteringRecordBatchTransformer {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
> +
> + final RecordBatch incoming;
> + final SelectionVector selectionVector;
> + final BatchSchema schema;
> +
> + public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
> + super();
> + this.incoming = incoming;
> + this.selectionVector = selectionVector;
> + this.schema = innerSetup();
> + }
> +
> + public abstract BatchSchema innerSetup();
> +
> + /**
> + * Applies the filter to the selection index. Ignores any values in the selection vector, instead creating a.
> + * @return
> + */
> + public abstract int apply();
> +
> + /**
> + * Applies the filter to the selection index. Utilizes the existing selection index and only evaluates on those records.
> + * @return
> + */
> + public abstract int applyWithSelection();
> +
> + public BatchSchema getSchema() {
> + return schema;
> + }
> +
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> new file mode 100644
> index 0000000..d98c107
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
> @@ -0,0 +1,102 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.Collections;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
> +import org.apache.drill.exec.physical.base.FragmentRoot;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.physical.base.Scan;
> +import org.apache.drill.exec.physical.config.MockScanBatchCreator;
> +import org.apache.drill.exec.physical.config.MockScanPOP;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +import com.google.common.base.Preconditions;
> +import com.google.common.collect.Lists;
> +
> +public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
> +
> + private MockScanBatchCreator msc = new MockScanBatchCreator();
> + private ScreenCreator sc = new ScreenCreator();
> + private RandomReceiverCreator rrc = new RandomReceiverCreator();
> + private SingleSenderCreator ssc = new SingleSenderCreator();
> + private RootExec root = null;
> +
> + private ImplCreator(){}
> +
> + public RootExec getRoot(){
> + return root;
> + }
> +
> +
> + @Override
> + public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
> + Preconditions.checkNotNull(scan);
> + Preconditions.checkNotNull(context);
> +
> + if(scan instanceof MockScanPOP){
> + return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
> + }else{
> + return super.visitScan(scan, context);
> + }
> +
> + }
> +
> + @Override
> + public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
> + Preconditions.checkArgument(root == null);
> + root = sc.getRoot(context, op, getChildren(op, context));
> + return null;
> + }
> +
> +
> +
> + @Override
> + public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
> + root = ssc.getRoot(context, op, getChildren(op, context));
> + return null;
> + }
> +
> + @Override
> + public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
> + return rrc.getBatch(context, op, null);
> + }
> +
> + private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
> + List<RecordBatch> children = Lists.newArrayList();
> + for(PhysicalOperator child : op){
> + children.add(child.accept(this, context));
> + }
> + return children;
> + }
> +
> + public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
> + ImplCreator i = new ImplCreator();
> + root.accept(i, context);
> + if(i.root == null) throw new ExecutionSetupException("The provided fragment did not have a root node that correctly created a RootExec value.");
> + return i.getRoot();
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> new file mode 100644
> index 0000000..ce0cf66
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
> @@ -0,0 +1,28 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public interface OutputMutator {
> + public void removeField(int fieldId) throws SchemaChangeException;
> + public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
> + public void setNewSchema() throws SchemaChangeException ;
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> new file mode 100644
> index 0000000..9995bc2
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
> @@ -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.drill.exec.physical.impl;
> +
> +import java.lang.annotation.ElementType;
> +import java.lang.annotation.Retention;
> +import java.lang.annotation.RetentionPolicy;
> +import java.lang.annotation.Target;
> +
> +@Target({ElementType.TYPE})
> +@Retention(RetentionPolicy.RUNTIME)
> +public @interface PhysicalConfig {
> + Class<?> value();
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> new file mode 100644
> index 0000000..4b991f8
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
> @@ -0,0 +1,46 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.RandomReceiver;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.work.batch.IncomingBuffers;
> +import org.apache.drill.exec.work.batch.RawBatchBuffer;
> +
> +public class RandomReceiverCreator implements BatchCreator<RandomReceiver>{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiverCreator.class);
> +
> + @Override
> + public RecordBatch getBatch(FragmentContext context, RandomReceiver receiver, List<RecordBatch> children)
> + throws ExecutionSetupException {
> + assert children == null || children.isEmpty();
> + IncomingBuffers bufHolder = context.getBuffers();
> + assert bufHolder != null : "IncomingBuffers must be defined for any place a receiver is declared.";
> +
> + RawBatchBuffer[] buffers = bufHolder.getBuffers(receiver.getOppositeMajorFragmentId());
> + assert buffers.length == 1;
> + RawBatchBuffer buffer = buffers[0];
> + return new WireRecordBatch(context, buffer);
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> new file mode 100644
> index 0000000..80def05
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.base.PhysicalOperator;
> +import org.apache.drill.exec.record.RecordBatch;
> +
> +public interface RootCreator<T extends PhysicalOperator> {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootCreator.class);
> +
> + public RootExec getRoot(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> new file mode 100644
> index 0000000..3f8aac7
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
> @@ -0,0 +1,40 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import org.apache.drill.exec.exception.FragmentSetupException;
> +
> +/**
> + * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
> + * output nodes and storage nodes. They are there driving force behind the completion of a query.
> + */
> +public interface RootExec {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootExec.class);
> +
> + /**
> + * Do the next batch of work.
> + * @return Whether or not additional batches of work are necessary. False means that this fragment is done.
> + */
> + public boolean next();
> +
> + /**
> + * Inform all children to clean up and go away.
> + */
> + public void stop();
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> new file mode 100644
> index 0000000..33c1e29
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
> @@ -0,0 +1,172 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import java.util.Iterator;
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
> +import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.SchemaBuilder;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +import org.apache.drill.exec.store.RecordReader;
> +
> +import com.carrotsearch.hppc.IntObjectOpenHashMap;
> +import com.carrotsearch.hppc.procedures.IntObjectProcedure;
> +import com.google.common.collect.Lists;
> +
> +/**
> + * Record batch used for a particular scan. Operators against one or more
> + */
> +public class ScanBatch implements RecordBatch {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
> +
> + private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
> + private BatchSchema schema;
> + private int recordCount;
> + private boolean schemaChanged = true;
> + private final FragmentContext context;
> + private Iterator<RecordReader> readers;
> + private RecordReader currentReader;
> + private final Mutator mutator = new Mutator();
> +
> + public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
> + throws ExecutionSetupException {
> + this.context = context;
> + this.readers = readers;
> + if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
> + this.currentReader = readers.next();
> + this.currentReader.setup(mutator);
> + }
> +
> + private void schemaChanged() {
> + schema = null;
> + schemaChanged = true;
> + }
> +
> + @Override
> + public FragmentContext getContext() {
> + return context;
> + }
> +
> + @Override
> + public BatchSchema getSchema() {
> + return schema;
> + }
> +
> + @Override
> + public int getRecordCount() {
> + return recordCount;
> + }
> +
> + @Override
> + public void kill() {
> + releaseAssets();
> + }
> +
> + private void releaseAssets() {
> + fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
> + @Override
> + public void apply(int key, ValueVector<?> value) {
> + value.close();
> + }
> + });
> + }
> +
> + @SuppressWarnings("unchecked")
> + @Override
> + public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> + if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
> + ValueVector<?> vector = this.fields.lget();
> + if (vector.getClass().isAssignableFrom(clazz)) {
> + return (T) vector;
> + } else {
> + throw new InvalidValueAccessor(String.format(
> + "You requested a field accessor of type %s for field id %d but the actual type was %s.",
> + clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
> + }
> + }
> +
> + @Override
> + public IterOutcome next() {
> + while ((recordCount = currentReader.next()) == 0) {
> + try {
> + if (!readers.hasNext()) {
> + currentReader.cleanup();
> + releaseAssets();
> + return IterOutcome.NONE;
> + }
> + currentReader.cleanup();
> + currentReader = readers.next();
> + currentReader.setup(mutator);
> + } catch (ExecutionSetupException e) {
> + this.context.fail(e);
> + releaseAssets();
> + return IterOutcome.STOP;
> + }
> + }
> +
> + if (schemaChanged) {
> + schemaChanged = false;
> + return IterOutcome.OK_NEW_SCHEMA;
> + } else {
> + return IterOutcome.OK;
> + }
> + }
> +
> + private class Mutator implements OutputMutator {
> + private SchemaBuilder builder = BatchSchema.newBuilder();
> +
> + public void removeField(int fieldId) throws SchemaChangeException {
> + schemaChanged();
> + ValueVector<?> v = fields.remove(fieldId);
> + if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
> + v.close();
> + }
> +
> + public void addField(int fieldId, ValueVector<?> vector) {
> + schemaChanged();
> + ValueVector<?> v = fields.put(fieldId, vector);
> + vector.getField();
> + builder.addField(vector.getField());
> + if (v != null) v.close();
> + }
> +
> + @Override
> + public void setNewSchema() throws SchemaChangeException {
> + ScanBatch.this.schema = this.builder.build();
> + ScanBatch.this.schemaChanged = true;
> + }
> +
> + }
> +
> + @Override
> + public WritableBatch getWritableBatch() {
> + return WritableBatch.get(this.getRecordCount(), fields);
> + }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> new file mode 100644
> index 0000000..c0711db
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
> @@ -0,0 +1,90 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.Screen;
> +import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
> +import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
> +
> +import com.google.common.base.Preconditions;
> +
> +public class ScreenCreator implements RootCreator<Screen>{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
> +
> + @Override
> + public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
> + Preconditions.checkArgument(children.size() == 1);
> + return new ScreenRoot(context, children.iterator().next());
> + }
> +
> +
> + private static class ScreenRoot implements RootExec{
> +
> + final RecordBatch incoming;
> + final FragmentContext context;
> + final UserClientConnection connection;
> + private RecordMaterializer materializer;
> +
> + public ScreenRoot(FragmentContext context, RecordBatch incoming){
> + assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client. As such, this should always be true.";
> +
> + this.context = context;
> + this.incoming = incoming;
> + this.connection = context.getConnection();
> + }
> +
> + @Override
> + public boolean next() {
> + IterOutcome outcome = incoming.next();
> + boolean isLast = false;
> + switch(outcome){
> + case NONE:
> + case STOP:
> + connection.sendResult(materializer.convertNext(true));
> + context.batchesCompleted.inc(1);
> + context.recordsCompleted.inc(incoming.getRecordCount());
> + return false;
> +
> + case OK_NEW_SCHEMA:
> + materializer = new VectorRecordMaterializer(context, incoming);
> + // fall through.
> + // fall through
> + case OK:
> + connection.sendResult(materializer.convertNext(false));
> + context.batchesCompleted.inc(1);
> + context.recordsCompleted.inc(incoming.getRecordCount());
> + return !isLast;
> + default:
> + throw new UnsupportedOperationException();
> + }
> + }
> +
> + @Override
> + public void stop() {
> + incoming.kill();
> + }
> +
> +
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> new file mode 100644
> index 0000000..60c2d78
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
> @@ -0,0 +1,89 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.List;
> +
> +import org.apache.drill.common.exceptions.ExecutionSetupException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.physical.config.SingleSender;
> +import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
> +import org.apache.drill.exec.record.FragmentWritableBatch;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.rpc.bit.BitTunnel;
> +
> +public class SingleSenderCreator implements RootCreator<SingleSender>{
> +
> + @Override
> + public RootExec getRoot(FragmentContext context, SingleSender config, List<RecordBatch> children)
> + throws ExecutionSetupException {
> + assert children != null && children.size() == 1;
> + return new SingleSenderRootExec(context, children.iterator().next(), config);
> + }
> +
> +
> + private static class SingleSenderRootExec implements RootExec{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
> + private RecordBatch incoming;
> + private BitTunnel tunnel;
> + private FragmentHandle handle;
> + private int recMajor;
> + private FragmentContext context;
> +
> + public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
> + logger.debug("Creating single sender root exec base on config: {}", config);
> + this.incoming = batch;
> + this.handle = context.getHandle();
> + this.recMajor = config.getOppositeMajorFragmentId();
> + this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
> + this.context = context;
> + }
> +
> + @Override
> + public boolean next() {
> + IterOutcome out = incoming.next();
> + logger.debug("Outcome of sender next {}", out);
> + switch(out){
> + case STOP:
> + case NONE:
> + FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
> + tunnel.sendRecordBatch(context, b2);
> + return false;
> +
> +
> + case OK:
> + case OK_NEW_SCHEMA:
> + FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
> + tunnel.sendRecordBatch(context, batch);
> + return true;
> +
> + case NOT_YET:
> + default:
> + throw new IllegalStateException();
> + }
> + }
> +
> + @Override
> + public void stop() {
> + }
> +
> +
> +
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> new file mode 100644
> index 0000000..fc7f833
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
> @@ -0,0 +1,99 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl;
> +
> +import java.util.Iterator;
> +
> +import org.apache.drill.exec.exception.SchemaChangeException;
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
> +import org.apache.drill.exec.record.BatchSchema;
> +import org.apache.drill.exec.record.InvalidValueAccessor;
> +import org.apache.drill.exec.record.RawFragmentBatch;
> +import org.apache.drill.exec.record.RawFragmentBatchProvider;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.RecordBatchLoader;
> +import org.apache.drill.exec.record.WritableBatch;
> +import org.apache.drill.exec.record.RecordBatch.IterOutcome;
> +import org.apache.drill.exec.record.vector.ValueVector;
> +
> +public class WireRecordBatch implements RecordBatch{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
> +
> + private RecordBatchLoader batchLoader;
> + private RawFragmentBatchProvider fragProvider;
> + private FragmentContext context;
> +
> +
> + public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
> + this.fragProvider = fragProvider;
> + this.context = context;
> + this.batchLoader = new RecordBatchLoader(context.getAllocator());
> + }
> +
> + @Override
> + public FragmentContext getContext() {
> + return context;
> + }
> +
> + @Override
> + public BatchSchema getSchema() {
> + return null;
> + }
> +
> + @Override
> + public int getRecordCount() {
> + return batchLoader.getRecordCount();
> + }
> +
> + @Override
> + public void kill() {
> + fragProvider.kill(context);
> + }
> +
> + @Override
> + public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
> + return batchLoader.getValueVector(fieldId, clazz);
> + }
> +
> + @Override
> + public IterOutcome next() {
> + RawFragmentBatch batch = this.fragProvider.getNext();
> + try{
> + if(batch == null) return IterOutcome.NONE;
> +
> + RecordBatchDef rbd = batch.getHeader().getDef();
> + boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
> + if(schemaChanged){
> + return IterOutcome.OK_NEW_SCHEMA;
> + }else{
> + return IterOutcome.OK;
> + }
> + }catch(SchemaChangeException ex){
> + context.fail(ex);
> + return IterOutcome.STOP;
> + }
> + }
> +
> + @Override
> + public WritableBatch getWritableBatch() {
> + return batchLoader.getWritableBatch();
> + }
> +
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> new file mode 100644
> index 0000000..187e6e9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
> @@ -0,0 +1,46 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl.materialize;
> +
> +import io.netty.buffer.ByteBuf;
> +
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public class QueryWritableBatch {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWritableBatch.class);
> +
> + private final QueryResult header;
> + private final ByteBuf[] buffers;
> +
> +
> + public QueryWritableBatch(QueryResult header, ByteBuf... buffers) {
> + super();
> + this.header = header;
> + this.buffers = buffers;
> + }
> +
> + public ByteBuf[] getBuffers(){
> + return buffers;
> + }
> +
> + public QueryResult getHeader() {
> + return header;
> + }
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> new file mode 100644
> index 0000000..17c65e9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
> @@ -0,0 +1,31 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl.materialize;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.QueryId;
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public interface RecordMaterializer {
> +
> + public QueryWritableBatch convertNext(boolean isLast);
> +
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> new file mode 100644
> index 0000000..e2d2eb9
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
> @@ -0,0 +1,52 @@
> +/*******************************************************************************
> + * 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.drill.exec.physical.impl.materialize;
> +
> +import org.apache.drill.exec.ops.FragmentContext;
> +import org.apache.drill.exec.proto.UserBitShared.QueryId;
> +import org.apache.drill.exec.proto.UserProtos.QueryResult;
> +import org.apache.drill.exec.record.MaterializedField;
> +import org.apache.drill.exec.record.RecordBatch;
> +import org.apache.drill.exec.record.WritableBatch;
> +
> +public class VectorRecordMaterializer implements RecordMaterializer{
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
> +
> + private QueryId queryId;
> + private RecordBatch batch;
> +
> + public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
> + this.queryId = context.getHandle().getQueryId();
> + this.batch = batch;
> +
> + for (MaterializedField f : batch.getSchema()) {
> + logger.debug("New Field: {}", f);
> + }
> + }
> +
> + public QueryWritableBatch convertNext(boolean isLast) {
> + WritableBatch w = batch.getWritableBatch();
> +
> + QueryResult header = QueryResult.newBuilder() //
> + .setQueryId(queryId) //
> + .setRowCount(batch.getRecordCount()) //
> + .setDef(w.getDef()).setIsLastChunk(isLast).build();
> + QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
> + return batch;
> + }
> +}
>
> http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> ----------------------------------------------------------------------
> diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> new file mode 100644
> index 0000000..9b2cb85
> --- /dev/null
> +++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
> @@ -0,0 +1,45 @@
> +/*******************************************************************************
> + * 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.drill.exec.planner;
> +
> +import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
> +import org.apache.drill.exec.physical.base.Exchange;
> +import org.apache.drill.exec.planner.fragment.Wrapper;
> +
> +public abstract class AbstractOpWrapperVisitor<RET, EXCEP extends Throwable> extends
> + AbstractPhysicalVisitor<RET, Wrapper, EXCEP> {
> + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractOpWrapperVisitor.class);
> +
> + @Override
> + public final RET visitExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> + if (wrapper.getNode().getSendingExchange() == exchange) {
> + return visitSendingExchange(exchange, wrapper);
> + } else {
> + return visitReceivingExchange(exchange, wrapper);
> + }
> + }
> +
> + public RET visitSendingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> + return visitOp(exchange, wrapper);
> + }
> +
> + public RET visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
> + return visitOp(exchange, wrapper);
> + }
> +
> +}
>
[09/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
new file mode 100644
index 0000000..0044628
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorageEngine.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.AbstractStorageEngine;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.StorageEngine;
+import org.apache.drill.exec.store.StorageEngine.ReadEntry;
+
+import com.google.common.collect.ListMultimap;
+
+public class MockStorageEngine extends AbstractStorageEngine{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
+
+ @Override
+ public boolean supportsRead() {
+ return true;
+ }
+
+ @Override
+ public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
+ return null;
+ }
+
+ @Override
+ public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
+ return null;
+ }
+
+ @Override
+ public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
+ return null;
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
new file mode 100644
index 0000000..639d0d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MockStorePOP.java
@@ -0,0 +1,75 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractStore;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("mock-store")
+public class MockStorePOP extends AbstractStore {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
+
+ @JsonCreator
+ public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
+ super(child);
+ }
+
+ public int getMaxWidth() {
+ return 1;
+ }
+
+ @Override
+ public List<EndpointAffinity> getOperatorAffinity() {
+ return Collections.emptyList();
+ }
+
+ @Override
+ public void applyAssignments(List<DrillbitEndpoint> endpoints) {
+
+ }
+
+ @Override
+ public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+ return new MockStorePOP(child);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return new OperatorCost(1,getSize().getRecordCount()*getSize().getRecordSize(),1,1);
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new MockStorePOP(child);
+ }
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
new file mode 100644
index 0000000..eb77eeb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/PartitionRange.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import org.apache.drill.common.expression.LogicalExpression;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class PartitionRange {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PartitionRange.class);
+
+ private LogicalExpression start;
+ private LogicalExpression finish;
+
+ @JsonCreator
+ public PartitionRange(@JsonProperty("start") LogicalExpression start, @JsonProperty("finish") LogicalExpression finish) {
+ super();
+ this.start = start;
+ this.finish = finish;
+ }
+
+ public LogicalExpression getStart() {
+ return start;
+ }
+
+ public LogicalExpression getFinish() {
+ return finish;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
new file mode 100644
index 0000000..e869393
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("project")
+public class Project extends AbstractSingle{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Project.class);
+
+ private final List<NamedExpression> exprs;
+
+ @JsonCreator
+ public Project(@JsonProperty("exprs") List<NamedExpression> exprs, @JsonProperty("child") PhysicalOperator child) {
+ super(child);
+ this.exprs = exprs;
+ }
+
+ public List<NamedExpression> getExprs() {
+ return exprs;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+ return physicalVisitor.visitProject(this, value);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
+ }
+
+ @Override
+ public Size getSize() {
+ //TODO: This should really change the row width...
+ return child.getSize();
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new Project(exprs, child);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
new file mode 100644
index 0000000..ed41586
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RandomReceiver.java
@@ -0,0 +1,83 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractReceiver;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("random-receiver")
+public class RandomReceiver extends AbstractReceiver{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
+
+ private List<DrillbitEndpoint> senders;
+
+ @JsonCreator
+ public RandomReceiver(@JsonProperty("sender-major-fragment") int oppositeMajorFragmentId, @JsonProperty("senders") List<DrillbitEndpoint> senders){
+ super(oppositeMajorFragmentId);
+ this.senders = senders;
+ }
+
+ @Override
+ public List<DrillbitEndpoint> getProvidingEndpoints() {
+ return senders;
+ }
+
+ @Override
+ public boolean supportsOutOfOrderExchange() {
+ return true;
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ //TODO: deal with receiver cost through exchange.
+ return new OperatorCost(1,1,1,1);
+ }
+
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitRandomReceiver(this, value);
+ }
+
+ @Override
+ public Size getSize() {
+ //TODO: deal with size info through exchange.
+ return new Size(1,1);
+ }
+
+ @Override
+ public int getOppositeMajorFragmentId() {
+ return 0;
+ }
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
new file mode 100644
index 0000000..7d64dba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/RangeSender.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("range-sender")
+public class RangeSender extends AbstractSender{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RangeSender.class);
+
+ List<EndpointPartition> partitions;
+
+ @JsonCreator
+ public RangeSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("partitions") List<EndpointPartition> partitions) {
+ super(oppositeMajorFragmentId, child);
+ this.partitions = partitions;
+ }
+
+ @Override
+ public List<DrillbitEndpoint> getDestinations() {
+ return null;
+ }
+
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new RangeSender(oppositeMajorFragmentId, child, partitions);
+ }
+
+
+ public static class EndpointPartition{
+ private final PartitionRange range;
+ private final DrillbitEndpoint endpoint;
+
+ @JsonCreator
+ public EndpointPartition(@JsonProperty("range") PartitionRange range, @JsonProperty("endpoint") DrillbitEndpoint endpoint) {
+ super();
+ this.range = range;
+ this.endpoint = endpoint;
+ }
+ public PartitionRange getRange() {
+ return range;
+ }
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
new file mode 100644
index 0000000..86a201d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -0,0 +1,106 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractStore;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Root;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.base.Preconditions;
+
+@JsonTypeName("screen")
+public class Screen extends AbstractStore implements Root{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Screen.class);
+
+ private final DrillbitEndpoint endpoint;
+
+ public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
+ super(child);
+ this.endpoint = endpoint;
+ }
+
+ @Override
+ public List<EndpointAffinity> getOperatorAffinity() {
+ return Collections.singletonList(new EndpointAffinity(endpoint, 1000000000000l));
+ }
+
+ @Override
+ public int getMaxWidth() {
+ return 1;
+ }
+
+ @Override
+ public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
+ // we actually don't have to do anything since nothing should have changed. we'll check just check that things
+ // didn't get screwed up.
+ if (endpoints.size() != 1) throw new PhysicalOperatorSetupException("A Screen operator can only be assigned to a single node.");
+ DrillbitEndpoint endpoint = endpoints.iterator().next();
+ logger.debug("Endpoint this: {}, assignment: {}", this.endpoint, endpoint);
+ if (!endpoint.equals(this.endpoint)) {
+ throw new PhysicalOperatorSetupException(String.format(
+ "A Screen operator can only be assigned to its home node. Expected endpoint %s, Actual endpoint: %s",
+ this.endpoint, endpoint));
+ }
+ }
+
+ @Override
+ public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
+ return new Screen(child, endpoint);
+ }
+
+ @JsonIgnore
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+ @Override
+ public String toString() {
+ return "Screen [endpoint=" + endpoint + ", getChild()=" + getChild() + "]";
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ return new OperatorCost(0, 0, 1000, child.getSize().getRecordCount());
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new Screen(child, endpoint);
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitScreen(this, value);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
new file mode 100644
index 0000000..79d937a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleSender.java
@@ -0,0 +1,78 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.graph.GraphVisitor;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSender;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+/**
+ * Sender that pushes all data to a single destination node.
+ */
+@JsonTypeName("single-sender")
+public class SingleSender extends AbstractSender {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSender.class);
+
+ private final DrillbitEndpoint destination;
+
+ @JsonCreator
+ public SingleSender(@JsonProperty("receiver-major-fragment") int oppositeMajorFragmentId, @JsonProperty("child") PhysicalOperator child, @JsonProperty("destination") DrillbitEndpoint destination) {
+ super(oppositeMajorFragmentId, child);
+ this.destination = destination;
+ }
+
+ @Override
+ @JsonIgnore
+ public List<DrillbitEndpoint> getDestinations() {
+ return Collections.singletonList(destination);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ long recordSize = child.getSize().getRecordSize() * child.getSize().getRecordCount();
+ return new OperatorCost((float) recordSize, recordSize, 0, child.getSize().getRecordCount()/(1<<16));
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new SingleSender(oppositeMajorFragmentId, child, destination);
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+ return physicalVisitor.visitSingleSender(this, value);
+ }
+
+
+ public DrillbitEndpoint getDestination() {
+ return destination;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
new file mode 100644
index 0000000..e4ece6b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
@@ -0,0 +1,86 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.defs.OrderDef;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractSingle;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Size;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("sort")
+public class Sort extends AbstractSingle{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Sort.class);
+
+ private final List<OrderDef> orderings;
+ private boolean reverse = false;
+
+ @JsonCreator
+ public Sort(@JsonProperty("child") PhysicalOperator child, @JsonProperty("orderings") List<OrderDef> orderings, @JsonProperty("reverse") boolean reverse) {
+ super(child);
+ this.orderings = orderings;
+ this.reverse = reverse;
+ }
+
+ public List<OrderDef> getOrderings() {
+ return orderings;
+ }
+
+ public boolean getReverse() {
+ return reverse;
+ }
+
+ @Override
+ public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
+ return physicalVisitor.visitSort(this, value);
+ }
+
+ @Override
+ public OperatorCost getCost() {
+ Size childSize = child.getSize();
+ long n = childSize.getRecordCount();
+ long width = childSize.getRecordSize();
+
+ //TODO: Magic Number, let's assume 1/10 of data can fit in memory.
+ int k = 10;
+ long n2 = n/k;
+ double cpuCost =
+ k * n2 * (Math.log(n2)/Math.log(2)) + //
+ n * (Math.log(k)/Math.log(2));
+ double diskCost = n*width*2;
+
+ return new OperatorCost(0, (float) diskCost, (float) n2*width, (float) cpuCost);
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new Sort(child, orderings, reverse);
+ }
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
new file mode 100644
index 0000000..56467ce
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/UnionExchange.java
@@ -0,0 +1,79 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.base.AbstractExchange;
+import org.apache.drill.exec.physical.base.ExchangeCost;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.physical.base.Sender;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("union-exchange")
+public class UnionExchange extends AbstractExchange{
+
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnionExchange.class);
+
+ private List<DrillbitEndpoint> senderLocations;
+ private DrillbitEndpoint destinationLocation;
+
+ public UnionExchange(@JsonProperty("child") PhysicalOperator child) {
+ super(child);
+ }
+
+ @Override
+ public void setupSenders(List<DrillbitEndpoint> senderLocations) {
+ this.senderLocations = senderLocations;
+ }
+
+ @Override
+ protected void setupReceivers(List<DrillbitEndpoint> receiverLocations) throws PhysicalOperatorSetupException {
+ if(receiverLocations.size() != 1) throw new PhysicalOperatorSetupException("A Union Exchange only supports a single receiver endpoint.");
+ this.destinationLocation = receiverLocations.iterator().next();
+ }
+
+ @Override
+ public Sender getSender(int minorFragmentId, PhysicalOperator child) {
+ return new SingleSender(this.receiverMajorFragmentId, child, destinationLocation);
+ }
+
+ @Override
+ public Receiver getReceiver(int minorFragmentId) {
+ return new RandomReceiver(this.senderMajorFragmentId, senderLocations);
+ }
+
+ @Override
+ public int getMaxSendWidth() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
+ return new UnionExchange(child);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
new file mode 100644
index 0000000..9a7df56
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/BatchCreator.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface BatchCreator<T extends PhysicalOperator> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCreator.class);
+
+ public RecordBatch getBatch(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
new file mode 100644
index 0000000..6592ca1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilterRecordBatch.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public abstract class FilterRecordBatch implements RecordBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilterRecordBatch.class);
+
+ private RecordBatch incoming;
+ private SelectionVector selectionVector;
+ private BatchSchema schema;
+ private FilteringRecordBatchTransformer transformer;
+ private int outstanding;
+
+ public FilterRecordBatch(RecordBatch batch) {
+ this.incoming = batch;
+ }
+
+ @Override
+ public FragmentContext getContext() {
+ return incoming.getContext();
+ }
+
+ @Override
+ public BatchSchema getSchema() {
+ return schema;
+ }
+
+ @Override
+ public int getRecordCount() {
+ return 0;
+ }
+
+ @Override
+ public void kill() {
+ incoming.kill();
+ }
+
+ @Override
+ public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+ return null;
+ }
+
+ abstract int applyFilter(SelectionVector vector, int count);
+
+ /**
+ * Release all assets.
+ */
+ private void close() {
+
+ }
+
+ @Override
+ public IterOutcome next() {
+ while (true) {
+ IterOutcome o = incoming.next();
+ switch (o) {
+ case OK_NEW_SCHEMA:
+ transformer = incoming.getContext().getFilteringExpression(null);
+ schema = transformer.getSchema();
+ // fall through to ok.
+ case OK:
+
+ case NONE:
+ case STOP:
+ close();
+ return IterOutcome.STOP;
+ }
+
+ if (outstanding > 0) {
+ // move data to output location.
+
+ for (int i = incoming.getRecordCount() - outstanding; i < incoming.getRecordCount(); i++) {
+
+ }
+ }
+
+ // make sure the bit vector is as large as the current record batch.
+ if (selectionVector.capacity() < incoming.getRecordCount()) {
+ selectionVector.allocateNew(incoming.getRecordCount());
+ }
+
+ return null;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
new file mode 100644
index 0000000..191521a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/FilteringRecordBatchTransformer.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.vector.SelectionVector;
+
+public abstract class FilteringRecordBatchTransformer {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FilteringRecordBatchTransformer.class);
+
+ final RecordBatch incoming;
+ final SelectionVector selectionVector;
+ final BatchSchema schema;
+
+ public FilteringRecordBatchTransformer(RecordBatch incoming, OutputMutator output, SelectionVector selectionVector) {
+ super();
+ this.incoming = incoming;
+ this.selectionVector = selectionVector;
+ this.schema = innerSetup();
+ }
+
+ public abstract BatchSchema innerSetup();
+
+ /**
+ * Applies the filter to the selection index. Ignores any values in the selection vector, instead creating a.
+ * @return
+ */
+ public abstract int apply();
+
+ /**
+ * Applies the filter to the selection index. Utilizes the existing selection index and only evaluates on those records.
+ * @return
+ */
+ public abstract int applyWithSelection();
+
+ public BatchSchema getSchema() {
+ return schema;
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
new file mode 100644
index 0000000..d98c107
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ImplCreator.java
@@ -0,0 +1,102 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.config.MockScanBatchCreator;
+import org.apache.drill.exec.physical.config.MockScanPOP;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.record.RecordBatch;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class ImplCreator extends AbstractPhysicalVisitor<RecordBatch, FragmentContext, ExecutionSetupException>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImplCreator.class);
+
+ private MockScanBatchCreator msc = new MockScanBatchCreator();
+ private ScreenCreator sc = new ScreenCreator();
+ private RandomReceiverCreator rrc = new RandomReceiverCreator();
+ private SingleSenderCreator ssc = new SingleSenderCreator();
+ private RootExec root = null;
+
+ private ImplCreator(){}
+
+ public RootExec getRoot(){
+ return root;
+ }
+
+
+ @Override
+ public RecordBatch visitScan(Scan<?> scan, FragmentContext context) throws ExecutionSetupException {
+ Preconditions.checkNotNull(scan);
+ Preconditions.checkNotNull(context);
+
+ if(scan instanceof MockScanPOP){
+ return msc.getBatch(context, (MockScanPOP) scan, Collections.<RecordBatch> emptyList());
+ }else{
+ return super.visitScan(scan, context);
+ }
+
+ }
+
+ @Override
+ public RecordBatch visitScreen(Screen op, FragmentContext context) throws ExecutionSetupException {
+ Preconditions.checkArgument(root == null);
+ root = sc.getRoot(context, op, getChildren(op, context));
+ return null;
+ }
+
+
+
+ @Override
+ public RecordBatch visitSingleSender(SingleSender op, FragmentContext context) throws ExecutionSetupException {
+ root = ssc.getRoot(context, op, getChildren(op, context));
+ return null;
+ }
+
+ @Override
+ public RecordBatch visitRandomReceiver(RandomReceiver op, FragmentContext context) throws ExecutionSetupException {
+ return rrc.getBatch(context, op, null);
+ }
+
+ private List<RecordBatch> getChildren(PhysicalOperator op, FragmentContext context) throws ExecutionSetupException{
+ List<RecordBatch> children = Lists.newArrayList();
+ for(PhysicalOperator child : op){
+ children.add(child.accept(this, context));
+ }
+ return children;
+ }
+
+ public static RootExec getExec(FragmentContext context, FragmentRoot root) throws ExecutionSetupException{
+ ImplCreator i = new ImplCreator();
+ root.accept(i, context);
+ if(i.root == null) throw new ExecutionSetupException("The provided fragment did not have a root node that correctly created a RootExec value.");
+ return i.getRoot();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
new file mode 100644
index 0000000..ce0cf66
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public interface OutputMutator {
+ public void removeField(int fieldId) throws SchemaChangeException;
+ public void addField(int fieldId, ValueVector<?> vector) throws SchemaChangeException ;
+ public void setNewSchema() throws SchemaChangeException ;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
new file mode 100644
index 0000000..9995bc2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/PhysicalConfig.java
@@ -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.drill.exec.physical.impl;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Target({ElementType.TYPE})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface PhysicalConfig {
+ Class<?> value();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
new file mode 100644
index 0000000..4b991f8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RandomReceiverCreator.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.RandomReceiver;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.batch.RawBatchBuffer;
+
+public class RandomReceiverCreator implements BatchCreator<RandomReceiver>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiverCreator.class);
+
+ @Override
+ public RecordBatch getBatch(FragmentContext context, RandomReceiver receiver, List<RecordBatch> children)
+ throws ExecutionSetupException {
+ assert children == null || children.isEmpty();
+ IncomingBuffers bufHolder = context.getBuffers();
+ assert bufHolder != null : "IncomingBuffers must be defined for any place a receiver is declared.";
+
+ RawBatchBuffer[] buffers = bufHolder.getBuffers(receiver.getOppositeMajorFragmentId());
+ assert buffers.length == 1;
+ RawBatchBuffer buffer = buffers[0];
+ return new WireRecordBatch(context, buffer);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
new file mode 100644
index 0000000..80def05
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootCreator.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.record.RecordBatch;
+
+public interface RootCreator<T extends PhysicalOperator> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootCreator.class);
+
+ public RootExec getRoot(FragmentContext context, T config, List<RecordBatch> children) throws ExecutionSetupException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
new file mode 100644
index 0000000..3f8aac7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/RootExec.java
@@ -0,0 +1,40 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+
+/**
+ * A FragmentRoot is a node which is the last processing node in a query plan. FragmentTerminals include Exchange
+ * output nodes and storage nodes. They are there driving force behind the completion of a query.
+ */
+public interface RootExec {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootExec.class);
+
+ /**
+ * Do the next batch of work.
+ * @return Whether or not additional batches of work are necessary. False means that this fragment is done.
+ */
+ public boolean next();
+
+ /**
+ * Inform all children to clean up and go away.
+ */
+ public void stop();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
new file mode 100644
index 0000000..33c1e29
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -0,0 +1,172 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.SchemaBuilder;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+
+/**
+ * Record batch used for a particular scan. Operators against one or more
+ */
+public class ScanBatch implements RecordBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanBatch.class);
+
+ private IntObjectOpenHashMap<ValueVector<?>> fields = new IntObjectOpenHashMap<ValueVector<?>>();
+ private BatchSchema schema;
+ private int recordCount;
+ private boolean schemaChanged = true;
+ private final FragmentContext context;
+ private Iterator<RecordReader> readers;
+ private RecordReader currentReader;
+ private final Mutator mutator = new Mutator();
+
+ public ScanBatch(FragmentContext context, Iterator<RecordReader> readers)
+ throws ExecutionSetupException {
+ this.context = context;
+ this.readers = readers;
+ if (!readers.hasNext()) throw new ExecutionSetupException("A scan batch must contain at least one reader.");
+ this.currentReader = readers.next();
+ this.currentReader.setup(mutator);
+ }
+
+ private void schemaChanged() {
+ schema = null;
+ schemaChanged = true;
+ }
+
+ @Override
+ public FragmentContext getContext() {
+ return context;
+ }
+
+ @Override
+ public BatchSchema getSchema() {
+ return schema;
+ }
+
+ @Override
+ public int getRecordCount() {
+ return recordCount;
+ }
+
+ @Override
+ public void kill() {
+ releaseAssets();
+ }
+
+ private void releaseAssets() {
+ fields.forEach(new IntObjectProcedure<ValueVector<?>>() {
+ @Override
+ public void apply(int key, ValueVector<?> value) {
+ value.close();
+ }
+ });
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+ if (fields.containsKey(fieldId)) throw new InvalidValueAccessor(String.format("Unknown value accesor for field id %d."));
+ ValueVector<?> vector = this.fields.lget();
+ if (vector.getClass().isAssignableFrom(clazz)) {
+ return (T) vector;
+ } else {
+ throw new InvalidValueAccessor(String.format(
+ "You requested a field accessor of type %s for field id %d but the actual type was %s.",
+ clazz.getCanonicalName(), fieldId, vector.getClass().getCanonicalName()));
+ }
+ }
+
+ @Override
+ public IterOutcome next() {
+ while ((recordCount = currentReader.next()) == 0) {
+ try {
+ if (!readers.hasNext()) {
+ currentReader.cleanup();
+ releaseAssets();
+ return IterOutcome.NONE;
+ }
+ currentReader.cleanup();
+ currentReader = readers.next();
+ currentReader.setup(mutator);
+ } catch (ExecutionSetupException e) {
+ this.context.fail(e);
+ releaseAssets();
+ return IterOutcome.STOP;
+ }
+ }
+
+ if (schemaChanged) {
+ schemaChanged = false;
+ return IterOutcome.OK_NEW_SCHEMA;
+ } else {
+ return IterOutcome.OK;
+ }
+ }
+
+ private class Mutator implements OutputMutator {
+ private SchemaBuilder builder = BatchSchema.newBuilder();
+
+ public void removeField(int fieldId) throws SchemaChangeException {
+ schemaChanged();
+ ValueVector<?> v = fields.remove(fieldId);
+ if (v == null) throw new SchemaChangeException("Failure attempting to remove an unknown field.");
+ v.close();
+ }
+
+ public void addField(int fieldId, ValueVector<?> vector) {
+ schemaChanged();
+ ValueVector<?> v = fields.put(fieldId, vector);
+ vector.getField();
+ builder.addField(vector.getField());
+ if (v != null) v.close();
+ }
+
+ @Override
+ public void setNewSchema() throws SchemaChangeException {
+ ScanBatch.this.schema = this.builder.build();
+ ScanBatch.this.schemaChanged = true;
+ }
+
+ }
+
+ @Override
+ public WritableBatch getWritableBatch() {
+ return WritableBatch.get(this.getRecordCount(), fields);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
new file mode 100644
index 0000000..c0711db
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScreenCreator.java
@@ -0,0 +1,90 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.impl.materialize.RecordMaterializer;
+import org.apache.drill.exec.physical.impl.materialize.VectorRecordMaterializer;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+
+import com.google.common.base.Preconditions;
+
+public class ScreenCreator implements RootCreator<Screen>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenCreator.class);
+
+ @Override
+ public RootExec getRoot(FragmentContext context, Screen config, List<RecordBatch> children) {
+ Preconditions.checkArgument(children.size() == 1);
+ return new ScreenRoot(context, children.iterator().next());
+ }
+
+
+ private static class ScreenRoot implements RootExec{
+
+ final RecordBatch incoming;
+ final FragmentContext context;
+ final UserClientConnection connection;
+ private RecordMaterializer materializer;
+
+ public ScreenRoot(FragmentContext context, RecordBatch incoming){
+ assert context.getConnection() != null : "A screen root should only be run on the driving node which is connected directly to the client. As such, this should always be true.";
+
+ this.context = context;
+ this.incoming = incoming;
+ this.connection = context.getConnection();
+ }
+
+ @Override
+ public boolean next() {
+ IterOutcome outcome = incoming.next();
+ boolean isLast = false;
+ switch(outcome){
+ case NONE:
+ case STOP:
+ connection.sendResult(materializer.convertNext(true));
+ context.batchesCompleted.inc(1);
+ context.recordsCompleted.inc(incoming.getRecordCount());
+ return false;
+
+ case OK_NEW_SCHEMA:
+ materializer = new VectorRecordMaterializer(context, incoming);
+ // fall through.
+ // fall through
+ case OK:
+ connection.sendResult(materializer.convertNext(false));
+ context.batchesCompleted.inc(1);
+ context.recordsCompleted.inc(incoming.getRecordCount());
+ return !isLast;
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ @Override
+ public void stop() {
+ incoming.kill();
+ }
+
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
new file mode 100644
index 0000000..60c2d78
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/SingleSenderCreator.java
@@ -0,0 +1,89 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.config.SingleSender;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+public class SingleSenderCreator implements RootCreator<SingleSender>{
+
+ @Override
+ public RootExec getRoot(FragmentContext context, SingleSender config, List<RecordBatch> children)
+ throws ExecutionSetupException {
+ assert children != null && children.size() == 1;
+ return new SingleSenderRootExec(context, children.iterator().next(), config);
+ }
+
+
+ private static class SingleSenderRootExec implements RootExec{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SingleSenderRootExec.class);
+ private RecordBatch incoming;
+ private BitTunnel tunnel;
+ private FragmentHandle handle;
+ private int recMajor;
+ private FragmentContext context;
+
+ public SingleSenderRootExec(FragmentContext context, RecordBatch batch, SingleSender config){
+ logger.debug("Creating single sender root exec base on config: {}", config);
+ this.incoming = batch;
+ this.handle = context.getHandle();
+ this.recMajor = config.getOppositeMajorFragmentId();
+ this.tunnel = context.getCommunicator().getTunnel(config.getDestination());
+ this.context = context;
+ }
+
+ @Override
+ public boolean next() {
+ IterOutcome out = incoming.next();
+ logger.debug("Outcome of sender next {}", out);
+ switch(out){
+ case STOP:
+ case NONE:
+ FragmentWritableBatch b2 = new FragmentWritableBatch(false, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+ tunnel.sendRecordBatch(context, b2);
+ return false;
+
+
+ case OK:
+ case OK_NEW_SCHEMA:
+ FragmentWritableBatch batch = new FragmentWritableBatch(true, handle.getQueryId(), handle.getMajorFragmentId(), handle.getMinorFragmentId(), recMajor, 0, incoming.getWritableBatch());
+ tunnel.sendRecordBatch(context, batch);
+ return true;
+
+ case NOT_YET:
+ default:
+ throw new IllegalStateException();
+ }
+ }
+
+ @Override
+ public void stop() {
+ }
+
+
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
new file mode 100644
index 0000000..fc7f833
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/WireRecordBatch.java
@@ -0,0 +1,99 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.Iterator;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.exec.record.InvalidValueAccessor;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.WritableBatch;
+import org.apache.drill.exec.record.RecordBatch.IterOutcome;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+public class WireRecordBatch implements RecordBatch{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WireRecordBatch.class);
+
+ private RecordBatchLoader batchLoader;
+ private RawFragmentBatchProvider fragProvider;
+ private FragmentContext context;
+
+
+ public WireRecordBatch(FragmentContext context, RawFragmentBatchProvider fragProvider) {
+ this.fragProvider = fragProvider;
+ this.context = context;
+ this.batchLoader = new RecordBatchLoader(context.getAllocator());
+ }
+
+ @Override
+ public FragmentContext getContext() {
+ return context;
+ }
+
+ @Override
+ public BatchSchema getSchema() {
+ return null;
+ }
+
+ @Override
+ public int getRecordCount() {
+ return batchLoader.getRecordCount();
+ }
+
+ @Override
+ public void kill() {
+ fragProvider.kill(context);
+ }
+
+ @Override
+ public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+ return batchLoader.getValueVector(fieldId, clazz);
+ }
+
+ @Override
+ public IterOutcome next() {
+ RawFragmentBatch batch = this.fragProvider.getNext();
+ try{
+ if(batch == null) return IterOutcome.NONE;
+
+ RecordBatchDef rbd = batch.getHeader().getDef();
+ boolean schemaChanged = batchLoader.load(rbd, batch.getBody());
+ if(schemaChanged){
+ return IterOutcome.OK_NEW_SCHEMA;
+ }else{
+ return IterOutcome.OK;
+ }
+ }catch(SchemaChangeException ex){
+ context.fail(ex);
+ return IterOutcome.STOP;
+ }
+ }
+
+ @Override
+ public WritableBatch getWritableBatch() {
+ return batchLoader.getWritableBatch();
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
new file mode 100644
index 0000000..187e6e9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/QueryWritableBatch.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl.materialize;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class QueryWritableBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWritableBatch.class);
+
+ private final QueryResult header;
+ private final ByteBuf[] buffers;
+
+
+ public QueryWritableBatch(QueryResult header, ByteBuf... buffers) {
+ super();
+ this.header = header;
+ this.buffers = buffers;
+ }
+
+ public ByteBuf[] getBuffers(){
+ return buffers;
+ }
+
+ public QueryResult getHeader() {
+ return header;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
new file mode 100644
index 0000000..17c65e9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/RecordMaterializer.java
@@ -0,0 +1,31 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl.materialize;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.WritableBatch;
+
+public interface RecordMaterializer {
+
+ public QueryWritableBatch convertNext(boolean isLast);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
new file mode 100644
index 0000000..e2d2eb9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/materialize/VectorRecordMaterializer.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl.materialize;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.WritableBatch;
+
+public class VectorRecordMaterializer implements RecordMaterializer{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(VectorRecordMaterializer.class);
+
+ private QueryId queryId;
+ private RecordBatch batch;
+
+ public VectorRecordMaterializer(FragmentContext context, RecordBatch batch) {
+ this.queryId = context.getHandle().getQueryId();
+ this.batch = batch;
+
+ for (MaterializedField f : batch.getSchema()) {
+ logger.debug("New Field: {}", f);
+ }
+ }
+
+ public QueryWritableBatch convertNext(boolean isLast) {
+ WritableBatch w = batch.getWritableBatch();
+
+ QueryResult header = QueryResult.newBuilder() //
+ .setQueryId(queryId) //
+ .setRowCount(batch.getRecordCount()) //
+ .setDef(w.getDef()).setIsLastChunk(isLast).build();
+ QueryWritableBatch batch = new QueryWritableBatch(header, w.getBuffers());
+ return batch;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
new file mode 100644
index 0000000..9b2cb85
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/AbstractOpWrapperVisitor.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.planner;
+
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.planner.fragment.Wrapper;
+
+public abstract class AbstractOpWrapperVisitor<RET, EXCEP extends Throwable> extends
+ AbstractPhysicalVisitor<RET, Wrapper, EXCEP> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractOpWrapperVisitor.class);
+
+ @Override
+ public final RET visitExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+ if (wrapper.getNode().getSendingExchange() == exchange) {
+ return visitSendingExchange(exchange, wrapper);
+ } else {
+ return visitReceivingExchange(exchange, wrapper);
+ }
+ }
+
+ public RET visitSendingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+ return visitOp(exchange, wrapper);
+ }
+
+ public RET visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws EXCEP {
+ return visitOp(exchange, wrapper);
+ }
+
+}
[02/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
new file mode 100644
index 0000000..3f710ed
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/LocalFragmentHandler.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * 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.drill.exec.work.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+
+public class LocalFragmentHandler implements IncomingFragmentHandler{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalFragmentHandler.class);
+
+ private final IncomingBuffers buffers;
+ private final FragmentRunner runner;
+ private final FragmentHandle handle;
+ private volatile boolean cancel = false;
+
+ public LocalFragmentHandler(FragmentHandle handle, IncomingBuffers buffers, FragmentRunner runner) {
+ super();
+ this.handle = handle;
+ this.buffers = buffers;
+ this.runner = runner;
+ }
+
+ @Override
+ public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+ return buffers.batchArrived(throttle, batch);
+ }
+
+ @Override
+ public FragmentRunner getRunnable() {
+ return runner;
+ }
+
+
+ public FragmentHandle getHandle() {
+ return handle;
+ }
+
+ @Override
+ public void cancel() {
+ cancel = true;
+ }
+
+ @Override
+ public boolean isDone() {
+ return cancel || isDone();
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
new file mode 100644
index 0000000..70d7e93
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/RemoteFragmentHandler.java
@@ -0,0 +1,123 @@
+/*******************************************************************************
+ * 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.drill.exec.work.fragment;
+
+import java.io.IOException;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentLeaf;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.FragmentRunnerListener;
+import org.apache.drill.exec.work.RemotingFragmentRunnerListener;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+
+/**
+ * This handler receives all incoming traffic for a particular FragmentHandle. It will monitor the state of incoming batches
+ */
+public class RemoteFragmentHandler implements IncomingFragmentHandler {
+ private final PlanFragment fragment;
+ private FragmentLeaf root;
+ private final IncomingBuffers buffers;
+ private final FragmentRunnerListener runnerListener;
+ private volatile FragmentRunner runner;
+ private volatile boolean cancel = false;
+ private final FragmentContext context;
+ private final PhysicalPlanReader reader;
+
+ public RemoteFragmentHandler(PlanFragment fragment, DrillbitContext context, BitTunnel foremanTunnel) throws FragmentSetupException{
+ try{
+ this.fragment = fragment;
+ this.root = context.getPlanReader().readFragmentOperator(fragment.getFragmentJson());
+ this.buffers = new IncomingBuffers(root);
+ this.context = new FragmentContext(context, fragment.getHandle(), null, buffers);
+ this.runnerListener = new RemotingFragmentRunnerListener(this.context, foremanTunnel);
+ this.reader = context.getPlanReader();
+
+ }catch(IOException e){
+ throw new FragmentSetupException("Failure while decoding fragment.", e);
+ }
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.fragment.FragmentHandler#handle(org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle, org.apache.drill.exec.record.RawFragmentBatch)
+ */
+ @Override
+ public boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+ return buffers.batchArrived(throttle, batch);
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.fragment.FragmentHandler#getRunnable()
+ */
+ @Override
+ public FragmentRunner getRunnable(){
+ synchronized(this){
+ if(runner != null) throw new IllegalStateException("Get Runnable can only be run once.");
+ if(cancel) return null;
+ try {
+ FragmentRoot fragRoot = reader.readFragmentOperator(fragment.getFragmentJson());
+ RootExec exec = ImplCreator.getExec(context, fragRoot);
+ runner = new FragmentRunner(context, exec, runnerListener);
+ return this.runner;
+ } catch (IOException | ExecutionSetupException e) {
+ runnerListener.fail(fragment.getHandle(), "Failure while setting up remote fragment.", e);
+ return null;
+ }
+ }
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.fragment.FragmentHandler#cancel()
+ */
+ @Override
+ public void cancel(){
+ synchronized(this){
+ cancel = true;
+ if(runner != null){
+ runner.cancel();
+ }
+ }
+ }
+
+ @Override
+ public FragmentHandle getHandle() {
+ return fragment.getHandle();
+ }
+
+ @Override
+ public boolean isDone() {
+ return cancel || buffers.isDone();
+ }
+
+
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
new file mode 100644
index 0000000..621c7cb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
@@ -0,0 +1,72 @@
+/*******************************************************************************
+ * 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.drill.exec.work.user;
+
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.foreman.Foreman;
+
+public class UserWorker{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserWorker.class);
+
+ private final WorkerBee bee;
+
+ public UserWorker(WorkerBee bee) {
+ super();
+ this.bee = bee;
+ }
+
+ public QueryId submitWork(UserClientConnection connection, RunQuery query){
+ UUID uuid = UUID.randomUUID();
+ QueryId id = QueryId.newBuilder().setPart1(uuid.getMostSignificantBits()).setPart2(uuid.getLeastSignificantBits()).build();
+ Foreman foreman = new Foreman(bee, bee.getContext(), connection, id, query);
+ bee.addNewForeman(foreman);
+ return id;
+ }
+
+ public QueryResult getResult(UserClientConnection connection, RequestResults req){
+ Foreman foreman = bee.getForemanForQueryId(req.getQueryId());
+ if(foreman == null) return QueryResult.newBuilder().setQueryState(QueryState.UNKNOWN_QUERY).build();
+ return foreman.getResult(connection, req);
+ }
+
+ public Ack cancelQuery(QueryId query){
+ Foreman foreman = bee.getForemanForQueryId(query);
+ if(foreman != null){
+ foreman.cancel();
+ }
+ return Acks.OK;
+ }
+
+ public Ack cancelFragment(FragmentHandle handle){
+ FragmentRunner runner = bee.getFragmentRunner(handle);
+ if(runner != null) runner.cancel();
+ return Acks.OK;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
new file mode 100644
index 0000000..5cc5cab
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/Coordination.proto
@@ -0,0 +1,26 @@
+package exec;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "CoordinationProtos";
+option optimize_for = SPEED;
+
+message DrillbitEndpoint{
+ optional string address = 1;
+ optional int32 user_port = 2;
+ optional int32 bit_port = 3;
+ optional Roles roles = 4;
+}
+
+message DrillServiceInstance{
+ optional string id = 1;
+ optional int64 registrationTimeUTC = 2;
+ optional DrillbitEndpoint endpoint = 3;
+}
+
+message Roles{
+ optional bool sql_query = 1 [default = true];
+ optional bool logical_plan = 2 [default = true];
+ optional bool physical_plan = 3 [default = true];
+ optional bool java_executor = 4 [default = true];
+ optional bool distributed_cache = 5 [default = true];
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
index 77a7ee1..7501d7c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/ExecutionProtos.proto
@@ -2,9 +2,11 @@ package exec.bit;
option java_package = "org.apache.drill.exec.proto";
option java_outer_classname = "ExecProtos";
-option optimize_for = LITE_RUNTIME;
-import "SchemaDef.proto";
+option optimize_for = SPEED;
+
import "Coordination.proto";
+import "UserBitShared.proto";
+
////// UserToBit RPC ///////
@@ -16,7 +18,7 @@ enum RpcType {
// bit requests
REQ_INIATILIZE_FRAGMENT = 3; // Returns Handle
REQ_RECORD_BATCH = 4; // send record batch overview, returns Ack
- REQ_BATCH_CHUNK = 5; // send additional batch chunk, returns Ack.
+
REQ_CANCEL_FRAGMENT = 6; // send a cancellation message for a fragment, returns Ack
REQ_FRAGMENT_STATUS = 7; // get a fragment status, returns FragmentStatus
REQ_BIT_STATUS = 8; // get bit status.
@@ -25,34 +27,29 @@ enum RpcType {
RESP_FRAGMENT_HANDLE = 9;
RESP_FRAGMENT_STATUS = 10;
RESP_BIT_STATUS = 11;
- RESP_BATCH_CHUNK = 12;
}
message BitHandshake{
- optional DrillbitEndpoint endpoint = 1;
+ optional int32 rpc_version = 1;
+ optional DrillbitEndpoint endpoint = 2;
}
message BitBatchChunk {}
message BitStatus {
- repeated ActiveFragment fragment = 1;
-}
-
-message ActiveFragment {
- optional FragmentStatus status = 1;
- optional int64 fragment_id = 2;
- optional int64 query_id = 3;
+ repeated FragmentStatus fragment_status = 1;
}
message FragmentStatus {
enum FragmentState {
- AWAITING_ALLOCATION = 0;
- RUNNING = 1;
- FINISHED = 2;
- CANCELLED = 3;
- FAILED = 4;
+ SENDING = 0;
+ AWAITING_ALLOCATION = 1;
+ RUNNING = 2;
+ FINISHED = 3;
+ CANCELLED = 4;
+ FAILED = 5;
}
optional int64 memory_use = 1;
@@ -61,27 +58,37 @@ message FragmentStatus {
optional int32 estimated_completion_percentage = 4;
optional FragmentState state = 5;
optional int64 data_processed = 6;
+
+ optional FragmentHandle handle = 7;
+ optional exec.shared.DrillPBError error = 8;
+ optional int64 running_time = 9;
}
-message RecordBatchHeader {
+message FragmentRecordBatch{
+ optional FragmentHandle handle = 1;
+ optional int32 sending_major_fragment_id = 2;
+ optional int32 sending_minor_fragment_id = 3;
+ optional exec.shared.RecordBatchDef def = 4;
+ optional bool isLastBatch = 5;
}
message PlanFragment {
- optional int64 query_id = 1;
- optional int32 major_fragment_id = 2;
- optional int32 minor_fragment_id = 3;
+ optional FragmentHandle handle = 1;
optional float network_cost = 4;
optional float cpu_cost = 5;
optional float disk_cost = 6;
optional float memory_cost = 7;
optional string fragment_json = 8;
- optional bool self_driven = 9;
optional DrillbitEndpoint assignment = 10;
+ optional bool leaf_fragment = 9;
+ optional DrillbitEndpoint foreman = 11;
+
}
message FragmentHandle {
- optional int32 major_fragment_id = 1;
- optional int32 minor_fragment_id = 1;
+ optional exec.shared.QueryId query_id = 1;
+ optional int32 major_fragment_id = 2;
+ optional int32 minor_fragment_id = 3;
}
message WorkQueueStatus{
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
index ebc7dca..48011bf 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/GeneralRPC.proto
@@ -2,7 +2,9 @@ package exec.rpc;
option java_package = "org.apache.drill.exec.proto";
option java_outer_classname = "GeneralRPCProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
+
+import "Coordination.proto";
message Ack{
optional bool ok = 1;
@@ -33,3 +35,5 @@ message RpcFailure {
optional string short_error = 3;
optional string long_error = 4;
}
+
+
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
index 6e983d4..de0009a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/SchemaDef.proto
@@ -2,29 +2,62 @@ package exec;
option java_package = "org.apache.drill.exec.proto";
option java_outer_classname = "SchemaDefProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
// Schema Definitions //
-enum DataType {
- LATE = 0;
- INT32 = 1;
- INT64 = 2;
- FLOAT32 = 3;
- FLOAT64 = 4;
- UTF8 = 5;
- BYTES = 6;
+enum MinorType {
+ LATE = 0; // late binding type
+ MAP = 1; // an empty map column. Useful for conceptual setup. Children listed within here
+ REPEATMAP = 2; // a repeated map column (means that multiple children sit below this)
+ TINYINT = 3; // single byte signed integer
+ SMALLINT = 4; // two byte signed integer
+ INT = 5; // four byte signed integer
+ BIGINT = 6; // eight byte signed integer
+ DECIMAL4 = 7; // a decimal supporting precision between 1 and 8 (4 bits for decimal location, 1 sign)
+ DECIMAL8 = 8; // a decimal supporting precision between 9 and 18 (5 bits for decimal location, 1 sign)
+ DECIMAL12 = 9; // a decimal supporting precision between19 and 28 (5 bits for decimal location, 1 sign)
+ DECIMAL16 = 10; // a decimal supporting precision between 29 and 37 (6 bits for decimal location, 1 sign)
+ MONEY = 11; // signed decimal with two digit precision
+ DATE = 12; // days since 4713bc
+ TIME = 13; // time in micros before or after 2000/1/1
+ TIMETZ = 14; // time in micros before or after 2000/1/1 with timezone
+ TIMESTAMP = 15; // unix epoch time in millis
+ DATETIME = 16; // TBD
+ INTERVAL = 17; // TBD
+ FLOAT4 = 18; // 4 byte ieee 754
+ FLOAT8 = 19; // 8 byte ieee 754
+ BOOLEAN = 20; // single bit value
+ FIXEDCHAR = 21; // utf8 fixed length string, padded with spaces
+ VARCHAR1 = 22; // utf8 variable length string (up to 2^8 in length)
+ VARCHAR2 = 23; // utf8 variable length string (up to 2^16 in length)
+ VARCHAR4 = 24; // utf8 variable length string (up to 2^32 in length)
+ FIXEDBINARY = 25; // fixed length binary, padded with 0 bytes
+ VARBINARY1 = 26; // variable length binary (up to 2^8 in length)
+ VARBINARY2 = 27; // variable length binary (up to 2^16 in length)
+ VARBINARY4 = 28; // variable length binary (up to 2^32 in length)
+ UINT1 = 29; // unsigned 1 byte integer
+ UINT2 = 30; // unsigned 2 byte integer
+ UINT4 = 31; // unsigned 4 byte integer
+ UINT8 = 32; // unsigned 8 byte integer
+ PROTO2 = 33; // protobuf encoded complex type. (up to 2^16 in length)
+ PROTO4 = 34; // protobuf encoded complex type. (up to 2^32 in length)
+ MSGPACK2 = 35; // msgpack encoded complex type. (up to 2^16 in length)
+ MSGPACK4 = 36; // msgpack encoded complex type. (up to 2^32 in length)
}
-enum DataMode {
- REQUIRED = 0;
- OPTIONAL = 1;
- REPEATED = 2;
- MAP = 3;
+message MajorType {
+ optional MinorType minor_type = 1;
+ optional DataMode mode = 2;
+ optional int32 width = 3; // optional width for fixed size values.
+ optional int32 precision = 4; // used for decimal types
+ optional int32 scale = 5; // used for decimal types
}
-message SchemaDef {
- repeated FieldDef field = 1;
+enum DataMode {
+ OPTIONAL = 0; // nullable
+ REQUIRED = 1; // non-nullable
+ REPEATED = 2; // single, repeated-field
}
enum ValueMode {
@@ -33,12 +66,21 @@ enum ValueMode {
DICT = 2;
}
+message NamePart {
+ enum Type{
+ NAME = 0;
+ ARRAY = 1;
+ }
+
+ optional Type type = 1;
+ optional string name = 2; // only required if this is a named type.
+}
+
message FieldDef {
- optional string name = 1;
- optional DataMode data_mode = 2;
- optional ValueMode value_mode = 3;
+ optional int32 field_id = 1;
+ optional int32 parent_id = 2; // the field_id of the parent of this field. populated when this is a repeated field. a field_id of 0 means that the record is the parent of this repeated field.
+ repeated NamePart name = 3; // multipart description of entire field name
+ optional MajorType major_type = 4; // the type associated with this field.
+ repeated FieldDef field = 5; // only in the cases of type == MAP or REPEATMAP
- // If DataMode == 0-2, type should be populated and fields should be empty. Otherwise, type should empty and fields should be defined.
- optional DataType type = 4;
- repeated FieldDef fields = 5;
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
index 225d1a0..cbf5b4c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/User.proto
@@ -2,8 +2,12 @@ package exec.user;
option java_package = "org.apache.drill.exec.proto";
option java_outer_classname = "UserProtos";
-option optimize_for = LITE_RUNTIME;
+option optimize_for = SPEED;
+
import "SchemaDef.proto";
+import "UserBitShared.proto";
+
+
////// UserToBit RPC ///////
enum RpcType {
@@ -13,7 +17,8 @@ enum RpcType {
// user to bit
RUN_QUERY = 3;
- REQUEST_RESULTS = 4;
+ CANCEL_QUERY = 4;
+ REQUEST_RESULTS = 5;
// bit to user
QUERY_RESULT = 6;
@@ -21,33 +26,36 @@ enum RpcType {
}
message UserToBitHandshake {
- optional bool support_listening = 1;
- optional int32 rpc_version = 2;
+ optional bool support_listening = 2;
+ optional int32 rpc_version = 3;
}
message RequestResults {
- optional int64 query_id = 1;
+ optional exec.shared.QueryId query_id = 1;
optional int32 maximum_responses = 2;
}
message RunQuery {
- optional QueryResultsMode mode = 1;
- optional string plan = 2;
+ optional QueryResultsMode results_mode = 1;
+ optional QueryType type = 2;
+ optional string plan = 3;
+}
+
+enum QueryType {
+ SQL = 1;
+ LOGICAL = 2;
+ PHYSICAL = 3;
}
enum QueryResultsMode {
STREAM_FULL = 1; // Server will inform the client regularly on the status of the query. Once the query is completed, service will inform the client as each query chunk is made available.
- STREAM_FIRST = 2; // Server will inform the client regularly on the status of the query. Once the query is completed, server will inform the client of the first query chunk.
- QUERY_FOR_STATUS = 3; // Client will need to query for status of query.
+ // STREAM_FIRST = 2; // Server will inform the client regularly on the status of the query. Once the query is completed, server will inform the client of the first query chunk.
+ // QUERY_FOR_STATUS = 3; // Client will need to query for status of query.
}
message BitToUserHandshake {
- optional int32 rpc_version = 1;
-}
-
-message QueryHandle {
- optional int64 query_id = 1;
+ optional int32 rpc_version = 2;
}
message NodeStatus {
@@ -56,37 +64,26 @@ message NodeStatus {
}
message QueryResult {
- enum Outcome {
- RUNNING = 0;
- FAILED = 1;
+ enum QueryState {
+ PENDING = 0;
+ RUNNING = 1;
COMPLETED = 2;
- WAITING = 3;
+ CANCELED = 3;
+ FAILED = 4;
+ UNKNOWN_QUERY = 5;
}
- optional Outcome outcome = 1;
- optional SchemaDef schema = 2;
+ optional QueryState query_state = 1;
+ optional exec.shared.QueryId query_id = 2;
optional bool is_last_chunk = 3;
optional int32 row_count = 4;
optional int64 records_scan = 5;
optional int64 records_error = 6;
optional int64 submission_time = 7;
repeated NodeStatus node_status = 8;
- repeated Error error = 9;
-}
-
-message TextErrorLocation{
- optional int32 start_column = 2;
- optional int32 start_row = 3;
- optional int32 end_column = 4;
- optional int32 end_row = 5;
-}
-
-message Error{
- optional int64 error_id = 1; // for debug tracing purposes
- optional string host = 2;
- optional int32 error_type = 3;
- optional string message = 4;
- optional TextErrorLocation error = 5; //optional, used when providing location of error within a piece of text.
+ repeated exec.shared.DrillPBError error = 9;
+ optional exec.shared.RecordBatchDef def = 10;
+ optional bool schema_changed = 11;
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
new file mode 100644
index 0000000..5643c0f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/protobuf/UserBitShared.proto
@@ -0,0 +1,46 @@
+package exec.shared;
+
+option java_package = "org.apache.drill.exec.proto";
+option java_outer_classname = "UserBitShared";
+option optimize_for = SPEED;
+
+import "Coordination.proto";
+import "SchemaDef.proto";
+
+message QueryId {
+ optional sfixed64 part1 = 1;
+ optional sfixed64 part2 = 2;
+}
+
+message DrillPBError{
+ optional string error_id = 1; // for debug tracing purposes
+ optional DrillbitEndpoint endpoint = 2;
+ optional int32 error_type = 3;
+ optional string message = 4;
+ repeated ParsingError parsing_error = 5; //optional, used when providing location of error within a piece of text.
+}
+
+message ParsingError{
+ optional int32 start_column = 2;
+ optional int32 start_row = 3;
+ optional int32 end_column = 4;
+ optional int32 end_row = 5;
+}
+
+message RecordBatch{
+
+}
+
+message RecordBatchDef {
+ repeated FieldMetadata field = 1;
+ optional int32 record_count = 2;
+
+}
+
+message FieldMetadata {
+ optional FieldDef def = 1;
+ optional int32 value_count = 2;
+ optional int32 var_byte_length = 3;
+ optional int32 group_count = 4; // number of groups. (number of repeated records)
+ optional int32 buffer_length = 5;
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
index 37ba12b..d113ca3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/DrillSystemTestBase.java
@@ -103,4 +103,11 @@ public class DrillSystemTestBase {
}
}
+ public Drillbit getABit(){
+ return this.servers.iterator().next();
+ }
+
+ public static DrillConfig getConfig(){
+ return config;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
index 09a06d7..dc463e3 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/client/DrillClientSystemTest.java
@@ -1,17 +1,20 @@
package org.apache.drill.exec.client;
-import com.google.common.base.Charsets;
-import com.google.common.io.Resources;
+import java.util.List;
+
import org.apache.drill.exec.DrillSystemTestBase;
-import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
import org.junit.After;
import org.junit.BeforeClass;
+import org.junit.Ignore;
import org.junit.Test;
-/**
- * @author David Alves
- */
+import com.google.common.base.Charsets;
+import com.google.common.io.Resources;
+
+
+@Ignore
public class DrillClientSystemTest extends DrillSystemTestBase {
private static String plan;
@@ -34,8 +37,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
startCluster(1);
DrillClient client = new DrillClient();
client.connect();
- DrillRpcFuture<UserProtos.QueryHandle> result = client.submitPlan(plan);
- System.out.println(result.get());
+ List<QueryResultBatch> result = client.runQuery(QueryType.LOGICAL, plan);
+ System.out.println(result);
client.close();
}
@@ -45,8 +48,8 @@ public class DrillClientSystemTest extends DrillSystemTestBase {
startCluster(2);
DrillClient client = new DrillClient();
client.connect();
- DrillRpcFuture<UserProtos.QueryHandle> result = client.submitPlan(plan);
- System.out.println(result.get());
+ List<QueryResultBatch> result = client.runQuery(QueryType.LOGICAL, plan);
+ System.out.println(result);
client.close();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
index 3d5d84e..2f8aa18 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/compile/TestClassCompilationTypes.java
@@ -18,14 +18,15 @@
package org.apache.drill.exec.compile;
import org.codehaus.commons.compiler.jdk.ExpressionEvaluator;
+import org.junit.Ignore;
import org.junit.Test;
public class TestClassCompilationTypes {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestClassCompilationTypes.class);
- @Test
+ @Ignore @Test
public void comparePerfs() throws Exception {
- for(int i =0; i < 50000; i++){
+ for(int i =0; i < 500; i++){
int r = 0;
long n0 = System.nanoTime();
r += janino();
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
new file mode 100644
index 0000000..3b6bf6a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/config/ParsePhysicalPlan.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.config;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class ParsePhysicalPlan {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePhysicalPlan.class);
+
+
+ @Test
+ public void parseSimplePlan() throws Exception{
+ DrillConfig c = DrillConfig.create();
+ ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
+ ObjectWriter writer = c.getMapper().writer();
+ PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
+ System.out.println(plan.unparse(writer));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
new file mode 100644
index 0000000..7c6bfe5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/DistributedFragmentRun.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import java.util.List;
+
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+@Ignore
+public class DistributedFragmentRun extends PopUnitTestBase{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedFragmentRun.class);
+
+
+ @Test
+ public void simpleDistributedQuery() throws Exception{
+ RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+ try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet); Drillbit bit2 = new Drillbit(CONFIG, serviceSet); DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+ bit1.run();
+ bit2.run();
+ client.connect();
+ List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_single_exchange.json"), Charsets.UTF_8));
+ System.out.println(results);
+ }
+
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
new file mode 100644
index 0000000..6755bb6
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleFragmentRun.java
@@ -0,0 +1,100 @@
+/*******************************************************************************
+ * 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.drill.exec.physical.impl;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.record.vector.ValueVector;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.junit.Test;
+
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class SimpleFragmentRun extends PopUnitTestBase {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleFragmentRun.class);
+
+ @Test
+ public void runNoExchangeFragment() throws Exception {
+ try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+ Drillbit bit = new Drillbit(CONFIG, serviceSet);
+ DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
+
+ // run query.
+ bit.run();
+ client.connect();
+ List<QueryResultBatch> results = client.runQuery(QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile("/physical_test2.json"), Charsets.UTF_8));
+
+ // look at records
+ RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+ int recordCount = 0;
+ for (QueryResultBatch batch : results) {
+ if(!batch.hasData()) continue;
+ boolean schemaChanged = batchLoader.load(batch.getHeader().getDef(), batch.getData());
+ boolean firstColumn = true;
+
+ // print headers.
+ if (schemaChanged) {
+ System.out.println("\n\n========NEW SCHEMA=========\n\n");
+ for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+
+ if (firstColumn) {
+ firstColumn = false;
+ } else {
+ System.out.print("\t");
+ }
+ System.out.print(v.value.getField().getName());
+ System.out.print("[");
+ System.out.print(v.value.getField().getType().getMinorType());
+ System.out.print("]");
+ }
+ System.out.println();
+ }
+
+
+ for (int i = 0; i < batchLoader.getRecordCount(); i++) {
+ boolean first = true;
+ recordCount++;
+ for (IntObjectCursor<ValueVector<?>> v : batchLoader) {
+ if (first) {
+ first = false;
+ } else {
+ System.out.print("\t");
+ }
+ System.out.print(v.value.getObject(i));
+ }
+ if(!first) System.out.println();
+ }
+
+ }
+ logger.debug("Received results {}", results);
+ assertEquals(recordCount, 200);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
index 98bb874..7b7ab8e 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckFragmenter.java
@@ -24,63 +24,63 @@ import static org.junit.Assert.assertNull;
import java.io.IOException;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
import org.apache.drill.common.util.FileUtils;
import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.planner.FragmentNode;
-import org.apache.drill.exec.planner.FragmentingPhysicalVisitor;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.SimpleExecPlanner;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.QueryWorkUnit;
import org.junit.BeforeClass;
import org.junit.Test;
import com.google.common.base.Charsets;
import com.google.common.io.Files;
-public class CheckFragmenter {
+public class CheckFragmenter extends PopUnitTestBase {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CheckFragmenter.class);
-
- static DrillConfig config;
-
- @BeforeClass
- public static void setup(){
- config = DrillConfig.create();
- }
-
+
+
@Test
- public void ensureOneFragment() throws FragmentSetupException, IOException{
- FragmentNode b = getRootFragment("/physical_test1.json");
+ public void ensureOneFragment() throws FragmentSetupException, IOException {
+ PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+ Fragment b = getRootFragment(ppr, "/physical_test1.json");
assertEquals(1, getFragmentCount(b));
assertEquals(0, b.getReceivingExchangePairs().size());
assertNull(b.getSendingExchange());
}
-
+
@Test
- public void ensureTwoFragments() throws FragmentSetupException, IOException{
- FragmentNode b = getRootFragment("/physical_simpleexchange.json");
- assertEquals(2, getFragmentCount(b));
+ public void ensureThreeFragments() throws FragmentSetupException, IOException {
+ PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+ Fragment b = getRootFragment(ppr, "/physical_simpleexchange.json");
+ logger.debug("Fragment Node {}", b);
+ assertEquals(3, getFragmentCount(b));
assertEquals(1, b.getReceivingExchangePairs().size());
assertNull(b.getSendingExchange());
-
+
// get first child.
b = b.iterator().next().getNode();
+ assertEquals(1, b.getReceivingExchangePairs().size());
+ assertNotNull(b.getSendingExchange());
+
+ b = b.iterator().next().getNode();
assertEquals(0, b.getReceivingExchangePairs().size());
assertNotNull(b.getSendingExchange());
}
+
- private int getFragmentCount(FragmentNode b){
- int i =1;
- for(ExchangeFragmentPair p : b){
- i += getFragmentCount(p.getNode());
- }
- return i;
- }
+
+
- private FragmentNode getRootFragment(String file) throws FragmentSetupException, IOException{
- FragmentingPhysicalVisitor f = new FragmentingPhysicalVisitor();
-
- PhysicalPlan plan = PhysicalPlan.parse(config.getMapper().reader(PhysicalPlan.class), Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
- PhysicalOperator o = plan.getSortedOperators(false).iterator().next();
- return o.accept(f, null);
- }
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
index b8fd278..1d0fb91 100644
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/CheckInjectionValue.java
@@ -22,12 +22,12 @@ import static org.junit.Assert.*;
import java.util.List;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.Screen;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Screen;
import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.server.DrillbitContext;
import org.apache.drill.exec.store.StorageEngineRegistry;
import org.junit.BeforeClass;
@@ -48,8 +48,8 @@ public class CheckInjectionValue {
@Test
public void testInjected() throws Exception{
- PhysicalPlanReader r = new PhysicalPlanReader(config.getMapper(), DrillbitEndpoint.getDefaultInstance());
- PhysicalPlan p = r.read(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
+ PhysicalPlanReader r = new PhysicalPlanReader(config, config.getMapper(), DrillbitEndpoint.getDefaultInstance());
+ PhysicalPlan p = r.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/physical_screen.json"), Charsets.UTF_8));
List<PhysicalOperator> o = p.getSortedOperators(false);
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
new file mode 100644
index 0000000..6f229a3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/FragmentChecker.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * 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.drill.exec.pop;
+
+import static org.junit.Assert.*;
+
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.work.QueryWorkUnit;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class FragmentChecker extends PopUnitTestBase{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentChecker.class);
+
+
+ @Test
+ public void checkSimpleExchangePlan() throws Exception{
+
+ PhysicalPlanReader ppr = new PhysicalPlanReader(CONFIG, CONFIG.getMapper(), DrillbitEndpoint.getDefaultInstance());
+ Fragment fragmentRoot = getRootFragment(ppr, "/physical_simpleexchange.json");
+ PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
+ SimpleParallelizer par = new SimpleParallelizer();
+
+ DrillbitEndpoint b1 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(1234).build();
+ DrillbitEndpoint b2 = DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(2345).build();
+
+ QueryWorkUnit qwu = par.getFragments(b1, QueryId.getDefaultInstance(), Lists.newArrayList(b1, b2), ppr, fragmentRoot, planningSet, 10);
+ assertEquals(qwu.getFragments().size(), 3);
+ System.out.println("=========ROOT FRAGMENT=========");
+ System.out.print(qwu.getRootFragment().getFragmentJson());
+
+
+ for(PlanFragment f : qwu.getFragments()){
+ System.out.println("=========");
+ System.out.print(f.getFragmentJson());
+ }
+ logger.debug("Planning Set {}", planningSet);
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
new file mode 100644
index 0000000..e5e109e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/pop/PopUnitTestBase.java
@@ -0,0 +1,71 @@
+/*******************************************************************************
+ * 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.drill.exec.pop;
+
+import java.io.IOException;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.junit.rules.Timeout;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public abstract class PopUnitTestBase {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PopUnitTestBase.class);
+
+ // Determine if we are in Eclipse Debug mode.
+ static final boolean IS_DEBUG = java.lang.management.ManagementFactory.getRuntimeMXBean().getInputArguments().toString().indexOf("-agentlib:jdwp") > 0;
+ protected static DrillConfig CONFIG;
+
+ // Set a timeout unless we're debugging.
+ @Rule public TestRule globalTimeout = IS_DEBUG ? new TestName() : new Timeout(10000);
+
+ @BeforeClass
+ public static void setup() {
+ CONFIG = DrillConfig.create();
+ }
+
+
+ public static int getFragmentCount(Fragment b) {
+ int i = 1;
+ for (ExchangeFragmentPair p : b) {
+ i += getFragmentCount(p.getNode());
+ }
+ return i;
+ }
+
+ public Fragment getRootFragment(PhysicalPlanReader reader, String file) throws FragmentSetupException, IOException {
+ MakeFragmentsVisitor f = new MakeFragmentsVisitor();
+
+ PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
+ PhysicalOperator o = plan.getSortedOperators(false).iterator().next();
+ return o.accept(f, null);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
deleted file mode 100644
index d003373..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/RunRemoteQuery.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.rpc.user;
-
-import io.netty.buffer.UnpooledByteBufAllocator;
-import io.netty.channel.nio.NioEventLoopGroup;
-
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.junit.Test;
-
-public class RunRemoteQuery {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunRemoteQuery.class);
-
- @Test
- public void runRemoteQuery() throws Exception{
- UserClient c = new UserClient(UnpooledByteBufAllocator.DEFAULT, new NioEventLoopGroup(1));
- c.connectAsClient("localhost", 31010);
- DrillRpcFuture<QueryHandle> futureHandle = c.submitQuery(RunQuery.getDefaultInstance());
- QueryHandle h = futureHandle.checkedGet();
- System.out.println(h);
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
deleted file mode 100644
index c8ce877..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/UserRpcTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.rpc.user;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
-import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.channel.nio.NioEventLoopGroup;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.proto.UserProtos.QueryResultsMode;
-import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.NamedThreadFactory;
-import org.junit.Test;
-
-public class UserRpcTest {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserRpcTest.class);
-
-
-
-
- @Test
- public void doBasicRpcTest() throws Exception {
- final int bufferSize = 25000;
- final int batchSize = 1000;
- final int batchCount = 100;
-
-
- int sends = 0;
- int receives = 0;
- long nanoSend = 0;
- long nanoReceive = 0;
-
-
- try {
- ByteBufAllocator bb = new PooledByteBufAllocator(true);
-// ByteBufAllocator bb = UnpooledByteBufAllocator.DEFAULT;
- UserServer s = new UserServer(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Server-")), null);
- s.bind(31515);
-
- logger.debug("Starting user client.");
- UserClient c = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
-
- logger.debug("Connecting as client to server.");
- c.connectAsClient("localhost", 31515);
-
-
- @SuppressWarnings("unchecked")
- DrillRpcFuture<QueryHandle>[] handles = new DrillRpcFuture[batchSize];
-
- for (int x = 0; x < batchCount; x++) {
- long s1 = System.nanoTime();
- for (int i = 0; i < batchSize; i++) {
- sends++;
- ByteBuf rawBody = bb.buffer(bufferSize);
- rawBody.writerIndex(bufferSize);
- if(rawBody.readableBytes() != bufferSize) throw new RuntimeException();
- handles[i] = c.submitQuery(RunQuery.newBuilder().setMode(QueryResultsMode.QUERY_FOR_STATUS).build(), rawBody);
- }
-
- long s2 = System.nanoTime();
-
- for (int i = 0; i < batchSize; i++) {
- handles[i].checkedGet(2, TimeUnit.SECONDS).getQueryId();
- receives++;
- }
-
- long s3 = System.nanoTime();
- nanoSend += (s2-s1);
- nanoReceive += (s3-s2);
- logger.debug("Submission time {}ms, return time {}ms", (s2 - s1) / 1000 / 1000, (s3 - s2) / 1000 / 1000);
- }
- // logger.debug("Submitting query.");
- // DrillRpcFuture<QueryHandle> handleFuture =
- // c.submitQuery(RunQuery.newBuilder().setMode(QueryResultsMode.QUERY_FOR_STATUS).build());
- //
- // logger.debug("Got query id handle of {}", handleFuture.get(2, TimeUnit.SECONDS).getQueryId());
- } catch (Exception e) {
- logger.error("Exception of type {} occurred while doing test.", e.getClass().getCanonicalName());
- throw e;
- } finally{
- long mbsTransferred = (1l * bufferSize * batchSize * batchCount)/1024/1024;
- double sSend = nanoSend*1.0d/1000/1000/1000;
- double sReceive = nanoReceive*1.0d/1000/1000/1000;
- logger.info(String.format("Completed %d sends and %d receives. Total data transferred was %d. Send bw: %f, Receive bw: %f.", sends, receives, mbsTransferred, mbsTransferred*1.0/sSend, mbsTransferred*1.0/sReceive));
- logger.info("Completed {} sends and {} receives.", sends, receives);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
new file mode 100644
index 0000000..9684e9f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestBitRpc.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * 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.drill.exec.server;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitClient;
+import org.apache.drill.exec.rpc.bit.BitComImpl;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.rpc.bit.BitRpcConfig;
+import org.apache.drill.exec.rpc.bit.BitServer;
+import org.apache.drill.exec.rpc.bit.ListenerPool;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class TestBitRpc {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestBitRpc.class);
+
+ @Test
+ public void testBasicConnectionAndHandshake() throws Exception{
+ int port = 1234;
+ BootStrapContext c = new BootStrapContext(DrillConfig.create());
+ ConcurrentMap<DrillbitEndpoint, BitConnection> registry = Maps.newConcurrentMap();
+ BitServer server = new BitServer(new BitComTestHandler(), c, registry, new ListenerPool(2));
+ port = server.bind(port);
+ for(int i =0; i < 10; i++){
+ BitClient client = new BitClient(DrillbitEndpoint.newBuilder().setAddress("localhost").setBitPort(port).build(), null, new BitComTestHandler(), c, registry, new ListenerPool(2));
+ client.connect();
+
+ }
+ System.out.println("connected");
+ }
+
+
+
+ private class BitComTestHandler implements BitComHandler{
+
+ @Override
+ public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return BitRpcConfig.OK;
+ }
+
+ @Override
+ public void startNewRemoteFragment(PlanFragment fragment) {
+ }
+
+ @Override
+ public Ack cancelFragment(FragmentHandle handle) {
+ return null;
+ }
+
+ @Override
+ public void registerIncomingFragmentHandler(IncomingFragmentHandler handler) {
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
deleted file mode 100644
index 18c6955..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordConfig.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.store;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.logical.StorageEngineConfigBase;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-config")
-public class MockRecordConfig extends StorageEngineConfigBase{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordConfig.class);
-
- private int recordCount;
- private DataType[] types;
-
- public int getRecordCount() {
- return recordCount;
- }
- public void setRecordCount(int recordCount) {
- this.recordCount = recordCount;
- }
- public DataType[] getTypes() {
- return types;
- }
- public void setTypes(DataType[] types) {
- this.types = types;
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
deleted file mode 100644
index e1f56bd..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockRecordReader.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.store;
-
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OutputMutator;
-import org.apache.drill.exec.record.BatchSchema;
-import org.apache.drill.exec.record.vector.Int16Vector;
-import org.apache.drill.exec.record.vector.Int32Vector;
-import org.apache.drill.exec.record.vector.ValueVector;
-
-public class MockRecordReader implements RecordReader {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockRecordReader.class);
-
- private BatchSchema expectedSchema;
- private OutputMutator output;
- private MockRecordConfig config;
- private FragmentContext context;
- private ValueVector<?>[] valueVectors;
- private int recordsRead;
-
- public MockRecordReader(FragmentContext context, MockRecordConfig config) {
- this.config = config;
- }
-
- private int getEstimatedRecordSize(DataType[] types) {
- int x = 0;
- for (int i = 0; i < types.length; i++) {
- x += getEstimatedColumnSize(i);
- }
- return x;
- }
-
- private int getEstimatedColumnSize(int fieldId) {
- return 4;
- }
-
- private ValueVector<?> getVector(int fieldId, DataType dt, int length) {
- ValueVector<?> v;
- if (dt == DataType.INT16) {
- v = new Int16Vector(fieldId, context.getAllocator());
- } else if (dt == DataType.INT32) {
- v = new Int32Vector(fieldId, context.getAllocator());
- } else {
- throw new UnsupportedOperationException();
- }
- v.allocateNew(length);
- return v;
-
- }
-
- @Override
- public void setup(BatchSchema expectedSchema, OutputMutator output) throws ExecutionSetupException {
- try {
- this.expectedSchema = expectedSchema;
- this.output = output;
- int estimateRowSize = getEstimatedRecordSize(config.getTypes());
- valueVectors = new ValueVector<?>[config.getTypes().length];
- int batchRecordCount = 250000 / estimateRowSize;
-
- for (int i = 0; i < config.getTypes().length; i++) {
- valueVectors[i] = getVector(i, config.getTypes()[i], batchRecordCount);
- output.addField(i, valueVectors[i]);
- }
- } catch (SchemaChangeException e) {
- throw new ExecutionSetupException("Failure while setting up fields", e);
- }
-
- }
-
- @Override
- public int next() {
- int recordSetSize = Math.min(valueVectors[0].size(), this.config.getRecordCount()- recordsRead);
- recordsRead += recordSetSize;
- return recordSetSize;
- }
-
- @Override
- public void cleanup() {
- for (int i = 0; i < valueVectors.length; i++) {
- try {
- output.removeField(valueVectors[i].getField().getFieldId());
- } catch (SchemaChangeException e) {
- logger.warn("Failure while trying tremove field.", e);
- }
- valueVectors[i].close();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java b/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
deleted file mode 100644
index cc82540..0000000
--- a/sandbox/prototype/exec/java-exec/src/test/java/org/apache/drill/exec/store/MockStorageEngine.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.store;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.FragmentContext;
-
-import com.google.common.collect.ListMultimap;
-
-public class MockStorageEngine extends AbstractStorageEngine{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
-
- @Override
- public boolean supportsRead() {
- return true;
- }
-
- @Override
- public Collection<ReadEntry> getReadEntries(Scan scan) throws IOException {
- return null;
- }
-
- @Override
- public ListMultimap<ReadEntry, DrillbitEndpoint> getReadLocations(Collection<ReadEntry> entries) {
- return null;
- }
-
- @Override
- public RecordReader getReader(FragmentContext context, ReadEntry readEntry) throws IOException {
- return null;
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
index 8785736..771a2fd 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/drill-module.conf
@@ -7,6 +7,9 @@ drill.exec: {
user.port : 31010,
bit.port : 32010
},
+ operator: {
+ packages += "org.apache.drill.exec.physical.config"
+ },
optimizer: {
implementation: "org.apache.drill.exec.opt.IdentityOptimizer"
},
@@ -30,4 +33,4 @@ drill.exec: {
network: {
start: 35000
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
index b79b811..3af6e10 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/logback.xml
@@ -1,16 +1,14 @@
<?xml version="1.0" encoding="UTF-8" ?>
<configuration>
-
+<!--
<appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
<Compressing>true</Compressing>
<ReconnectionDelay>10000</ReconnectionDelay>
<IncludeCallerData>true</IncludeCallerData>
<RemoteHosts>localhost</RemoteHosts>
</appender>
-
+-->
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
- <!-- encoders are assigned the type
- ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
<encoder>
<pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
</encoder>
@@ -30,14 +28,14 @@
-->
<logger name="org.apache.drill" additivity="false">
<level value="debug" />
- <appender-ref ref="SOCKET" />
+<!-- <appender-ref ref="SOCKET" /> -->
<appender-ref ref="STDOUT" />
<!-- <appender-ref ref="FILE" /> -->
</logger>
<root>
<level value="error" />
- <appender-ref ref="SOCKET" />
+<!-- <appender-ref ref="SOCKET" /> -->
<appender-ref ref="STDOUT" />
<!-- <appender-ref ref="FILE" /> -->
</root>
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
index c63aef1..8bb97db 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_screen.json
@@ -12,9 +12,8 @@
pop:"mock-scan",
url: "http://apache.org",
entries:[
- {id:1}
- ],
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+ {id:1, records: 100, size: 100}
+ ]
},
{
@id:2,
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
index e332785..85823cf 100644
--- a/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_simpleexchange.json
@@ -12,30 +12,44 @@
pop:"mock-scan",
url: "http://apache.org",
entries:[
- {id:1}
- ],
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]},
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]}
+ ]
},
{
@id:2,
- pop: "partition-to-random-exchange",
- child: 1,
- partition: {
- mode: "DUPLICATE"
- }
+ pop: "hash-to-random-exchange",
+ child: 1
},
{
@id:3,
child: 2,
pop:"filter",
expr: "b > 5",
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+ selectivity: 0.8
},
{
@id: 4,
child: 3,
- pop: "mock-store",
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
+ pop: "mock-store"
+ },
+ {
+ @id:5,
+ child: 4,
+ pop: "union-exchange"
+ },
+ {
+ @id: 6,
+ child: 5,
+ pop: "screen"
}
]
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
new file mode 100644
index 0000000..675ecfb
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_single_exchange.json
@@ -0,0 +1,34 @@
+{
+ head:{
+ type:"APACHE_DRILL_PHYSICAL",
+ version:"1",
+ generator:{
+ type:"manual"
+ }
+ },
+ graph:[
+ {
+ @id:1,
+ pop:"mock-scan",
+ url: "http://apache.org",
+ entries:[
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]}
+
+ ]
+ },
+ {
+ @id:2,
+ child: 1,
+ pop: "union-exchange"
+ },
+ {
+ @id: 3,
+ child: 2,
+ pop: "screen"
+ }
+ ]
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
new file mode 100644
index 0000000..0ddd48f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test1.json
@@ -0,0 +1,40 @@
+{
+ head:{
+ type:"APACHE_DRILL_PHYSICAL",
+ version:"1",
+ generator:{
+ type:"manual"
+ }
+ },
+ graph:[
+ {
+ @id:1,
+ pop:"mock-scan",
+ url: "http://apache.org",
+ entries:[
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]}
+ ]
+ },
+ {
+ @id:2,
+ child: 1,
+ pop:"filter",
+ expr: "b > 5",
+ selectivity: 0.80
+ },
+ {
+ @id: 3,
+ child: 2,
+ pop: "mock-store"
+ },
+ {
+ @id: 4,
+ child: 3,
+ pop: "screen"
+ }
+ ]
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
new file mode 100644
index 0000000..b001921
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/test/resources/physical_test2.json
@@ -0,0 +1,34 @@
+{
+ head:{
+ type:"APACHE_DRILL_PHYSICAL",
+ version:"1",
+ generator:{
+ type:"manual"
+ }
+ },
+ graph:[
+ {
+ @id:1,
+ pop:"mock-scan",
+ url: "http://apache.org",
+ entries:[
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]},
+ {records: 100, types: [
+ {name: "blue", type: "INT", mode: "REQUIRED"},
+ {name: "red", type: "BIGINT", mode: "REQUIRED"},
+ {name: "green", type: "INT", mode: "REQUIRED"}
+ ]}
+ ]
+ },
+ {
+ @id: 2,
+ child: 1,
+ pop: "screen"
+ }
+ ]
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/ref/src/test/resources/donuts.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/ref/src/test/resources/donuts.json b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
index 2d98b59..9fd6e3e 100644
--- a/sandbox/prototype/exec/ref/src/test/resources/donuts.json
+++ b/sandbox/prototype/exec/ref/src/test/resources/donuts.json
@@ -20,7 +20,7 @@
[
{ "id": "5001", "type": "None" },
{ "id": "5002", "type": "Glazed" },
- { "id": "5005", "type": "Sugar" },
+ { "id": "5005", "type": "Sugar", color: "White"},
{ "id": "5007", "type": "Powdered Sugar" },
{ "id": "5006", "type": "Chocolate with Sprinkles" },
{ "id": "5003", "type": "Chocolate" },
[04/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
new file mode 100644
index 0000000..73980f9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnection.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.util.concurrent.GenericFutureListener;
+
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.RpcBus;
+
+import com.google.common.base.Preconditions;
+import com.google.common.io.Closeables;
+
+public class BitConnection extends RemoteConnection{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnection.class);
+
+ private final RpcBus<RpcType, BitConnection> bus;
+ private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+ private final ListenerPool listeners;
+
+ private final AvailabilityListener listener;
+ private volatile DrillbitEndpoint endpoint;
+ private volatile boolean active = false;
+ private final UUID id;
+
+ public BitConnection(AvailabilityListener listener, Channel channel, RpcBus<RpcType, BitConnection> bus, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners){
+ super(channel);
+ this.bus = bus;
+ this.registry = registry;
+ // we use a local listener pool unless a global one is provided.
+ this.listeners = listeners != null ? listeners : new ListenerPool(2);
+ this.listener = listener;
+ this.id = UUID.randomUUID();
+ }
+
+ protected DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+ public ListenerPool getListenerPool(){
+ return listeners;
+ }
+
+ protected void setEndpoint(DrillbitEndpoint endpoint) {
+ Preconditions.checkNotNull(endpoint);
+ Preconditions.checkArgument(this.endpoint == null);
+
+ this.endpoint = endpoint;
+ BitServer.logger.debug("Adding new endpoint to available BitServer connections. Endpoint: {}.", endpoint);
+ synchronized(this){
+ BitConnection c = registry.putIfAbsent(endpoint, this);
+
+ if(c != null){ // the registry already has a connection like this
+
+ // give the awaiting future an alternative connection.
+ if(listener != null){
+ listener.isAvailable(c);
+ }
+
+ // shut this down if this is a client as it won't be available in the registry.
+ // otherwise we'll leave as, possibly allowing to bit coms to use different tunnels to talk to each other. This shouldn't cause a problem.
+ logger.debug("Shutting down connection to {} since the registry already has an active connection that endpoint.", endpoint);
+ shutdownIfClient();
+
+ }
+ active = true;
+ if(listener != null) listener.isAvailable(this);
+ }
+ }
+
+ public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch){
+ return bus.send(this, RpcType.REQ_RECORD_BATCH, batch.getHeader(), Ack.class, batch.getBuffers());
+ }
+
+ public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment){
+ return bus.send(this, RpcType.REQ_INIATILIZE_FRAGMENT, fragment, Ack.class);
+ }
+
+ public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle){
+ return bus.send(this, RpcType.REQ_CANCEL_FRAGMENT, handle, Ack.class);
+ }
+
+ public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
+ return bus.send(this, RpcType.REQ_FRAGMENT_STATUS, status, Ack.class);
+ }
+
+ public void disable(){
+ active = false;
+ }
+
+ public boolean isActive(){
+ return active;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((id == null) ? 0 : id.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null) return false;
+ if (getClass() != obj.getClass()) return false;
+ BitConnection other = (BitConnection) obj;
+ if (id == null) {
+ if (other.id != null) return false;
+ } else if (!id.equals(other.id)) return false;
+ return true;
+ }
+
+ public GenericFutureListener<ChannelFuture> getCloseHandler(GenericFutureListener<ChannelFuture> parent){
+ return new CloseHandler(this, parent);
+ }
+
+ private class CloseHandler implements GenericFutureListener<ChannelFuture>{
+ private BitConnection connection;
+ private GenericFutureListener<ChannelFuture> parent;
+
+ public CloseHandler(BitConnection connection, GenericFutureListener<ChannelFuture> parent) {
+ super();
+ this.connection = connection;
+ this.parent = parent;
+ }
+
+ @Override
+ public void operationComplete(ChannelFuture future) throws Exception {
+ if(connection.getEndpoint() != null) registry.remove(connection.getEndpoint(), connection);
+ parent.operationComplete(future);
+ }
+
+ }
+
+ public void shutdownIfClient(){
+ if(bus.isClient()) Closeables.closeQuietly(bus);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
new file mode 100644
index 0000000..0160d24
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitConnectionManager.java
@@ -0,0 +1,80 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.common.util.concurrent.CheckedFuture;
+
+public class BitConnectionManager {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitConnectionManager.class);
+
+ private final int maxAttempts;
+ private final BitComImpl com;
+ private final DrillbitEndpoint endpoint;
+ private final AtomicReference<BitConnection> connection;
+ private final AtomicReference<CheckedFuture<BitConnection, RpcException>> future;
+
+ BitConnectionManager(DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection, CheckedFuture<BitConnection, RpcException> future, int maxAttempts) {
+ assert endpoint != null && endpoint.getAddress() != null && endpoint.getBitPort() > 0;
+ this.com = com;
+ this.connection = new AtomicReference<BitConnection>(connection);
+ this.future = new AtomicReference<CheckedFuture<BitConnection, RpcException>>(future);
+ this.endpoint = endpoint;
+ this.maxAttempts = maxAttempts;
+ }
+
+ BitConnection getConnection(int attempt) throws RpcException{
+ BitConnection con = connection.get();
+
+ if(con != null){
+ if(con.isActive()) return con;
+ connection.compareAndSet(con, null);
+ }
+
+ CheckedFuture<BitConnection, RpcException> fut = future.get();
+
+ if(fut != null){
+ try{
+ return fut.checkedGet();
+ }catch(RpcException ex){
+ future.compareAndSet(fut, null);
+ if(attempt < maxAttempts){
+ return getConnection(attempt + 1);
+ }else{
+ throw ex;
+ }
+ }
+ }
+
+ // no checked future, let's make one.
+ fut = com.getConnectionAsync(endpoint);
+ future.compareAndSet(null, fut);
+ return getConnection(attempt);
+
+ }
+
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
new file mode 100644
index 0000000..32fd4f9
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitRpcConfig.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConfig;
+
+public class BitRpcConfig {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitRpcConfig.class);
+
+ public static RpcConfig MAPPING = RpcConfig.newBuilder("BIT-RPC-MAPPING") //
+ .add(RpcType.HANDSHAKE, BitHandshake.class, RpcType.HANDSHAKE, BitHandshake.class)
+ .add(RpcType.REQ_INIATILIZE_FRAGMENT, PlanFragment.class, RpcType.ACK, Ack.class)
+ .add(RpcType.REQ_CANCEL_FRAGMENT, FragmentHandle.class, RpcType.ACK, Ack.class)
+ .add(RpcType.REQ_FRAGMENT_STATUS, FragmentStatus.class, RpcType.ACK, Ack.class)
+ .add(RpcType.REQ_RECORD_BATCH, FragmentRecordBatch.class, RpcType.ACK, Ack.class)
+ .build();
+
+ public static int RPC_VERSION = 2;
+
+ public static final Response OK = new Response(RpcType.ACK, Acks.OK);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
index e17b25c..88ac6cc 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitServer.java
@@ -18,47 +18,76 @@
package org.apache.drill.exec.rpc.bit;
import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
+import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
-import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
import io.netty.util.concurrent.GenericFutureListener;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.rpc.BasicServer;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
import com.google.protobuf.MessageLite;
-public class BitServer extends BasicServer<RpcType>{
+public class BitServer extends BasicServer<RpcType, BitConnection>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitServer.class);
- private final DrillbitContext context;
private final BitComHandler handler;
+ private final ConcurrentMap<DrillbitEndpoint, BitConnection> registry;
+ private final ListenerPool listeners;
- public BitServer(BitComHandler handler, ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
- super(alloc, eventLoopGroup);
- this.context = context;
+ public BitServer(BitComHandler handler, BootStrapContext context, ConcurrentMap<DrillbitEndpoint, BitConnection> registry, ListenerPool listeners) {
+ super(BitRpcConfig.MAPPING, context.getAllocator().getUnderlyingAllocator(), context.getBitLoopGroup());
this.handler = handler;
+ this.registry = registry;
+ this.listeners = listeners;
}
@Override
- protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- return handler.getResponseDefaultInstance(rpcType);
+ public MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
+ return BitComDefaultInstanceHandler.getResponseDefaultInstance(rpcType);
}
@Override
- protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- return handler.handle(context, rpcType, pBody, dBody);
+ protected Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ return handler.handle(connection, rpcType, pBody, dBody);
}
@Override
- protected GenericFutureListener<ChannelFuture> getCloseHandler(SocketChannel ch) {
-
- return super.getCloseHandler(ch);
+ protected GenericFutureListener<ChannelFuture> getCloseHandler(BitConnection connection) {
+ return connection.getCloseHandler(super.getCloseHandler(connection));
+ }
+
+ @Override
+ public BitConnection initRemoteConnection(Channel channel) {
+ return new BitConnection(null, channel, this, registry, listeners);
+ }
+
+
+ @Override
+ protected ServerHandshakeHandler<BitHandshake> getHandshakeHandler() {
+ return new ServerHandshakeHandler<BitHandshake>(RpcType.HANDSHAKE, BitHandshake.PARSER){
+
+ @Override
+ public MessageLite getHandshakeResponse(BitHandshake inbound) throws Exception {
+ logger.debug("Handling handshake from other bit. {}", inbound);
+ if(inbound.getRpcVersion() != BitRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), BitRpcConfig.RPC_VERSION));
+ return BitHandshake.newBuilder().setRpcVersion(BitRpcConfig.RPC_VERSION).build();
+ }
+
+ };
}
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
index 02991ad..652fa52 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/BitTunnel.java
@@ -17,47 +17,222 @@
******************************************************************************/
package org.apache.drill.exec.rpc.bit;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
-import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.record.FragmentWritableBatch;
+import org.apache.drill.exec.record.WritableBatch;
import org.apache.drill.exec.rpc.DrillRpcFuture;
-import org.apache.drill.exec.rpc.RpcBus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
-import com.google.common.io.Closeables;
+import com.google.common.util.concurrent.AbstractCheckedFuture;
+import com.google.common.util.concurrent.CheckedFuture;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.SettableFuture;
/**
- * Interface provided for communication between two bits. Provided by both a server and a client implementation.
+ * Interface provided for communication between two bits. Underlying connection may be server or client based. Resilient
+ * to connection loss. Right now, this has to jump through some hoops and bridge futures between the connection creation
+ * and action. A better approach should be done.
*/
public class BitTunnel {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitTunnel.class);
- final RpcBus<?> bus;
+ private static final int MAX_ATTEMPTS = 3;
- public BitTunnel(RpcBus<?> bus){
- this.bus = bus;
- }
+ private final BitConnectionManager manager;
+ private final Executor exec;
- public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, RecordBatch batch){
- return null;
+
+ public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com, BitConnection connection) {
+ this.manager = new BitConnectionManager(endpoint, com, connection, null, MAX_ATTEMPTS);
+ this.exec = exec;
}
-
- public DrillRpcFuture<FragmentHandle> sendFragment(FragmentContext context, PlanFragment fragment){
- return null;
+
+ public BitTunnel(Executor exec, DrillbitEndpoint endpoint, BitComImpl com,
+ CheckedFuture<BitConnection, RpcException> future) {
+ this.manager = new BitConnectionManager(endpoint, com, (BitConnection) null, future, MAX_ATTEMPTS);
+ this.exec = exec;
}
- public DrillRpcFuture<Ack> cancelFragment(FragmentContext context, FragmentHandle handle){
- return null;
+ public DrillbitEndpoint getEndpoint(){
+ return manager.getEndpoint();
}
-
- public DrillRpcFuture<FragmentStatus> getFragmentStatus(FragmentContext context, FragmentHandle handle){
- return null;
+
+ private <T> DrillRpcFuture<T> submit(BitCommand<T> command) {
+ exec.execute(command);
+ return command;
+ }
+
+ public DrillRpcFuture<Ack> sendRecordBatch(FragmentContext context, FragmentWritableBatch batch) {
+ return submit(new SendBatch(batch, context));
+ }
+
+ public DrillRpcFuture<Ack> sendFragment(PlanFragment fragment) {
+ return submit(new SendFragment(fragment));
+ }
+
+ public DrillRpcFuture<Ack> cancelFragment(FragmentHandle handle) {
+ return submit(new CancelFragment(handle));
}
+
+ public DrillRpcFuture<Ack> sendFragmentStatus(FragmentStatus status){
+ return submit(new SendFragmentStatus(status));
+ }
+
+ public class SendBatch extends BitCommand<Ack> {
+ final FragmentWritableBatch batch;
+ final FragmentContext context;
+
+ public SendBatch(FragmentWritableBatch batch, FragmentContext context) {
+ super();
+ this.batch = batch;
+ this.context = context;
+ }
+
+ @Override
+ public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+ logger.debug("Sending record batch. {}", batch);
+ return connection.sendRecordBatch(context, batch);
+ }
+
+ }
+
+ public class SendFragmentStatus extends BitCommand<Ack> {
+ final FragmentStatus status;
+
+ public SendFragmentStatus(FragmentStatus status) {
+ super();
+ this.status = status;
+ }
+
+ @Override
+ public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+ return connection.sendFragmentStatus(status);
+ }
+ }
+
+ public class CancelFragment extends BitCommand<Ack> {
+ final FragmentHandle handle;
+
+ public CancelFragment(FragmentHandle handle) {
+ super();
+ this.handle = handle;
+ }
+
+ @Override
+ public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+ return connection.cancelFragment(handle);
+ }
+
+ }
+
+ public class SendFragment extends BitCommand<Ack> {
+ final PlanFragment fragment;
+
+ public SendFragment(PlanFragment fragment) {
+ super();
+ this.fragment = fragment;
+ }
+
+ @Override
+ public CheckedFuture<Ack, RpcException> doRpcCall(BitConnection connection) {
+ return connection.sendFragment(fragment);
+ }
+
+ }
+
+
- public void shutdownIfClient(){
- if(bus.isClient()) Closeables.closeQuietly(bus);
+
+ private abstract class BitCommand<T> extends AbstractCheckedFuture<T, RpcException> implements Runnable, DrillRpcFuture<T> {
+
+ public void addLightListener(RpcOutcomeListener<T> outcomeListener){
+ this.addListener(new RpcOutcomeListenerWrapper(outcomeListener), MoreExecutors.sameThreadExecutor());
+ }
+
+ public BitCommand() {
+ super(SettableFuture.<T> create());
+ }
+
+ public abstract CheckedFuture<T, RpcException> doRpcCall(BitConnection connection);
+
+ public final void run() {
+
+ try {
+
+ BitConnection connection = manager.getConnection(0);
+ assert connection != null : "The connection manager should never return a null connection. Worse case, it should throw an exception.";
+ CheckedFuture<T, RpcException> rpc = doRpcCall(connection);
+ rpc.addListener(new FutureBridge<T>((SettableFuture<T>) delegate(), rpc), MoreExecutors.sameThreadExecutor());
+ } catch (RpcException ex) {
+ ((SettableFuture<T>) delegate()).setException(ex);
+ }
+
+ }
+
+ @Override
+ protected RpcException mapException(Exception e) {
+ Throwable t = e;
+ if (e instanceof ExecutionException) {
+ t = e.getCause();
+ }
+ if (t instanceof RpcException) return (RpcException) t;
+ return new RpcException(t);
+ }
+
+ public class RpcOutcomeListenerWrapper implements Runnable{
+ final RpcOutcomeListener<T> inner;
+
+ public RpcOutcomeListenerWrapper(RpcOutcomeListener<T> inner) {
+ this.inner = inner;
+ }
+
+ @Override
+ public void run() {
+ try{
+ inner.success(BitCommand.this.checkedGet());
+ }catch(RpcException e){
+ inner.failed(e);
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "BitCommand ["+this.getClass().getSimpleName()+"]";
+ }
+
+
+
+ }
+
+ private class FutureBridge<T> implements Runnable {
+ final SettableFuture<T> out;
+ final CheckedFuture<T, RpcException> in;
+
+ public FutureBridge(SettableFuture<T> out, CheckedFuture<T, RpcException> in) {
+ super();
+ this.out = out;
+ this.in = in;
+ }
+
+ @Override
+ public void run() {
+ try {
+ out.set(in.checkedGet());
+ } catch (RpcException ex) {
+ out.setException(ex);
+ }
+ }
+
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
new file mode 100644
index 0000000..8f299d2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/bit/ListenerPool.java
@@ -0,0 +1,56 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.bit;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.work.foreman.FragmentStatusListener;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+
+public class ListenerPool {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ListenerPool.class);
+
+ private final ConcurrentMap<FragmentHandle, FragmentStatusListener> listeners;
+
+ public ListenerPool(int par){
+ listeners = new ConcurrentHashMap<FragmentHandle, FragmentStatusListener>(16, 0.75f, par);
+ }
+
+ public void removeFragmentStatusListener(FragmentHandle handle) throws RpcException{
+ listeners.remove(handle);
+ }
+
+ public void addFragmentStatusListener(FragmentHandle handle, FragmentStatusListener listener) throws RpcException{
+ FragmentStatusListener old = listeners.putIfAbsent(handle, listener);
+ if(old != null) throw new RpcException("Failure. The provided handle already exists in the listener pool. You need to remove one listener before adding another.");
+ }
+
+ public void status(FragmentStatus status){
+ FragmentStatusListener l = listeners.get(status.getHandle());
+ if(l == null){
+ logger.info("A fragment message arrived but there was no registered listener for that message.");
+ return;
+ }else{
+ l.statusUpdate(status);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
new file mode 100644
index 0000000..3df88b7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/QueryResultBatch.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.user;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+
+public class QueryResultBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryResultBatch.class);
+
+ private final QueryResult header;
+ private final ByteBuf data;
+
+ public QueryResultBatch(QueryResult header, ByteBuf data) {
+// logger.debug("New Result Batch with header {} and data {}", header, data);
+ this.header = header;
+ this.data = data;
+ }
+
+ public QueryResult getHeader() {
+ return header;
+ }
+
+ public ByteBuf getData() {
+ return data;
+ }
+
+
+ public boolean hasData(){
+ return data != null;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
index 0088522..5d2e799 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserClient.java
@@ -20,57 +20,194 @@ package org.apache.drill.exec.rpc.user;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
import org.apache.drill.exec.proto.UserProtos.QueryResult;
import org.apache.drill.exec.proto.UserProtos.RpcType;
import org.apache.drill.exec.proto.UserProtos.RunQuery;
-import org.apache.drill.exec.rpc.BasicClient;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.BasicClientWithConnection;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Queues;
import com.google.protobuf.MessageLite;
-public class UserClient extends BasicClient<RpcType> {
+public class UserClient extends BasicClientWithConnection<RpcType> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserClient.class);
-
+
+ private ConcurrentMap<QueryId, UserResultsListener> resultsListener = Maps.newConcurrentMap();
+
public UserClient(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup) {
- super(alloc, eventLoopGroup);
+ super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
}
+ public Future<Void> submitQuery(RunQuery query, UserResultsListener resultsListener) throws RpcException {
+ this.send(RpcType.RUN_QUERY, query, QueryId.class).addLightListener(new SubmissionListener(resultsListener));
+ return resultsListener.getFuture();
+ }
- public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query) throws RpcException {
- return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, null);
+ public BitToUserHandshake connect(DrillbitEndpoint endpoint) throws RpcException, InterruptedException{
+ return this.connectAsClient(RpcType.HANDSHAKE, UserToBitHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build(), endpoint.getAddress(), endpoint.getUserPort(), BitToUserHandshake.class);
}
- public DrillRpcFuture<QueryHandle> submitQuery(RunQuery query, ByteBuf data) throws RpcException {
- return this.send(RpcType.RUN_QUERY, query, QueryHandle.class, data);
+ private class BufferingListener extends UserResultsListener {
+
+ private ConcurrentLinkedQueue<QueryResultBatch> results = Queues.newConcurrentLinkedQueue();
+ private ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+ private volatile UserResultsListener output;
+
+ public boolean transferTo(UserResultsListener l) {
+ lock.writeLock().lock();
+ output = l;
+ boolean last = false;
+ for (QueryResultBatch r : results) {
+ l.resultArrived(r);
+ last = r.getHeader().getIsLastChunk();
+ }
+ if (future.isDone()) {
+ l.set();
+ }
+ return last;
+ }
+
+ @Override
+ public void resultArrived(QueryResultBatch result) {
+ logger.debug("Result arrvied.");
+ lock.readLock().lock();
+ try {
+ if (output == null) {
+ this.results.add(result);
+ } else {
+ output.resultArrived(result);
+ }
+
+ } finally {
+ lock.readLock().unlock();
+ }
+
+ }
+
+ @Override
+ public void submissionFailed(RpcException ex) {
+ throw new UnsupportedOperationException("You cannot report failed submissions to a buffering listener.");
+ }
+
+ }
+
+ private class SubmissionListener extends RpcOutcomeListener<QueryId> {
+ private UserResultsListener listener;
+
+ public SubmissionListener(UserResultsListener listener) {
+ super();
+ this.listener = listener;
+ }
+
+ @Override
+ public void failed(RpcException ex) {
+ listener.submissionFailed(ex);
+ }
+
+ @Override
+ public void success(QueryId queryId) {
+ logger.debug("Received QueryId {} succesfully. Adding listener {}", queryId, listener);
+ UserResultsListener oldListener = resultsListener.putIfAbsent(queryId, listener);
+
+ // we need to deal with the situation where we already received results by the time we got the query id back. In
+ // that case, we'll need to transfer the buffering listener over, grabbing a lock against reception of additional
+ // results during the transition
+ if (oldListener != null) {
+ logger.debug("Unable to place user results listener, buffering listener was already in place.");
+ if (oldListener instanceof BufferingListener) {
+ resultsListener.remove(oldListener);
+ boolean all = ((BufferingListener) oldListener).transferTo(this.listener);
+ // simply remove the buffering listener if we already have the last response.
+ if (all) {
+ resultsListener.remove(oldListener);
+ } else {
+ boolean replaced = resultsListener.replace(queryId, oldListener, listener);
+ if (!replaced) throw new IllegalStateException();
+ }
+ } else {
+ throw new IllegalStateException("Trying to replace a non-buffering User Results listener.");
+ }
+ }
+
+ }
+
}
@Override
protected MessageLite getResponseDefaultInstance(int rpcType) throws RpcException {
- switch(rpcType){
+ switch (rpcType) {
case RpcType.ACK_VALUE:
return Ack.getDefaultInstance();
case RpcType.HANDSHAKE_VALUE:
return BitToUserHandshake.getDefaultInstance();
case RpcType.QUERY_HANDLE_VALUE:
- return QueryHandle.getDefaultInstance();
+ return QueryId.getDefaultInstance();
case RpcType.QUERY_RESULT_VALUE:
return QueryResult.getDefaultInstance();
}
throw new RpcException(String.format("Unable to deal with RpcType of %d", rpcType));
}
+ protected Response handle(int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ switch (rpcType) {
+ case RpcType.QUERY_RESULT_VALUE:
+ final QueryResult result = get(pBody, QueryResult.PARSER);
+ final QueryResultBatch batch = new QueryResultBatch(result, dBody);
+ UserResultsListener l = resultsListener.get(result.getQueryId());
+// logger.debug("For QueryId [{}], retrieved result listener {}", result.getQueryId(), l);
+ if (l != null) {
+// logger.debug("Results listener available, using existing.");
+ l.resultArrived(batch);
+ if (result.getIsLastChunk()) {
+ resultsListener.remove(result.getQueryId(), l);
+ l.set();
+ }
+ } else {
+ logger.debug("Results listener not available, creating a buffering listener.");
+ // manage race condition where we start getting results before we receive the queryid back.
+ BufferingListener bl = new BufferingListener();
+ l = resultsListener.putIfAbsent(result.getQueryId(), bl);
+ if (l != null) {
+ l.resultArrived(batch);
+ } else {
+ bl.resultArrived(batch);
+ }
+ }
+
+ return new Response(RpcType.ACK, Ack.getDefaultInstance());
+ default:
+ throw new RpcException(String.format("Unknown Rpc Type %d. ", rpcType));
+ }
+
+ }
@Override
- protected Response handle(SocketChannel ch, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
- logger.debug("Received a server > client message of type " + rpcType);
- return new Response(RpcType.ACK, Ack.getDefaultInstance(), null);
+ protected ClientHandshakeHandler<BitToUserHandshake> getHandshakeHandler() {
+ return new ClientHandshakeHandler<BitToUserHandshake>(RpcType.HANDSHAKE, BitToUserHandshake.class, BitToUserHandshake.PARSER) {
+
+ @Override
+ protected void validateHandshake(BitToUserHandshake inbound) throws Exception {
+ logger.debug("Handling handshake from bit to user. {}", inbound);
+ if (inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION)
+ throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.",
+ inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+ }
+
+ };
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
new file mode 100644
index 0000000..3ce14f0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserResultsListener.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * 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.drill.exec.rpc.user;
+
+import java.util.concurrent.Future;
+
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.rpc.RpcException;
+
+import com.google.common.util.concurrent.SettableFuture;
+
+public abstract class UserResultsListener {
+ SettableFuture<Void> future = SettableFuture.create();
+
+ final void set(){
+ future.set(null);
+ }
+
+ Future<Void> getFuture(){
+ return future;
+ }
+
+ public abstract void submissionFailed(RpcException ex);
+ public abstract void resultArrived(QueryResultBatch result);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
new file mode 100644
index 0000000..893e432
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserRpcConfig.java
@@ -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.drill.exec.rpc.user;
+
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.RpcConfig;
+
+public class UserRpcConfig {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserRpcConfig.class);
+
+ public static RpcConfig MAPPING = RpcConfig.newBuilder("USER-RPC-MAPPING") //
+ .add(RpcType.HANDSHAKE, UserToBitHandshake.class, RpcType.HANDSHAKE, BitToUserHandshake.class) //user to bit.
+ .add(RpcType.RUN_QUERY, RunQuery.class, RpcType.QUERY_HANDLE, QueryId.class) //user to bit
+ .add(RpcType.QUERY_RESULT, QueryResult.class, RpcType.ACK, Ack.class) //bit to user
+ .build();
+
+ public static int RPC_VERSION = 1;
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index cccaa55..406afc4 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -19,31 +19,36 @@ package org.apache.drill.exec.rpc.user;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.channel.Channel;
import io.netty.channel.EventLoopGroup;
-import io.netty.channel.socket.SocketChannel;
+import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
import org.apache.drill.exec.proto.UserProtos.RpcType;
import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
import org.apache.drill.exec.rpc.BasicServer;
import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.rpc.RemoteConnection;
import org.apache.drill.exec.rpc.Response;
import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.user.UserWorker;
+import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.MessageLite;
-public class UserServer extends BasicServer<RpcType> {
+public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnection> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserServer.class);
-
- final DrillbitContext context;
-
- public UserServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, DrillbitContext context) {
- super(alloc, eventLoopGroup);
- this.context = context;
+
+ final UserWorker worker;
+
+ public UserServer(ByteBufAllocator alloc, EventLoopGroup eventLoopGroup, UserWorker worker) {
+ super(UserRpcConfig.MAPPING, alloc, eventLoopGroup);
+ this.worker = worker;
}
@Override
@@ -55,36 +60,70 @@ public class UserServer extends BasicServer<RpcType> {
default:
throw new UnsupportedOperationException();
}
-
}
- public DrillRpcFuture<QueryResult> sendResult(RunQuery query, ByteBuf data) throws RpcException {
- return this.send(RpcType.QUERY_RESULT, query, QueryResult.class, data);
- }
-
-
@Override
- protected Response handle(SocketChannel channel, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ protected Response handle(UserClientConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
+ throws RpcException {
switch (rpcType) {
case RpcType.HANDSHAKE_VALUE:
-// logger.debug("Received handshake, responding in kind.");
- return new Response(RpcType.HANDSHAKE, BitToUserHandshake.getDefaultInstance(), null);
-
+ // logger.debug("Received handshake, responding in kind.");
+ return new Response(RpcType.HANDSHAKE, BitToUserHandshake.getDefaultInstance());
+
case RpcType.RUN_QUERY_VALUE:
-// logger.debug("Received query to run. Returning query handle.");
- return new Response(RpcType.QUERY_HANDLE, QueryHandle.newBuilder().setQueryId(1).build(), null);
-
+ // logger.debug("Received query to run. Returning query handle.");
+ try {
+ RunQuery query = RunQuery.PARSER.parseFrom(new ByteBufInputStream(pBody));
+ return new Response(RpcType.QUERY_HANDLE, worker.submitWork(connection, query));
+ } catch (InvalidProtocolBufferException e) {
+ throw new RpcException("Failure while decoding RunQuery body.", e);
+ }
+
case RpcType.REQUEST_RESULTS_VALUE:
-// logger.debug("Received results requests. Returning empty query result.");
- return new Response(RpcType.QUERY_RESULT, QueryResult.getDefaultInstance(), null);
-
+ // logger.debug("Received results requests. Returning empty query result.");
+ try {
+ RequestResults req = RequestResults.PARSER.parseFrom(new ByteBufInputStream(pBody));
+ return new Response(RpcType.QUERY_RESULT, worker.getResult(connection, req));
+ } catch (InvalidProtocolBufferException e) {
+ throw new RpcException("Failure while decoding RequestResults body.", e);
+ }
+
default:
throw new UnsupportedOperationException();
}
}
+
+ public class UserClientConnection extends RemoteConnection {
+ public UserClientConnection(Channel channel) {
+ super(channel);
+ }
+
+ public DrillRpcFuture<Ack> sendResult(QueryWritableBatch result){
+ return send(this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
+ }
+
+ }
+
+ @Override
+ public UserClientConnection initRemoteConnection(Channel channel) {
+ return new UserClientConnection(channel);
+ }
-
+ @Override
+ protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler() {
+ return new ServerHandshakeHandler<UserToBitHandshake>(RpcType.HANDSHAKE, UserToBitHandshake.PARSER){
+
+ @Override
+ public MessageLite getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
+ logger.debug("Handling handshake from user to bit. {}", inbound);
+ if(inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version. Expected %d, actual %d.", inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+ return BitToUserHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build();
+ }
+ };
+ }
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
new file mode 100644
index 0000000..3c4d9af
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -0,0 +1,68 @@
+/*******************************************************************************
+ * 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.drill.exec.server;
+
+import io.netty.channel.nio.NioEventLoopGroup;
+
+import java.io.Closeable;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.rpc.NamedThreadFactory;
+
+import com.yammer.metrics.MetricRegistry;
+
+public class BootStrapContext implements Closeable{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class);
+
+ private final DrillConfig config;
+ private final NioEventLoopGroup loop;
+ private final MetricRegistry metrics;
+ private final BufferAllocator allocator;
+
+ public BootStrapContext(DrillConfig config) {
+ super();
+ this.config = config;
+ this.loop = new NioEventLoopGroup(4, new NamedThreadFactory("BitServer-"));
+ this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
+ this.allocator = BufferAllocator.getAllocator(config);
+ }
+
+ public DrillConfig getConfig() {
+ return config;
+ }
+
+ public NioEventLoopGroup getBitLoopGroup() {
+ return loop;
+ }
+
+ public MetricRegistry getMetrics() {
+ return metrics;
+ }
+
+ public BufferAllocator getAllocator() {
+ return allocator;
+ }
+
+ public void close(){
+ loop.shutdown();
+ allocator.close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
index c33afce..7d745e1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/Drillbit.java
@@ -17,10 +17,9 @@
******************************************************************************/
package org.apache.drill.exec.server;
-import java.net.InetAddress;
+import java.io.Closeable;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.cache.DistributedCache;
import org.apache.drill.exec.cache.HazelCache;
@@ -28,15 +27,16 @@ import org.apache.drill.exec.coord.ClusterCoordinator;
import org.apache.drill.exec.coord.ClusterCoordinator.RegistrationHandle;
import org.apache.drill.exec.coord.ZKClusterCoordinator;
import org.apache.drill.exec.exception.DrillbitStartupException;
-import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.service.ServiceEngine;
+import org.apache.drill.exec.work.WorkManager;
import com.google.common.io.Closeables;
/**
* Starts, tracks and stops all the required services for a Drillbit daemon to work.
*/
-public class Drillbit {
+public class Drillbit implements Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Drillbit.class);
public static Drillbit start(StartupOptions options) throws DrillbitStartupException {
@@ -47,7 +47,7 @@ public class Drillbit {
Drillbit bit;
try {
logger.debug("Setting up Drillbit.");
- bit = new Drillbit(config);
+ bit = new Drillbit(config, null);
} catch (Exception ex) {
throw new DrillbitStartupException("Failure while initializing values in Drillbit.", ex);
}
@@ -65,35 +65,37 @@ public class Drillbit {
start(options);
}
- private final DrillbitContext context;
- final BufferAllocator pool;
final ClusterCoordinator coord;
final ServiceEngine engine;
final DistributedCache cache;
- final DrillConfig config;
- private RegistrationHandle handle;
-
- public Drillbit(DrillConfig config) throws Exception {
- final DrillbitContext context = new DrillbitContext(config, this);
- Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
- this.context = context;
- this.pool = BufferAllocator.getAllocator(context);
- this.coord = new ZKClusterCoordinator(config);
- this.engine = new ServiceEngine(context);
- this.cache = new HazelCache(context.getConfig());
- this.config = config;
+ final WorkManager manager;
+ final BootStrapContext context;
+
+ private volatile RegistrationHandle handle;
+
+ public Drillbit(DrillConfig config, RemoteServiceSet serviceSet) throws Exception {
+ if(serviceSet != null){
+ this.context = new BootStrapContext(config);
+ this.manager = new WorkManager(context);
+ this.coord = serviceSet.getCoordinator();
+ this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+ this.cache = serviceSet.getCache();
+ }else{
+ Runtime.getRuntime().addShutdownHook(new ShutdownThread(config));
+ this.context = new BootStrapContext(config);
+ this.manager = new WorkManager(context);
+ this.coord = new ZKClusterCoordinator(config);
+ this.engine = new ServiceEngine(manager.getBitComWorker(), manager.getUserWorker(), context);
+ this.cache = new HazelCache(config);
+ }
}
public void run() throws Exception {
- coord.start();
- engine.start();
- DrillbitEndpoint md = DrillbitEndpoint.newBuilder()
- .setAddress(InetAddress.getLocalHost().getHostAddress())
- .setBitPort(engine.getBitPort())
- .setUserPort(engine.getUserPort())
- .build();
+ coord.start(10000);
+ DrillbitEndpoint md = engine.start();
+ cache.run();
+ manager.start(md, cache, engine.getBitCom(), coord);
handle = coord.register(md);
- cache.run(md);
}
public void close() {
@@ -107,7 +109,8 @@ public class Drillbit {
Closeables.closeQuietly(engine);
Closeables.closeQuietly(coord);
- Closeables.closeQuietly(pool);
+ Closeables.closeQuietly(manager);
+ Closeables.closeQuietly(context);
logger.info("Shutdown completed.");
}
@@ -123,5 +126,11 @@ public class Drillbit {
}
}
+ public ClusterCoordinator getCoordinator(){
+ return coord;
+ }
+ public DrillbitContext getContext(){
+ return this.manager.getContext();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index b08b070..d5aaab2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -23,42 +23,60 @@ import java.util.Collection;
import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.common.logical.StorageEngineConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.rpc.NamedThreadFactory;
import org.apache.drill.exec.rpc.bit.BitCom;
import org.apache.drill.exec.store.StorageEngine;
+import com.google.common.base.Preconditions;
import com.yammer.metrics.MetricRegistry;
public class DrillbitContext {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
+
+ private BootStrapContext context;
+
+ private PhysicalPlanReader reader;
+ private final ClusterCoordinator coord;
+ private final BitCom com;
+ private final DistributedCache cache;
+ private final DrillbitEndpoint endpoint;
- private final DrillConfig config;
- private final Drillbit underlyingBit;
- private final NioEventLoopGroup loop;
- private final MetricRegistry metrics;
-
- public DrillbitContext(DrillConfig config, Drillbit underlyingBit) {
+ public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, BitCom com, DistributedCache cache) {
super();
- this.config = config;
- this.underlyingBit = underlyingBit;
- this.loop = new NioEventLoopGroup(1, new NamedThreadFactory("BitServer-"));
- this.metrics = new MetricRegistry(config.getString(ExecConstants.METRICS_CONTEXT_NAME));
+ Preconditions.checkNotNull(endpoint);
+ Preconditions.checkNotNull(context);
+ Preconditions.checkNotNull(context);
+ Preconditions.checkNotNull(com);
+ Preconditions.checkNotNull(com);
+
+ this.context = context;
+ this.coord = coord;
+ this.com = com;
+ this.cache = cache;
+ this.endpoint = endpoint;
+ this.reader = new PhysicalPlanReader(context.getConfig(), context.getConfig().getMapper(), endpoint);
+ }
+
+ public DrillbitEndpoint getEndpoint(){
+ return endpoint;
}
public DrillConfig getConfig() {
- return config;
+ return context.getConfig();
}
public Collection<DrillbitEndpoint> getBits(){
- return underlyingBit.coord.getAvailableEndpoints();
+ return coord.getAvailableEndpoints();
}
public BufferAllocator getAllocator(){
- return underlyingBit.pool;
+ return context.getAllocator();
}
public StorageEngine getStorageEngine(StorageEngineConfig config){
@@ -66,19 +84,23 @@ public class DrillbitContext {
}
public NioEventLoopGroup getBitLoopGroup(){
- return loop;
+ return context.getBitLoopGroup();
}
public BitCom getBitCom(){
- return underlyingBit.engine.getBitCom();
+ return com;
}
public MetricRegistry getMetrics(){
- return metrics;
+ return context.getMetrics();
}
public DistributedCache getCache(){
- return underlyingBit.cache;
+ return cache;
+ }
+
+ public PhysicalPlanReader getPlanReader(){
+ return reader;
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
new file mode 100644
index 0000000..0337a68
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/server/RemoteServiceSet.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * 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.drill.exec.server;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.cache.LocalCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.coord.LocalClusterCoordinator;
+
+public class RemoteServiceSet implements Closeable{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemoteServiceSet.class);
+
+ private final DistributedCache cache;
+ private final ClusterCoordinator coordinator;
+
+ public RemoteServiceSet(DistributedCache cache, ClusterCoordinator coordinator) {
+ super();
+ this.cache = cache;
+ this.coordinator = coordinator;
+ }
+
+ public DistributedCache getCache() {
+ return cache;
+ }
+
+ public ClusterCoordinator getCoordinator() {
+ return coordinator;
+ }
+
+
+ @Override
+ public void close() throws IOException {
+ cache.close();
+ coordinator.close();
+ }
+
+ public static RemoteServiceSet getLocalServiceSet(){
+ return new RemoteServiceSet(new LocalCache(), new LocalClusterCoordinator());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
index 5d83bdb..d6d3b9c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/service/ServiceEngine.java
@@ -17,49 +17,48 @@
******************************************************************************/
package org.apache.drill.exec.service;
-import io.netty.buffer.ByteBufAllocator;
import io.netty.channel.nio.NioEventLoopGroup;
import java.io.Closeable;
import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import org.apache.drill.common.config.DrillConfig;
import org.apache.drill.exec.ExecConstants;
import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.rpc.NamedThreadFactory;
import org.apache.drill.exec.rpc.bit.BitCom;
import org.apache.drill.exec.rpc.bit.BitComImpl;
import org.apache.drill.exec.rpc.user.UserServer;
-import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.user.UserWorker;
import com.google.common.io.Closeables;
public class ServiceEngine implements Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ServiceEngine.class);
- UserServer userServer;
- BitComImpl bitCom;
- int userPort;
- int bitPort;
- DrillbitContext context;
+ private final UserServer userServer;
+ private final BitCom bitCom;
+ private final DrillConfig config;
- public ServiceEngine(DrillbitContext context){
- this.context = context;
- ByteBufAllocator allocator = context.getAllocator().getUnderlyingAllocator();
- this.userServer = new UserServer(allocator, new NioEventLoopGroup(1, new NamedThreadFactory("UserServer-")), context);
- this.bitCom = new BitComImpl(context);
+ public ServiceEngine(BitComHandler bitComWorker, UserWorker userWorker, BootStrapContext context){
+ this.userServer = new UserServer(context.getAllocator().getUnderlyingAllocator(), new NioEventLoopGroup(1, new NamedThreadFactory("UserServer-")), userWorker);
+ this.bitCom = new BitComImpl(context, bitComWorker);
+ this.config = context.getConfig();
}
- public void start() throws DrillbitStartupException, InterruptedException{
- userPort = userServer.bind(context.getConfig().getInt(ExecConstants.INITIAL_USER_PORT));
- bitPort = bitCom.start();
- }
-
- public int getBitPort(){
- return bitPort;
- }
-
- public int getUserPort(){
- return userPort;
+ public DrillbitEndpoint start() throws DrillbitStartupException, InterruptedException, UnknownHostException{
+ int userPort = userServer.bind(config.getInt(ExecConstants.INITIAL_USER_PORT));
+ int bitPort = bitCom.start();
+ return DrillbitEndpoint.newBuilder()
+ .setAddress(InetAddress.getLocalHost().getHostAddress())
+ .setBitPort(bitPort)
+ .setUserPort(userPort)
+ .build();
}
public BitCom getBitCom(){
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
index d89b431..80704fa 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStorageEngine.java
@@ -23,8 +23,8 @@ import java.util.Collections;
import java.util.List;
import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Multimap;
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
index 9fc4165..67c84ed 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/RecordReader.java
@@ -17,9 +17,8 @@
******************************************************************************/
package org.apache.drill.exec.store;
-import org.apache.drill.exec.exception.ExecutionSetupException;
-import org.apache.drill.exec.ops.OutputMutator;
-import org.apache.drill.exec.record.BatchSchema;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
public interface RecordReader {
@@ -35,7 +34,7 @@ public interface RecordReader {
* mutating the set of schema values for that particular record.
* @throws ExecutionSetupException
*/
- public abstract void setup(BatchSchema expectedSchema, OutputMutator output) throws ExecutionSetupException;
+ public abstract void setup(OutputMutator output) throws ExecutionSetupException;
/**
* Increment record reader forward, writing into the provided output batch.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
index 67ea5b6..4884b7a 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/store/StorageEngine.java
@@ -22,8 +22,8 @@ import java.util.Collection;
import java.util.List;
import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Multimap;
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
new file mode 100644
index 0000000..d2e8b8f
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/util/AtomicState.java
@@ -0,0 +1,58 @@
+/*******************************************************************************
+ * 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.drill.exec.util;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+
+import com.google.protobuf.Internal.EnumLite;
+
+/**
+ * Simple wrapper class around AtomicInteger which allows management of a State value extending EnumLite.
+ * @param <T> The type of EnumLite to use for state.
+ */
+public abstract class AtomicState<T extends EnumLite> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AtomicState.class);
+
+ private final AtomicInteger state = new AtomicInteger();
+
+ /**
+ * Constructor that defines initial T state.
+ * @param initial
+ */
+ public AtomicState(T initial){
+ state.set(initial.getNumber());
+ }
+
+ protected abstract T getStateFromNumber(int i);
+
+ /**
+ * Does an atomic conditional update from one state to another.
+ * @param oldState The expected current state.
+ * @param newState The desired new state.
+ * @return Whether or not the update was successful.
+ */
+ public boolean updateState(T oldState, T newState){
+ return state.compareAndSet(oldState.getNumber(), newState.getNumber());
+ }
+
+ public T getState(){
+ return getStateFromNumber(state.get());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
new file mode 100644
index 0000000..0e8edd5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/AbstractFragmentRunnerListener.java
@@ -0,0 +1,109 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
+
+public class AbstractFragmentRunnerListener implements FragmentRunnerListener{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFragmentRunnerListener.class);
+
+ private FragmentContext context;
+ private volatile long startNanos;
+
+ public AbstractFragmentRunnerListener(FragmentContext context) {
+ super();
+ this.context = context;
+ }
+
+ private FragmentStatus.Builder getBuilder(FragmentState state){
+ FragmentStatus.Builder status = FragmentStatus.newBuilder();
+ context.addMetricsToStatus(status);
+ status.setState(state);
+ status.setRunningTime(System.nanoTime() - startNanos);
+ status.setHandle(context.getHandle());
+ status.setMemoryUse(context.getAllocator().getAllocatedMemory());
+ return status;
+ }
+
+ @Override
+ public void stateChanged(FragmentHandle handle, FragmentState newState) {
+ FragmentStatus.Builder status = getBuilder(newState);
+
+ switch(newState){
+ case AWAITING_ALLOCATION:
+ awaitingAllocation(handle, status);
+ break;
+ case CANCELLED:
+ cancelled(handle, status);
+ break;
+ case FAILED:
+ // no op since fail should have also been called.
+ break;
+ case FINISHED:
+ finished(handle, status);
+ break;
+ case RUNNING:
+ this.startNanos = System.nanoTime();
+ running(handle, status);
+ break;
+ case SENDING:
+ // no op.
+ break;
+ default:
+ break;
+
+ }
+ }
+
+ protected void awaitingAllocation(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+ statusChange(handle, statusBuilder.build());
+ }
+
+ protected void running(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+ statusChange(handle, statusBuilder.build());
+ }
+
+ protected void cancelled(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+ statusChange(handle, statusBuilder.build());
+ }
+
+ protected void finished(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+ statusChange(handle, statusBuilder.build());
+ }
+
+ protected void statusChange(FragmentHandle handle, FragmentStatus status){
+
+ }
+
+ @Override
+ public final void fail(FragmentHandle handle, String message, Throwable excep) {
+ FragmentStatus.Builder status = getBuilder(FragmentState.FAILED);
+ status.setError(ErrorHelper.logAndConvertError(context.getIdentity(), message, excep, logger));
+ fail(handle, status);
+ }
+
+ protected void fail(FragmentHandle handle, FragmentStatus.Builder statusBuilder){
+ statusChange(handle, statusBuilder.build());
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
new file mode 100644
index 0000000..3c7ef04
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/CancelableQuery.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+public interface CancelableQuery {
+ public void cancel();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
new file mode 100644
index 0000000..f6a9786
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/EndpointListener.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+
+public abstract class EndpointListener<RET, V> extends RpcOutcomeListener<RET>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(EndpointListener.class);
+
+ protected final DrillbitEndpoint endpoint;
+ protected final V value;
+
+ public EndpointListener(DrillbitEndpoint endpoint, V value) {
+ super();
+ this.endpoint = endpoint;
+ this.value = value;
+ }
+
+ protected DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+ protected V getValue() {
+ return value;
+ }
+
+
+}
[12/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
deleted file mode 100644
index 58edf03..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/AbstractStore.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-
-public abstract class AbstractStore extends AbstractSingle implements Store{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractStore.class);
-
- public AbstractStore(PhysicalOperator child) {
- super(child);
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E{
- return physicalVisitor.visitStore(this, value);
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
deleted file mode 100644
index d779eb8..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Exchange.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public interface Exchange extends PhysicalOperator{
-
- @JsonIgnore
- public abstract OperatorCost getAggregateSendCost();
-
- @JsonIgnore
- public abstract OperatorCost getAggregateReceiveCost();
-
- @JsonProperty("cost")
- public abstract ExchangeCost getExchangeCost();
-
- /**
- * Inform this Exchange node about its sender locations.
- * @param senderLocations
- */
- public abstract void setupSenders(List<DrillbitEndpoint> senderLocations);
-
- /**
- * Inform this Exchange node about its receiver locations.
- * @param receiverLocations
- */
- public abstract void setupReceivers(List<DrillbitEndpoint> receiverLocations);
-
- /**
- * Get the Sender associated with the given minorFragmentId.
- * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
- * @param child The feeding node for the requested sender.
- * @return The materialized sender for the given arguments.
- */
- public abstract Sender getSender(int minorFragmentId, PhysicalOperator child);
-
- /**
- * Get the Receiver associated with the given minorFragmentId.
- * @param minorFragmentId The minor fragment id, must be in the range [0, fragment.width).
- * @return The materialized recevier for the given arguments.
- */
- public abstract Receiver getReceiver(int minorFragmentId);
-
- public abstract int getMaxSendWidth();
-
- public PhysicalOperator getChild();
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
deleted file mode 100644
index f17203e..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/ExchangeCost.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.OperatorCost;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class ExchangeCost {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExchangeCost.class);
-
- private final OperatorCost send;
- private final OperatorCost receive;
- private final OperatorCost combined;
-
- @JsonCreator
- public ExchangeCost(@JsonProperty("send") OperatorCost send, @JsonProperty("receive") OperatorCost receive) {
- this.send = send;
- this.receive = receive;
- this.combined = OperatorCost.combine(send, receive);
- }
-
- @JsonIgnore
- public OperatorCost getCombinedCost(){
- return combined;
- }
-
- public OperatorCost getSend() {
- return send;
- }
-
- public OperatorCost getReceive() {
- return receive;
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
deleted file mode 100644
index 4557df4..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentLeaf.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-
-/**
- * A POP which relies on no other nodes within the current fragment.
- */
-public interface FragmentLeaf extends PhysicalOperator{
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
deleted file mode 100644
index 8d87d56..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/FragmentRoot.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-
-/**
- * Describes the root operation within a particular Fragment. This includes things like Sinks, and Sender nodes.
- */
-public interface FragmentRoot extends PhysicalOperator{
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
deleted file mode 100644
index feb32ec..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/HasAffinity.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.EndpointAffinity;
-
-public interface HasAffinity extends PhysicalOperator{
- public List<EndpointAffinity> getOperatorAffinity();
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
deleted file mode 100644
index 28efb94..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Leaf.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-public interface Leaf extends PhysicalOperator{
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
deleted file mode 100644
index d8d1b64..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperator.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.graph.GraphValue;
-import org.apache.drill.common.physical.OperatorCost;
-
-import com.fasterxml.jackson.annotation.JsonIdentityInfo;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonInclude;
-import com.fasterxml.jackson.annotation.JsonInclude.Include;
-import com.fasterxml.jackson.annotation.JsonPropertyOrder;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import com.fasterxml.jackson.annotation.ObjectIdGenerators;
-
-@JsonInclude(Include.NON_NULL)
-@JsonPropertyOrder({ "@id" })
-@JsonIdentityInfo(generator = ObjectIdGenerators.IntSequenceGenerator.class, property = "@id")
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "pop")
-public interface PhysicalOperator extends GraphValue<PhysicalOperator> {
-
- public OperatorCost getCost();
-
- /**
- * Describes whether or not a particular physical operator can actually be executed. Most physical operators can be
- * executed. However, Exchange nodes cannot be executed. In order to be executed, they must be converted into their
- * Exec sub components.
- *
- * @return
- */
- @JsonIgnore
- public boolean isExecutable();
-
- /**
- * Provides capability to build a set of output based on traversing a query graph tree.
- * @param physicalVisitor
- * @return
- */
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E;
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
deleted file mode 100644
index fb1fdcd..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalOperatorUtil.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.config.CommonConstants;
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.PathScanner;
-
-public class PhysicalOperatorUtil {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalOperatorUtil.class);
-
- private PhysicalOperatorUtil(){}
-
- public synchronized static Class<?>[] getSubTypes(DrillConfig config){
- Class<?>[] ops = PathScanner.scanForImplementationsArr(PhysicalOperator.class, config.getStringList(CommonConstants.PHYSICAL_OPERATOR_SCAN_PACKAGES));
- logger.debug("Adding Physical Operator sub types: {}", ((Object) ops) );
- return ops;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
deleted file mode 100644
index 2ecc6ce..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/PhysicalVisitor.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import org.apache.drill.common.physical.pop.Filter;
-import org.apache.drill.common.physical.pop.PartitionToRandomExchange;
-import org.apache.drill.common.physical.pop.Project;
-import org.apache.drill.common.physical.pop.Sort;
-
-public interface PhysicalVisitor<RETURN, EXTRA, EXCEP extends Throwable> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalVisitor.class);
-
-
- public RETURN visitExchange(Exchange exchange, EXTRA value) throws EXCEP;
- public RETURN visitScan(Scan<?> scan, EXTRA value) throws EXCEP;
- public RETURN visitStore(Store store, EXTRA value) throws EXCEP;
-
- public RETURN visitFilter(Filter filter, EXTRA value) throws EXCEP;
- public RETURN visitProject(Project project, EXTRA value) throws EXCEP;
- public RETURN visitSort(Sort sort, EXTRA value) throws EXCEP;
- public RETURN visitSender(Sender sender, EXTRA value) throws EXCEP;
- public RETURN visitReceiver(Receiver receiver, EXTRA value) throws EXCEP;
-
- public RETURN visitUnknown(PhysicalOperator op, EXTRA value) throws EXCEP;
-
- public RETURN visitPartitionToRandomExchange(PartitionToRandomExchange partitionToRandom, EXTRA value) throws EXCEP;
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
deleted file mode 100644
index db8f71f..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Receiver.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public interface Receiver extends FragmentLeaf {
- public abstract List<DrillbitEndpoint> getProvidingEndpoints();
-
- /**
- * Whether or not this receive supports out of order exchange. This provides a hint for the scheduling node on whether
- * the receiver can start work if only a subset of all sending endpoints are currently providing data. A random
- * receiver would supports this form of operation. A NWAY receiver would not.
- *
- * @return True if this receiver supports working on a streaming/out of order input.
- */
- public abstract boolean supportsOutOfOrderExchange();
-
-
- public int getSenderCount();
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
deleted file mode 100644
index c4f9982..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Root.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-/**
- * Marker interface describe the root of a query plan.
- */
-public interface Root extends PhysicalOperator{
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
deleted file mode 100644
index c7b45a8..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Scan.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.ReadEntry;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public interface Scan<R extends ReadEntry> extends Leaf, HasAffinity{
-
- @JsonProperty("entries")
- public abstract List<R> getReadEntries();
-
- public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
-
- public abstract Scan<?> getSpecificScan(int minorFragmentId);
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
deleted file mode 100644
index 1859657..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Sender.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-
-
-
-public interface Sender extends FragmentRoot{
- public abstract List<DrillbitEndpoint> getDestinations();
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java b/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
deleted file mode 100644
index eec4a6c..0000000
--- a/sandbox/prototype/common/src/main/java/org/apache/drill/common/physical/pop/base/Store.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical.pop.base;
-
-import java.util.List;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public interface Store extends Root, HasAffinity{
-
- public abstract void applyAssignments(List<DrillbitEndpoint> endpoints);
- public abstract Store getSpecificStore(PhysicalOperator child, int minorFragmentId);
- public abstract int getMaxWidth();
- public abstract PhysicalOperator getChild();
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/main/protobuf/Coordination.proto
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/main/protobuf/Coordination.proto b/sandbox/prototype/common/src/main/protobuf/Coordination.proto
deleted file mode 100644
index f98d2c5..0000000
--- a/sandbox/prototype/common/src/main/protobuf/Coordination.proto
+++ /dev/null
@@ -1,26 +0,0 @@
-package exec;
-
-option java_package = "org.apache.drill.common.proto";
-option java_outer_classname = "CoordinationProtos";
-option optimize_for = LITE_RUNTIME;
-
-message DrillbitEndpoint{
- optional string address = 1;
- optional int32 user_port = 2;
- optional int32 bit_port = 3;
- optional Roles roles = 4;
-}
-
-message DrillServiceInstance{
- optional string id = 1;
- optional int64 registrationTimeUTC = 2;
- optional DrillbitEndpoint endpoint = 3;
-}
-
-message Roles{
- optional bool sql_query = 1 [default = true];
- optional bool logical_plan = 2 [default = true];
- optional bool physical_plan = 3 [default = true];
- optional bool java_executor = 4 [default = true];
- optional bool distributed_cache = 5 [default = true];
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
deleted file mode 100644
index 1b042c5..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockScanPOP.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractScan;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-scan")
-public class MockScanPOP extends AbstractScan<MockScanPOP.MockScanEntry>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockScanPOP.class);
-
- private final String url;
-
- @JsonCreator
- public MockScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockScanEntry> readEntries) {
- super(readEntries);
- this.url = url;
- }
-
- public String getUrl() {
- return url;
- }
-
- public static class MockScanEntry implements ReadEntry{
- public int id;
- }
-
- @Override
- public List<EndpointAffinity> getOperatorAffinity() {
- return Collections.emptyList();
- }
-
- @Override
- public void applyAssignments(List<DrillbitEndpoint> endpoints) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Scan<?> getSpecificScan(int minorFragmentId) {
- return this;
- }
-
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
deleted file mode 100644
index f48c539..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/MockStorePOP.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("mock-store")
-public class MockStorePOP extends AbstractStore {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorePOP.class);
-
- @JsonCreator
- public MockStorePOP(@JsonProperty("child") PhysicalOperator child) {
- super(child);
- }
-
- public int getMaxWidth() {
- return 1;
- }
-
- @Override
- public List<EndpointAffinity> getOperatorAffinity() {
- return Collections.emptyList();
- }
-
- @Override
- public void applyAssignments(List<DrillbitEndpoint> endpoints) {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public Store getSpecificStore(PhysicalOperator child, int minorFragmentId) {
- return this;
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java b/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
deleted file mode 100644
index 0ad1f76..0000000
--- a/sandbox/prototype/common/src/test/java/org/apache/drill/common/physical/ParsePhysicalPlan.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*******************************************************************************
- * 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.drill.common.physical;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.util.FileUtils;
-import org.junit.Test;
-
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.fasterxml.jackson.databind.ObjectWriter;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
-
-public class ParsePhysicalPlan {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParsePhysicalPlan.class);
-
-
- @Test
- public void parseSimplePlan() throws Exception{
- DrillConfig c = DrillConfig.create();
- ObjectReader r = c.getMapper().reader(PhysicalPlan.class);
- ObjectWriter writer = c.getMapper().writer();
- PhysicalPlan plan = PhysicalPlan.parse(r, Files.toString(FileUtils.getResourceAsFile("/physical_test1.json"), Charsets.UTF_8));
- System.out.println(plan.unparse(writer));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/drill-module.conf b/sandbox/prototype/common/src/test/resources/drill-module.conf
index 86e828a..0e2c84e 100644
--- a/sandbox/prototype/common/src/test/resources/drill-module.conf
+++ b/sandbox/prototype/common/src/test/resources/drill-module.conf
@@ -1,2 +1 @@
drill.logical.storage.packages += "org.apache.drill.storage"
-drill.physical.operator.packages += "org.apache.drill.common.physical.pop"
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/common/src/test/resources/physical_test1.json
----------------------------------------------------------------------
diff --git a/sandbox/prototype/common/src/test/resources/physical_test1.json b/sandbox/prototype/common/src/test/resources/physical_test1.json
deleted file mode 100644
index 16bc87a..0000000
--- a/sandbox/prototype/common/src/test/resources/physical_test1.json
+++ /dev/null
@@ -1,33 +0,0 @@
-{
- head:{
- type:"APACHE_DRILL_PHYSICAL",
- version:"1",
- generator:{
- type:"manual"
- }
- },
- graph:[
- {
- @id:1,
- pop:"mock-scan",
- url: "http://apache.org",
- entries:[
- {id:1}
- ],
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
- },
- {
- @id:2,
- child: 1,
- pop:"filter",
- expr: "b > 5",
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
- },
- {
- @id: 3,
- child: 2,
- pop: "mock-store",
- cost: { disk: 1.0, memory: 1.0, cpu: 1.0, network: 1.0}
- }
- ]
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/pom.xml b/sandbox/prototype/exec/java-exec/pom.xml
index 9766df7..f5ece33 100644
--- a/sandbox/prototype/exec/java-exec/pom.xml
+++ b/sandbox/prototype/exec/java-exec/pom.xml
@@ -56,7 +56,7 @@
<groupId>org.apache.drill</groupId>
<artifactId>common</artifactId>
<version>1.0-SNAPSHOT</version>
- <classifier>test</classifier>
+ <classifier>tests</classifier>
</dependency>
<dependency>
<groupId>com.beust</groupId>
@@ -110,7 +110,7 @@
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
- <version>4.0.0.CR1</version>
+ <version>4.0.0.CR2</version>
</dependency>
<dependency>
<groupId>com.google.protobuf</groupId>
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
deleted file mode 100644
index 82a8a85..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/ByteReorder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec;
-
-import java.util.Arrays;
-
-import com.google.common.base.Charsets;
-import com.google.common.primitives.Bytes;
-import com.google.common.primitives.UnsignedBytes;
-
-public class ByteReorder {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ByteReorder.class);
-
- public static void main(String[] args){
- String[] strings = {"hello", "goodbye", "my friend"};
- byte[][] bytes = new byte[strings.length][];
- for(int i =0; i < strings.length; i++){
- bytes[i] = strings[i].getBytes(Charsets.UTF_8);
- }
-
- for(int i =0; i < bytes.length; i++){
- for(int v = 0; v < bytes[i].length; v++){
- bytes[i][v] = (byte) ~bytes[i][v];
- }
- }
-
- Arrays.sort(bytes, UnsignedBytes.lexicographicalComparator());
-
- for(int i =0; i < bytes.length; i++){
- for(int v = 0; v < bytes[i].length; v++){
- bytes[i][v] = (byte) ~bytes[i][v];
- }
- }
-
- for(int i =0; i < bytes.length; i++){
- System.out.println(new String(bytes[i]));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
index 2928dbe..ba2c26b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/DistributedCache.java
@@ -20,8 +20,9 @@ package org.apache.drill.exec.cache;
import java.io.Closeable;
import java.util.List;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
@@ -29,14 +30,11 @@ import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
public interface DistributedCache extends Closeable{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DistributedCache.class);
- public void run(DrillbitEndpoint endpoint) throws DrillbitStartupException;
+ public void run() throws DrillbitStartupException;
- public void saveOptimizedPlan(TemplatizedLogicalPlan logical, TemplatizedPhysicalPlan physical);
- public TemplatizedPhysicalPlan getOptimizedPlan(TemplatizedLogicalPlan logical);
+// public void updateLocalQueueLength(int length);
+// public List<WorkQueueStatus> getQueueLengths();
- public void updateLocalQueueLength(int length);
- public List<WorkQueueStatus> getQueueLengths();
-
- public PlanFragment getFragment(long fragmentId);
+ public PlanFragment getFragment(FragmentHandle handle);
public void storeFragment(PlanFragment fragment);
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
index 943031d..f4fdbfa 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/HazelCache.java
@@ -17,20 +17,22 @@
******************************************************************************/
package org.apache.drill.exec.cache;
-import java.io.DataInput;
-import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.cache.ProtoBufImpl.HWorkQueueStatus;
+import org.apache.drill.exec.cache.ProtoBufImpl.HandlePlan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
import com.beust.jcommander.internal.Lists;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
+import com.google.protobuf.InvalidProtocolBufferException;
import com.hazelcast.config.Config;
import com.hazelcast.core.Hazelcast;
import com.hazelcast.core.HazelcastInstance;
@@ -38,39 +40,36 @@ import com.hazelcast.core.IMap;
import com.hazelcast.core.ITopic;
import com.hazelcast.core.Message;
import com.hazelcast.core.MessageListener;
-import com.hazelcast.nio.DataSerializable;
public class HazelCache implements DistributedCache {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HazelCache.class);
private final String instanceName;
private HazelcastInstance instance;
- private ITopic<WrappedWorkQueueStatus> workQueueLengths;
- private DrillbitEndpoint endpoint;
+ private ITopic<HWorkQueueStatus> workQueueLengths;
+ private HandlePlan fragments;
private Cache<WorkQueueStatus, Integer> endpoints;
- private IMap<TemplatizedLogicalPlan, TemplatizedPhysicalPlan> optimizedPlans;
public HazelCache(DrillConfig config) {
this.instanceName = config.getString(ExecConstants.SERVICE_NAME);
}
- private class Listener implements MessageListener<WrappedWorkQueueStatus>{
+ private class Listener implements MessageListener<HWorkQueueStatus>{
@Override
- public void onMessage(Message<WrappedWorkQueueStatus> wrapped) {
+ public void onMessage(Message<HWorkQueueStatus> wrapped) {
logger.debug("Received new queue length message.");
- endpoints.put(wrapped.getMessageObject().status, 0);
+ endpoints.put(wrapped.getMessageObject().get(), 0);
}
}
- public void run(DrillbitEndpoint endpoint) {
+ public void run() {
Config c = new Config();
c.setInstanceName(instanceName);
instance = getInstanceOrCreateNew(c);
workQueueLengths = instance.getTopic("queue-length");
- optimizedPlans = instance.getMap("plan-optimizations");
- this.endpoint = endpoint;
+ fragments = new HandlePlan(instance);
endpoints = CacheBuilder.newBuilder().maximumSize(2000).build();
workQueueLengths.addMessageListener(new Listener());
}
@@ -83,52 +82,16 @@ public class HazelCache implements DistributedCache {
return Hazelcast.newHazelcastInstance(c);
}
- @Override
- public void saveOptimizedPlan(TemplatizedLogicalPlan logical, TemplatizedPhysicalPlan physical) {
- optimizedPlans.put(logical, physical);
- }
-
- @Override
- public TemplatizedPhysicalPlan getOptimizedPlan(TemplatizedLogicalPlan logical) {
- return optimizedPlans.get(logical);
- }
-
- @Override
- public void updateLocalQueueLength(int length) {
- workQueueLengths.publish(new WrappedWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
- .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
- }
-
- @Override
- public List<WorkQueueStatus> getQueueLengths() {
- return Lists.newArrayList(endpoints.asMap().keySet());
- }
-
- public class WrappedWorkQueueStatus implements DataSerializable {
-
- public WorkQueueStatus status;
-
- public WrappedWorkQueueStatus(WorkQueueStatus status) {
- this.status = status;
- }
-
- @Override
- public void readData(DataInput arg0) throws IOException {
- int len = arg0.readShort();
- byte[] b = new byte[len];
- arg0.readFully(b);
- this.status = WorkQueueStatus.parseFrom(b);
- }
-
- @Override
- public void writeData(DataOutput arg0) throws IOException {
- byte[] b = status.toByteArray();
- if (b.length > Short.MAX_VALUE) throw new IOException("Unexpectedly long value.");
- arg0.writeShort(b.length);
- arg0.write(b);
- }
-
- }
+// @Override
+// public void updateLocalQueueLength(int length) {
+// workQueueLengths.publish(new HWorkQueueStatus(WorkQueueStatus.newBuilder().setEndpoint(endpoint)
+// .setQueueLength(length).setReportTime(System.currentTimeMillis()).build()));
+// }
+//
+// @Override
+// public List<WorkQueueStatus> getQueueLengths() {
+// return Lists.newArrayList(endpoints.asMap().keySet());
+// }
@Override
public void close() throws IOException {
@@ -136,13 +99,13 @@ public class HazelCache implements DistributedCache {
}
@Override
- public PlanFragment getFragment(long fragmentId) {
- throw new UnsupportedOperationException();
+ public PlanFragment getFragment(FragmentHandle handle) {
+ return this.fragments.get(handle);
}
@Override
public void storeFragment(PlanFragment fragment) {
- throw new UnsupportedOperationException();
+ fragments.put(fragment.getHandle(), fragment);
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
new file mode 100644
index 0000000..ddb2a02
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/LocalCache.java
@@ -0,0 +1,55 @@
+/*******************************************************************************
+ * 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.drill.exec.cache;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.google.common.collect.Maps;
+
+public class LocalCache implements DistributedCache {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalCache.class);
+
+ private volatile Map<FragmentHandle, PlanFragment> handles;
+
+ @Override
+ public void close() throws IOException {
+ handles = null;
+ }
+
+ @Override
+ public void run() throws DrillbitStartupException {
+ handles = Maps.newConcurrentMap();
+ }
+
+ @Override
+ public PlanFragment getFragment(FragmentHandle handle) {
+ return handles.get(handle);
+ }
+
+ @Override
+ public void storeFragment(PlanFragment fragment) {
+ handles.put(fragment.getHandle(), fragment);
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
new file mode 100644
index 0000000..46bb9ee
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufImpl.java
@@ -0,0 +1,50 @@
+/*******************************************************************************
+ * 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.drill.exec.cache;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
+
+import com.google.protobuf.Parser;
+import com.hazelcast.core.HazelcastInstance;
+
+public class ProtoBufImpl {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufImpl.class);
+
+ public static class HWorkQueueStatus extends ProtoBufWrap<WorkQueueStatus>{
+ public HWorkQueueStatus() {super(WorkQueueStatus.PARSER);}
+ public HWorkQueueStatus(WorkQueueStatus value) {super(value, WorkQueueStatus.PARSER);}
+ }
+
+ public static class HFragmentHandle extends ProtoBufWrap<FragmentHandle>{
+ public HFragmentHandle() {super(FragmentHandle.PARSER);}
+ public HFragmentHandle(FragmentHandle value) {super(value, FragmentHandle.PARSER);}
+ }
+
+ public static class HPlanFragment extends ProtoBufWrap<PlanFragment>{
+ public HPlanFragment() {super(PlanFragment.PARSER);}
+ public HPlanFragment(PlanFragment value) {super(value, PlanFragment.PARSER);}
+ }
+
+ public static class HandlePlan extends ProtoMap<FragmentHandle, PlanFragment, HFragmentHandle, HPlanFragment>{
+ public HandlePlan(HazelcastInstance instance) {super(instance, "plan-fragment-cache");}
+ public HFragmentHandle getNewKey(FragmentHandle key) {return new HFragmentHandle(key);}
+ public HPlanFragment getNewValue(PlanFragment value) {return new HPlanFragment(value);}
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
new file mode 100644
index 0000000..c3a9160
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoBufWrap.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * 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.drill.exec.cache;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.drill.exec.proto.ExecProtos.WorkQueueStatus;
+
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+import com.hazelcast.nio.DataSerializable;
+
+public abstract class ProtoBufWrap<T extends MessageLite> implements DataSerializable{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoBufWrap.class);
+
+ T value;
+ final Parser<T> parser;
+
+ public ProtoBufWrap(Parser<T> parser){
+ this(null, parser);
+ }
+
+ public ProtoBufWrap(T value, Parser<T> parser){
+ this.value = value;
+ this.parser = parser;
+ }
+
+ @Override
+ public void readData(DataInput arg0) throws IOException {
+ int len = arg0.readShort();
+ byte[] b = new byte[len];
+ arg0.readFully(b);
+ this.value = parser.parseFrom(b);
+ }
+
+ @Override
+ public void writeData(DataOutput arg0) throws IOException {
+ byte[] b = value.toByteArray();
+ if (b.length > Short.MAX_VALUE) throw new IOException("Unexpectedly long value.");
+ arg0.writeShort(b.length);
+ arg0.write(b);
+ }
+
+ protected T get() {
+ return value;
+ }
+
+ protected void set(T value) {
+ this.value = value;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
new file mode 100644
index 0000000..dac8201
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/ProtoMap.java
@@ -0,0 +1,52 @@
+/*******************************************************************************
+ * 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.drill.exec.cache;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.MessageLite;
+import com.google.protobuf.Parser;
+import com.hazelcast.core.HazelcastInstance;
+import com.hazelcast.core.IMap;
+
+public abstract class ProtoMap<K extends MessageLite, V extends MessageLite, HK extends ProtoBufWrap<K>, HV extends ProtoBufWrap<V>> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProtoMap.class);
+
+ private IMap<HK, HV> hzMap;
+
+ public ProtoMap(HazelcastInstance instance, String mapName){
+ hzMap = instance.getMap(mapName);
+ }
+
+ public V get(K key){
+ Preconditions.checkNotNull(key);
+ HK hk = getNewKey(key);
+ HV hv = hzMap.get(hk);
+ if(hv == null) return null;
+ return hv.get();
+ }
+
+ public V put(K key, V value){
+ Preconditions.checkNotNull(key);
+ Preconditions.checkNotNull(value);
+ HV oldValue = hzMap.put(getNewKey(key), getNewValue(value));
+ return oldValue.get();
+ }
+
+ public abstract HK getNewKey(K key);
+ public abstract HV getNewValue(V key);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedLogicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedLogicalPlan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedLogicalPlan.java
deleted file mode 100644
index 5ad9ef1..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedLogicalPlan.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.cache;
-
-public class TemplatizedLogicalPlan {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TemplatizedLogicalPlan.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedPhysicalPlan.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedPhysicalPlan.java
deleted file mode 100644
index 643720c..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/cache/TemplatizedPhysicalPlan.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.cache;
-
-public class TemplatizedPhysicalPlan {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TemplatizedPhysicalPlan.class);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index ee63213..bb7f77e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -25,24 +25,34 @@ import io.netty.buffer.ByteBufAllocator;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.channel.nio.NioEventLoopGroup;
+import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
+import java.util.List;
+import java.util.Vector;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.coord.ClusterCoordinator;
import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.proto.UserProtos.QueryHandle;
-import org.apache.drill.exec.rpc.DrillRpcFuture;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.proto.UserProtos.RpcType;
+import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
import org.apache.drill.exec.rpc.NamedThreadFactory;
import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
import org.apache.drill.exec.rpc.user.UserClient;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.rpc.user.UserRpcConfig;
/**
* Thin wrapper around a UserClient that handles connect/close and transforms String into ByteBuf
*/
-public class DrillClient {
-
+public class DrillClient implements Closeable{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
+
DrillConfig config;
private UserClient client;
private ClusterCoordinator clusterCoordinator;
@@ -56,8 +66,17 @@ public class DrillClient {
}
public DrillClient(DrillConfig config) {
+ this(config, null);
+ }
+
+ public DrillClient(DrillConfig config, ClusterCoordinator coordinator){
this.config = config;
+ this.clusterCoordinator = coordinator;
}
+
+
+
+
/**
* Connects the client to a Drillbit server
@@ -65,9 +84,11 @@ public class DrillClient {
* @throws IOException
*/
public void connect() throws Exception {
- this.clusterCoordinator = new ZKClusterCoordinator(this.config);
- this.clusterCoordinator.start();
- Thread.sleep(10000);
+ if(clusterCoordinator == null){
+ this.clusterCoordinator = new ZKClusterCoordinator(this.config);
+ this.clusterCoordinator.start(10000);
+ }
+
Collection<DrillbitEndpoint> endpoints = clusterCoordinator.getAvailableEndpoints();
checkState(!endpoints.isEmpty(), "No DrillbitEndpoint can be found");
// just use the first endpoint for now
@@ -75,7 +96,8 @@ public class DrillClient {
ByteBufAllocator bb = new PooledByteBufAllocator(true);
this.client = new UserClient(bb, new NioEventLoopGroup(1, new NamedThreadFactory("Client-")));
try {
- this.client.connectAsClient(endpoint.getAddress(), endpoint.getUserPort());
+ logger.debug("Connecting to server {}:{}", endpoint.getAddress(), endpoint.getUserPort());
+ this.client.connect(endpoint);
} catch (InterruptedException e) {
throw new IOException(e);
}
@@ -97,8 +119,37 @@ public class DrillClient {
* @return a handle for the query result
* @throws RpcException
*/
- public DrillRpcFuture<QueryHandle> submitPlan(String plan) throws RpcException {
- return this.client.submitQuery(newBuilder().setMode(STREAM_FULL).setPlan(plan).build(), null);
+ public List<QueryResultBatch> runQuery(QueryType type, String plan) throws RpcException {
+ try {
+ ListHoldingResultsListener listener = new ListHoldingResultsListener();
+ Future<Void> f = client.submitQuery(newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build(), listener);
+ f.get();
+ if(listener.ex != null){
+ throw listener.ex;
+ }else{
+ return listener.results;
+ }
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RpcException(e);
+ }
+ }
+
+ private class ListHoldingResultsListener extends UserResultsListener{
+ private RpcException ex;
+ private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
+
+ @Override
+ public void submissionFailed(RpcException ex) {
+ logger.debug("Submission failed.", ex);
+ this.ex = ex;
+ }
+
+ @Override
+ public void resultArrived(QueryResultBatch result) {
+ logger.debug("Result arrived. Is Last Chunk: {}. Full Result: {}", result.getHeader().getIsLastChunk(), result);
+ results.add(result);
+ }
+
}
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
index d3580b5..7fb1f5b 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/ClusterCoordinator.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.coord;
import java.io.Closeable;
import java.util.Collection;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
/**
* Pluggable interface built to manage cluster coordination. Allows Drillbit or DrillClient to register its capabilities
@@ -29,7 +29,12 @@ import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
public abstract class ClusterCoordinator implements Closeable {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ClusterCoordinator.class);
- public abstract void start() throws Exception;
+ /**
+ * Start the cluster coordinator. Millis to wait is
+ * @param millisToWait The maximum time to wait before throwing an exception if the cluster coordination service has not successfully started. Use 0 to wait indefinitely.
+ * @throws Exception
+ */
+ public abstract void start(long millisToWait) throws Exception;
public abstract RegistrationHandle register(DrillbitEndpoint data);
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
index ce0fb92..289aa3c 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillServiceInstanceHelper.java
@@ -17,9 +17,9 @@
******************************************************************************/
package org.apache.drill.exec.coord;
-import org.apache.drill.common.proto.CoordinationProtos.DrillServiceInstance;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillServiceInstance;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
import com.netflix.curator.x.discovery.ServiceInstance;
import com.netflix.curator.x.discovery.ServiceInstanceBuilder;
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
new file mode 100644
index 0000000..5886c2c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/DrillbitEndpointSerDe.java
@@ -0,0 +1,65 @@
+/*******************************************************************************
+ * 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.drill.exec.coord;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+public class DrillbitEndpointSerDe {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitEndpointSerDe.class);
+
+ public static class De extends StdDeserializer<DrillbitEndpoint> {
+
+ public De() {
+ super(DrillbitEndpoint.class);
+ }
+
+ @Override
+ public DrillbitEndpoint deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+ JsonProcessingException {
+ return DrillbitEndpoint.parseFrom(jp.getBinaryValue());
+ }
+
+
+ }
+
+
+ public static class Se extends StdSerializer<DrillbitEndpoint> {
+
+ public Se() {
+ super(DrillbitEndpoint.class);
+ }
+
+ @Override
+ public void serialize(DrillbitEndpoint value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+ JsonGenerationException {
+ jgen.writeBinary(value.toByteArray());
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
new file mode 100644
index 0000000..43a5430
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/coord/LocalClusterCoordinator.java
@@ -0,0 +1,95 @@
+/*******************************************************************************
+ * 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.drill.exec.coord;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Maps;
+
+public class LocalClusterCoordinator extends ClusterCoordinator{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(LocalClusterCoordinator.class);
+
+ private volatile Map<RegistrationHandle, DrillbitEndpoint> endpoints;
+
+ @Override
+ public void close() throws IOException {
+ endpoints = null;
+ }
+
+ @Override
+ public void start(long millis) throws Exception {
+ logger.debug("Local Cluster Coordinator started.");
+ endpoints = Maps.newConcurrentMap();
+ }
+
+ @Override
+ public RegistrationHandle register(DrillbitEndpoint data) {
+ logger.debug("Endpoint registered {}.", data);
+ Handle h = new Handle();
+ endpoints.put(h, data);
+ return h;
+ }
+
+ @Override
+ public void unregister(RegistrationHandle handle) {
+ endpoints.remove(handle);
+ }
+
+ @Override
+ public Collection<DrillbitEndpoint> getAvailableEndpoints() {
+ return endpoints.values();
+ }
+
+
+ private class Handle implements RegistrationHandle{
+ UUID id = UUID.randomUUID();
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + getOuterType().hashCode();
+ result = prime * result + ((id == null) ? 0 : id.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null) return false;
+ if (getClass() != obj.getClass()) return false;
+ Handle other = (Handle) obj;
+ if (!getOuterType().equals(other.getOuterType())) return false;
+ if (id == null) {
+ if (other.id != null) return false;
+ } else if (!id.equals(other.id)) return false;
+ return true;
+ }
+
+ private LocalClusterCoordinator getOuterType() {
+ return LocalClusterCoordinator.this;
+ }
+
+ }
+
+}
[03/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
new file mode 100644
index 0000000..2900d99
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunner.java
@@ -0,0 +1,124 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+
+import com.yammer.metrics.Timer;
+
+/**
+ * Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
+ * messages. Two child implementation, root (driving) and child (driven) exist.
+ */
+public class FragmentRunner implements Runnable, CancelableQuery, StatusProvider, Comparable<Object>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunner.class);
+
+ private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
+ private final RootExec root;
+ private final FragmentContext context;
+ private final FragmentRunnerListener listener;
+
+ public FragmentRunner(FragmentContext context, RootExec root, FragmentRunnerListener listener){
+ this.context = context;
+ this.root = root;
+ this.listener = listener;
+ }
+
+ @Override
+ public FragmentStatus getStatus() {
+ return FragmentStatus.newBuilder() //
+ .setBatchesCompleted(context.batchesCompleted.get()) //
+ .setDataProcessed(context.dataProcessed.get()) //
+ .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
+ .build();
+ }
+
+ @Override
+ public void cancel() {
+ updateState(FragmentState.CANCELLED);
+ }
+
+ public UserClientConnection getClient(){
+ return context.getConnection();
+ }
+
+ @Override
+ public void run() {
+ if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
+ internalFail(new RuntimeException(String.format("Run was called when fragment was in %s state. FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
+ return;
+ }
+
+ Timer.Context t = context.fragmentTime.time();
+
+ // run the query until root.next returns false.
+ try{
+ while(state.get() == FragmentState.RUNNING_VALUE){
+ if(!root.next()){
+ updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
+ }
+ }
+
+ // If this isn't a finished stop, we'll inform other batches to finish up.
+ if(state.get() != FragmentState.FINISHED_VALUE){
+ root.stop();
+ }
+
+ }catch(Exception ex){
+ internalFail(ex);
+ }finally{
+ t.stop();
+ }
+
+ }
+
+ private void internalFail(Throwable excep){
+ state.set(FragmentState.FAILED_VALUE);
+ listener.fail(context.getHandle(), "Failure while running fragment.", excep);
+ }
+
+ private void updateState(FragmentState update){
+ state.set(update.getNumber());
+ listener.stateChanged(context.getHandle(), update);
+ }
+
+ private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
+ boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
+ if (!success && exceptionOnFailure) {
+ internalFail(new RuntimeException(String.format(
+ "State was different than expected. Attempting to update state from %s to %s however current state was %s.",
+ current.name(), update.name(), FragmentState.valueOf(state.get()))));
+ return false;
+ }
+ listener.stateChanged(context.getHandle(), update);
+ return true;
+ }
+
+ @Override
+ public int compareTo(Object o) {
+ return o.hashCode() - this.hashCode();
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
new file mode 100644
index 0000000..d978470
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/FragmentRunnerListener.java
@@ -0,0 +1,28 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+
+public interface FragmentRunnerListener {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunnerListener.class);
+
+ void fail(FragmentHandle handle, String message, Throwable excep);
+ void stateChanged(FragmentHandle handle, FragmentState newState);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
new file mode 100644
index 0000000..243d677
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/QueryWorkUnit.java
@@ -0,0 +1,64 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+
+import com.google.common.base.Preconditions;
+
+public class QueryWorkUnit {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryWorkUnit.class);
+
+ private final PlanFragment rootFragment; // for local
+ private final FragmentRoot rootOperator; // for local
+ private final List<PlanFragment> fragments;
+
+ public QueryWorkUnit(FragmentRoot rootOperator, PlanFragment rootFragment, List<PlanFragment> fragments) {
+ super();
+ Preconditions.checkNotNull(rootFragment);
+ Preconditions.checkNotNull(fragments);
+ Preconditions.checkNotNull(rootOperator);
+
+ this.rootFragment = rootFragment;
+ this.fragments = fragments;
+ this.rootOperator = rootOperator;
+ }
+
+ public PlanFragment getRootFragment() {
+ return rootFragment;
+ }
+
+ public List<PlanFragment> getFragments() {
+ return fragments;
+ }
+
+ public FragmentRoot getRootOperator() {
+ return rootOperator;
+ }
+
+
+
+
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
new file mode 100644
index 0000000..abcb312
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RecordOutputStream.java
@@ -0,0 +1,22 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+public class RecordOutputStream {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordOutputStream.class);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
new file mode 100644
index 0000000..74fcd2b
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RemotingFragmentRunnerListener.java
@@ -0,0 +1,46 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.Builder;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.work.foreman.ErrorHelper;
+
+/**
+ * Informs remote node as fragment changes state.
+ */
+public class RemotingFragmentRunnerListener extends AbstractFragmentRunnerListener{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RemotingFragmentRunnerListener.class);
+
+ private final BitTunnel tunnel;
+
+ public RemotingFragmentRunnerListener(FragmentContext context, BitTunnel tunnel) {
+ super(context);
+ this.tunnel = tunnel;
+ }
+
+ @Override
+ protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+ tunnel.sendFragmentStatus(status);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
new file mode 100644
index 0000000..2e1296e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/ResourceRequest.java
@@ -0,0 +1,30 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+public class ResourceRequest {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ResourceRequest.class);
+
+ public long memoryMin;
+ public long memoryDesired;
+
+
+ public static class ResourceAllocation {
+ public long memory;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
new file mode 100644
index 0000000..12da7ba
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/RootNodeDriver.java
@@ -0,0 +1,25 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+public interface RootNodeDriver {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RootNodeDriver.class);
+
+ public boolean doNext();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
new file mode 100644
index 0000000..bbd9df7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/StatusProvider.java
@@ -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.drill.exec.work;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+
+public interface StatusProvider {
+ public FragmentStatus getStatus();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
new file mode 100644
index 0000000..d3664a0
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * 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.drill.exec.work;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.coord.ClusterCoordinator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.server.BootStrapContext;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.work.batch.BitComHandler;
+import org.apache.drill.exec.work.batch.BitComHandlerImpl;
+import org.apache.drill.exec.work.foreman.Foreman;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+import org.apache.drill.exec.work.user.UserWorker;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Queues;
+
+public class WorkManager implements Closeable{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WorkManager.class);
+
+ private Set<IncomingFragmentHandler> incomingFragments = Collections.newSetFromMap(Maps.<IncomingFragmentHandler, Boolean> newConcurrentMap());
+
+ private PriorityBlockingQueue<Runnable> pendingTasks = Queues.newPriorityBlockingQueue();
+
+ private Map<FragmentHandle, FragmentRunner> runningFragments = Maps.newConcurrentMap();
+
+ private ConcurrentMap<QueryId, Foreman> queries = Maps.newConcurrentMap();
+
+ private BootStrapContext bContext;
+ private DrillbitContext dContext;
+
+ private final BitComHandler bitComWorker;
+ private final UserWorker userWorker;
+ private final WorkerBee bee;
+ private Executor executor = Executors.newFixedThreadPool(4);
+ private final EventThread eventThread;
+
+ public WorkManager(BootStrapContext context){
+ this.bee = new WorkerBee();
+ this.bContext = context;
+ this.bitComWorker = new BitComHandlerImpl(bee);
+ this.userWorker = new UserWorker(bee);
+ this.eventThread = new EventThread();
+
+ }
+
+ public void start(DrillbitEndpoint endpoint, DistributedCache cache, BitCom com, ClusterCoordinator coord){
+ this.dContext = new DrillbitContext(endpoint, bContext, coord, com, cache);
+ eventThread.start();
+ }
+
+ public BitComHandler getBitComWorker(){
+ return bitComWorker;
+ }
+
+ public UserWorker getUserWorker(){
+ return userWorker;
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+
+
+ public DrillbitContext getContext() {
+ return dContext;
+ }
+
+ // create this so items can see the data here whether or not they are in this package.
+ public class WorkerBee{
+
+ public void addFragmentRunner(FragmentRunner runner){
+ pendingTasks.add(runner);
+ }
+
+ public void addNewForeman(Foreman foreman){
+ pendingTasks.add(foreman);
+ }
+
+
+ public void addFragmentPendingRemote(IncomingFragmentHandler handler){
+ incomingFragments.add(handler);
+ }
+
+ public void startFragmentPendingRemote(IncomingFragmentHandler handler){
+ incomingFragments.remove(handler);
+ pendingTasks.add(handler.getRunnable());
+ }
+
+ public FragmentRunner getFragmentRunner(FragmentHandle handle){
+ return runningFragments.get(handle);
+ }
+
+ public Foreman getForemanForQueryId(QueryId queryId){
+ return queries.get(queryId);
+ }
+
+ public void retireForeman(Foreman foreman){
+ queries.remove(foreman.getQueryId(), foreman);
+ }
+
+ public DrillbitContext getContext() {
+ return dContext;
+ }
+
+ }
+
+
+
+ private class EventThread extends Thread{
+ public EventThread(){
+ this.setDaemon(true);
+ this.setName("WorkManager Event Thread");
+ }
+
+ @Override
+ public void run() {
+ try {
+ while(true){
+ logger.debug("Checking for pending work tasks.");
+ Runnable r = pendingTasks.poll(10, TimeUnit.SECONDS);
+ if(r != null){
+ executor.execute(r);
+ }
+
+ }
+ } catch (InterruptedException e) {
+ logger.info("Work Manager stopping as it was interrupted.");
+ }
+ }
+
+
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
new file mode 100644
index 0000000..5dacb71
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/AbstractFragmentCollector.java
@@ -0,0 +1,84 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.base.Preconditions;
+
+public abstract class AbstractFragmentCollector implements BatchCollector{
+ private final List<DrillbitEndpoint> incoming;
+ private final int oppositeMajorFragmentId;
+ private final AtomicIntegerArray remainders;
+ private final AtomicInteger remainingRequired;
+ protected final RawBatchBuffer[] buffers;
+ private final AtomicInteger parentAccounter;
+ private final AtomicInteger finishedStreams = new AtomicInteger();
+
+ public AbstractFragmentCollector(AtomicInteger parentAccounter, Receiver receiver, int minInputsRequired) {
+ Preconditions.checkArgument(minInputsRequired > 0);
+ Preconditions.checkNotNull(receiver);
+ Preconditions.checkNotNull(parentAccounter);
+
+ this.parentAccounter = parentAccounter;
+ this.incoming = receiver.getProvidingEndpoints();
+ this.remainders = new AtomicIntegerArray(incoming.size());
+ this.oppositeMajorFragmentId = receiver.getOppositeMajorFragmentId();
+ this.buffers = new RawBatchBuffer[minInputsRequired];
+ for(int i = 0; i < buffers.length; i++){
+ buffers[i] = new UnlmitedRawBatchBuffer();
+ }
+ if (receiver.supportsOutOfOrderExchange()) {
+ this.remainingRequired = new AtomicInteger(1);
+ } else {
+ this.remainingRequired = new AtomicInteger(minInputsRequired);
+ }
+ }
+
+ public int getOppositeMajorFragmentId() {
+ return oppositeMajorFragmentId;
+ }
+
+ public RawBatchBuffer[] getBuffers(){
+ return buffers;
+ }
+
+ public abstract void streamFinished(int minorFragmentId);
+
+ public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch) {
+ if (remainders.compareAndSet(minorFragmentId, 0, 1)) {
+ int rem = remainingRequired.decrementAndGet();
+ if (rem == 0) {
+ parentAccounter.decrementAndGet();
+ }
+ }
+ if(batch.getHeader().getIsLastBatch()){
+ streamFinished(minorFragmentId);
+ }
+ getBuffer(minorFragmentId).enqueue(throttle, batch);
+ }
+
+ protected abstract RawBatchBuffer getBuffer(int minorFragmentId);
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
new file mode 100644
index 0000000..ff091d7
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BatchCollector.java
@@ -0,0 +1,32 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+
+interface BatchCollector {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchCollector.class);
+
+ public void batchArrived(ConnectionThrottle throttle, int minorFragmentId, RawFragmentBatch batch);
+ public int getOppositeMajorFragmentId();
+ public RawBatchBuffer[] getBuffers();
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
new file mode 100644
index 0000000..97064e3
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandler.java
@@ -0,0 +1,41 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+
+public interface BitComHandler {
+
+ public abstract Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody)
+ throws RpcException;
+
+ public abstract void startNewRemoteFragment(PlanFragment fragment);
+
+ public abstract Ack cancelFragment(FragmentHandle handle);
+
+ public abstract void registerIncomingFragmentHandler(IncomingFragmentHandler handler);
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
new file mode 100644
index 0000000..9b227da
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/BitComHandlerImpl.java
@@ -0,0 +1,205 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import static org.apache.drill.exec.rpc.RpcBus.get;
+import io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.ExecProtos.BitHandshake;
+import org.apache.drill.exec.proto.ExecProtos.BitStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.ExecProtos.RpcType;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.Acks;
+import org.apache.drill.exec.rpc.RemoteConnection;
+import org.apache.drill.exec.rpc.Response;
+import org.apache.drill.exec.rpc.RpcConstants;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitConnection;
+import org.apache.drill.exec.rpc.bit.BitRpcConfig;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.RemotingFragmentRunnerListener;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.fragment.IncomingFragmentHandler;
+import org.apache.drill.exec.work.fragment.RemoteFragmentHandler;
+
+import com.google.common.collect.Maps;
+import com.google.protobuf.MessageLite;
+
+public class BitComHandlerImpl implements BitComHandler {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BitComHandlerImpl.class);
+
+ private ConcurrentMap<FragmentHandle, IncomingFragmentHandler> handlers = Maps.newConcurrentMap();
+ private final WorkerBee bee;
+
+ public BitComHandlerImpl(WorkerBee bee) {
+ super();
+ this.bee = bee;
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#handle(org.apache.drill.exec.rpc.bit.BitConnection, int, io.netty.buffer.ByteBuf, io.netty.buffer.ByteBuf)
+ */
+ @Override
+ public Response handle(BitConnection connection, int rpcType, ByteBuf pBody, ByteBuf dBody) throws RpcException {
+ if(RpcConstants.EXTRA_DEBUGGING) logger.debug("Received bit com message of type {}", rpcType);
+
+ switch (rpcType) {
+
+ case RpcType.REQ_CANCEL_FRAGMENT_VALUE:
+ FragmentHandle handle = get(pBody, FragmentHandle.PARSER);
+ cancelFragment(handle);
+ return BitRpcConfig.OK;
+
+ case RpcType.REQ_FRAGMENT_STATUS_VALUE:
+ connection.getListenerPool().status( get(pBody, FragmentStatus.PARSER));
+ // TODO: Support a type of message that has no response.
+ return BitRpcConfig.OK;
+
+ case RpcType.REQ_INIATILIZE_FRAGMENT_VALUE:
+ PlanFragment fragment = get(pBody, PlanFragment.PARSER);
+ startNewRemoteFragment(fragment);
+ return BitRpcConfig.OK;
+
+ case RpcType.REQ_RECORD_BATCH_VALUE:
+ try {
+ FragmentRecordBatch header = get(pBody, FragmentRecordBatch.PARSER);
+ incomingRecordBatch(connection, header, dBody);
+ return BitRpcConfig.OK;
+ } catch (FragmentSetupException e) {
+ throw new RpcException("Failure receiving record batch.", e);
+ }
+
+ default:
+ throw new RpcException("Not yet supported.");
+ }
+
+ }
+
+
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#startNewRemoteFragment(org.apache.drill.exec.proto.ExecProtos.PlanFragment)
+ */
+ @Override
+ public void startNewRemoteFragment(PlanFragment fragment){
+ logger.debug("Received remote fragment start instruction", fragment);
+ FragmentContext context = new FragmentContext(bee.getContext(), fragment.getHandle(), null, null);
+ BitTunnel tunnel = bee.getContext().getBitCom().getTunnel(fragment.getForeman());
+ RemotingFragmentRunnerListener listener = new RemotingFragmentRunnerListener(context, tunnel);
+ try{
+ FragmentRoot rootOperator = bee.getContext().getPlanReader().readFragmentOperator(fragment.getFragmentJson());
+ RootExec exec = ImplCreator.getExec(context, rootOperator);
+ FragmentRunner fr = new FragmentRunner(context, exec, listener);
+ bee.addFragmentRunner(fr);
+
+ }catch(IOException e){
+ listener.fail(fragment.getHandle(), "Failure while parsing fragment execution plan.", e);
+ }catch(ExecutionSetupException e){
+ listener.fail(fragment.getHandle(), "Failure while setting up execution plan.", e);
+ }
+
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#cancelFragment(org.apache.drill.exec.proto.ExecProtos.FragmentHandle)
+ */
+ @Override
+ public Ack cancelFragment(FragmentHandle handle){
+ IncomingFragmentHandler handler = handlers.get(handle);
+ if(handler != null){
+ // try remote fragment cancel.
+ handler.cancel();
+ }else{
+ // then try local cancel.
+ FragmentRunner runner = bee.getFragmentRunner(handle);
+ if(runner != null) runner.cancel();
+ }
+
+ return Acks.OK;
+ }
+
+
+ /**
+ * Returns a positive Ack if this fragment is accepted.
+ */
+ private Ack incomingRecordBatch(RemoteConnection connection, FragmentRecordBatch fragmentBatch, ByteBuf body) throws FragmentSetupException{
+ FragmentHandle handle = fragmentBatch.getHandle();
+ IncomingFragmentHandler handler = handlers.get(handle);
+
+ // Create a handler if there isn't already one.
+ if(handler == null){
+
+
+
+ PlanFragment fragment = bee.getContext().getCache().getFragment(handle);
+ if(fragment == null){
+ logger.error("Received batch where fragment was not in cache.");
+ return Acks.FAIL;
+ }
+
+
+ IncomingFragmentHandler newHandler = new RemoteFragmentHandler(fragment, bee.getContext(), bee.getContext().getBitCom().getTunnel(fragment.getForeman()));
+
+ // since their could be a race condition on the check, we'll use putIfAbsent so we don't have two competing handlers.
+ handler = handlers.putIfAbsent(fragment.getHandle(), newHandler);
+
+ if(handler == null){
+ // we added a handler, inform foreman that we did so. This way, the foreman can track status. We also tell foreman that we don't need inform ourself.
+ bee.addFragmentPendingRemote(newHandler);
+ handler = newHandler;
+ }
+ }
+
+ boolean canRun = handler.handle(connection.getConnectionThrottle(), new RawFragmentBatch(fragmentBatch, body));
+ if(canRun){
+ // if we've reached the canRun threshold, we'll proceed. This expects handler.handle() to only return a single true.
+ bee.startFragmentPendingRemote(handler);
+ }
+ if(handler.isDone()){
+ handlers.remove(handler.getHandle());
+ }
+
+ return Acks.OK;
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.drill.exec.work.batch.BitComHandler#registerIncomingFragmentHandler(org.apache.drill.exec.work.fragment.IncomingFragmentHandler)
+ */
+ @Override
+ public void registerIncomingFragmentHandler(IncomingFragmentHandler handler){
+ IncomingFragmentHandler old = handlers.putIfAbsent(handler.getHandle(), handler);
+ assert old == null : "You can only register a fragment handler if one hasn't been registered already.";
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
new file mode 100644
index 0000000..20775c5
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/IncomingBuffers.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Receiver;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+/**
+ * Determines when a particular fragment has enough data for each of its receiving exchanges to commence execution.
+ */
+public class IncomingBuffers {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IncomingBuffers.class);
+
+ private final AtomicInteger streamsRemaining = new AtomicInteger(0);
+ private final AtomicInteger remainingRequired = new AtomicInteger(0);
+ private final Map<Integer, BatchCollector> fragCounts;
+
+ public IncomingBuffers(PhysicalOperator root) {
+ Map<Integer, BatchCollector> counts = Maps.newHashMap();
+ root.accept(new CountRequiredFragments(), counts);
+ streamsRemaining.set(remainingRequired.get());
+ fragCounts = ImmutableMap.copyOf(counts);
+ }
+
+ public boolean batchArrived(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException {
+ // no need to do anything if we've already enabled running.
+ logger.debug("New Batch Arrived {}", batch);
+ if(batch.getHeader().getIsLastBatch()){
+ streamsRemaining.decrementAndGet();
+ }
+
+ BatchCollector fSet = fragCounts.get(batch.getHeader().getSendingMajorFragmentId());
+ if (fSet == null) throw new FragmentSetupException("We received a major fragment id that we were not expecting.");
+ fSet.batchArrived(throttle, batch.getHeader().getSendingMinorFragmentId(), batch);
+ return remainingRequired.get() == 0;
+ }
+
+ public int getRemainingRequired() {
+ int rem = remainingRequired.get();
+ if (rem < 0) return 0;
+ return rem;
+ }
+
+ public RawBatchBuffer[] getBuffers(int senderMajorFragmentId){
+ return fragCounts.get(senderMajorFragmentId).getBuffers();
+ }
+
+
+ /**
+ * Designed to setup initial values for arriving fragment accounting.
+ */
+ public class CountRequiredFragments extends AbstractPhysicalVisitor<Void, Map<Integer, BatchCollector>, RuntimeException> {
+
+ @Override
+ public Void visitReceiver(Receiver receiver, Map<Integer, BatchCollector> counts) throws RuntimeException {
+ BatchCollector set;
+ if (receiver.supportsOutOfOrderExchange()) {
+ set = new MergingCollector(remainingRequired, receiver);
+ } else {
+ set = new PartitionedCollector(remainingRequired, receiver);
+ }
+
+ counts.put(set.getOppositeMajorFragmentId(), set);
+ remainingRequired.incrementAndGet();
+ return null;
+ }
+
+
+ @Override
+ public Void visitOp(PhysicalOperator op, Map<Integer, BatchCollector> value) throws RuntimeException {
+ for(PhysicalOperator o : op){
+ o.accept(this, value);
+ }
+ return null;
+ }
+
+
+ }
+
+ public boolean isDone(){
+ return streamsRemaining.get() < 1;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
new file mode 100644
index 0000000..e21d69a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/MergingCollector.java
@@ -0,0 +1,45 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.physical.base.Receiver;
+
+public class MergingCollector extends AbstractFragmentCollector{
+
+ private AtomicInteger streamsRunning;
+
+ public MergingCollector(AtomicInteger parentAccounter, Receiver receiver) {
+ super(parentAccounter, receiver, 1);
+ streamsRunning = new AtomicInteger(parentAccounter.get());
+ }
+
+ @Override
+ protected RawBatchBuffer getBuffer(int minorFragmentId) {
+ return buffers[0];
+ }
+
+ @Override
+ public void streamFinished(int minorFragmentId) {
+ if(streamsRunning.decrementAndGet() == 0) buffers[0].finished();
+ }
+
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
new file mode 100644
index 0000000..116ca26
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/PartitionedCollector.java
@@ -0,0 +1,42 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.physical.base.Receiver;
+
+public class PartitionedCollector extends AbstractFragmentCollector{
+
+ public PartitionedCollector(AtomicInteger parentAccounter, Receiver receiver) {
+ super(parentAccounter, receiver, receiver.getProvidingEndpoints().size());
+ }
+
+ @Override
+ protected RawBatchBuffer getBuffer(int minorFragmentId) {
+ return buffers[minorFragmentId];
+ }
+
+ @Override
+ public void streamFinished(int minorFragmentId) {
+ buffers[minorFragmentId].finished();
+ }
+
+
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
new file mode 100644
index 0000000..0f10e26
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/RawBatchBuffer.java
@@ -0,0 +1,33 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.record.RawFragmentBatchProvider;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+public interface RawBatchBuffer extends RawFragmentBatchProvider{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawBatchBuffer.class);
+
+ public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch);
+
+ /**
+ * Inform the buffer that no more records are expected.
+ */
+ public void finished();
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
new file mode 100644
index 0000000..f97d878
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/batch/UnlmitedRawBatchBuffer.java
@@ -0,0 +1,73 @@
+/*******************************************************************************
+ * 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.drill.exec.work.batch;
+
+import java.util.Iterator;
+import java.util.concurrent.LinkedBlockingDeque;
+
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+
+import com.google.common.collect.Queues;
+
+public class UnlmitedRawBatchBuffer implements RawBatchBuffer{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UnlmitedRawBatchBuffer.class);
+
+ private final LinkedBlockingDeque<RawFragmentBatch> buffer = Queues.newLinkedBlockingDeque();
+ private volatile boolean finished = false;
+
+ @Override
+ public void enqueue(ConnectionThrottle throttle, RawFragmentBatch batch) {
+ buffer.add(batch);
+ }
+
+// @Override
+// public RawFragmentBatch dequeue() {
+// return buffer.poll();
+// }
+
+ @Override
+ public void kill(FragmentContext context) {
+ // TODO: Pass back or kill handler?
+ }
+
+
+ @Override
+ public void finished() {
+ finished = true;
+ }
+
+ @Override
+ public RawFragmentBatch getNext(){
+
+ RawFragmentBatch b = buffer.poll();
+ if(b == null && !finished){
+ try {
+ return buffer.take();
+ } catch (InterruptedException e) {
+ return null;
+ }
+ }
+
+ return null;
+
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
new file mode 100644
index 0000000..d4c4014
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/ErrorHelper.java
@@ -0,0 +1,47 @@
+/*******************************************************************************
+ * 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.drill.exec.work.foreman;
+
+import java.util.UUID;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.slf4j.Logger;
+
+
+public class ErrorHelper {
+
+ public static DrillPBError logAndConvertError(DrillbitEndpoint endpoint, String message, Throwable t, Logger logger){
+ String id = UUID.randomUUID().toString();
+ DrillPBError.Builder builder = DrillPBError.newBuilder();
+ builder.setEndpoint(endpoint);
+ builder.setErrorId(id);
+ if(message != null){
+ builder.setMessage(message);
+ }else{
+ builder.setMessage(t.getMessage());
+ }
+ builder.setErrorType(0);
+
+ // record the error to the log for later reference.
+ logger.error("Error {}: {}", id, message, t);
+
+
+ return builder.build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
new file mode 100644
index 0000000..dea8282
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -0,0 +1,272 @@
+/*******************************************************************************
+ * 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.drill.exec.work.foreman;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.proto.UserProtos.RequestResults;
+import org.apache.drill.exec.proto.UserProtos.RunQuery;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.util.AtomicState;
+import org.apache.drill.exec.work.QueryWorkUnit;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Foreman manages all queries where this is the driving/root node.
+ */
+public class Foreman implements Runnable, Closeable, Comparable<Object>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Foreman.class);
+
+ private QueryId queryId;
+ private RunQuery queryRequest;
+ private QueryContext context;
+ private RunningFragmentManager fragmentManager;
+ private WorkerBee bee;
+ private UserClientConnection initiatingClient;
+ private final AtomicState<QueryState> state;
+
+
+ public Foreman(WorkerBee bee, DrillbitContext dContext, UserClientConnection connection, QueryId queryId,
+ RunQuery queryRequest) {
+ this.queryId = queryId;
+ this.queryRequest = queryRequest;
+ this.context = new QueryContext(queryId, dContext);
+ this.initiatingClient = connection;
+ this.fragmentManager = new RunningFragmentManager(new ForemanManagerListener(), new TunnelManager(dContext.getBitCom()));
+ this.bee = bee;
+
+ this.state = new AtomicState<QueryState>(QueryState.PENDING) {
+ protected QueryState getStateFromNumber(int i) {
+ return QueryState.valueOf(i);
+ }
+ };
+ }
+
+ private boolean isFinished(){
+ switch(state.getState()){
+ case PENDING:
+ case RUNNING:
+ return false;
+ default:
+ return true;
+ }
+
+ }
+
+ private void fail(String message, Throwable t) {
+ if(isFinished()){
+ logger.error("Received a failure message query finished of: {}", message, t);
+ }
+ DrillPBError error = ErrorHelper.logAndConvertError(context.getCurrentEndpoint(), message, t, logger);
+ QueryResult result = QueryResult //
+ .newBuilder() //
+ .addError(error) //
+ .setIsLastChunk(true) //
+ .setQueryState(QueryState.FAILED) //
+ .build();
+ cleanupAndSendResult(result);
+ }
+
+
+ public void cancel() {
+ if(isFinished()){
+ return;
+ }
+
+ // cancel remote fragments.
+ fragmentManager.cancel();
+
+ QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.CANCELED).setIsLastChunk(true).setQueryId(queryId).build();
+ cleanupAndSendResult(result);
+ }
+
+ void cleanupAndSendResult(QueryResult result){
+ bee.retireForeman(this);
+ initiatingClient.sendResult(new QueryWritableBatch(result)).addLightListener(new ResponseSendListener());
+ }
+
+ private class ResponseSendListener extends RpcOutcomeListener<Ack> {
+ @Override
+ public void failed(RpcException ex) {
+ logger
+ .info(
+ "Failure while trying communicate query result to initating client. This would happen if a client is disconnected before response notice can be sent.",
+ ex);
+ }
+ }
+
+
+
+ /**
+ * Called by execution pool to do foreman setup. Actual query execution is a separate phase (and can be scheduled).
+ */
+ public void run() {
+ // convert a run query request into action
+
+ switch (queryRequest.getType()) {
+
+ case LOGICAL:
+ parseAndRunLogicalPlan(queryRequest.getPlan());
+ break;
+ case PHYSICAL:
+ parseAndRunPhysicalPlan(queryRequest.getPlan());
+ break;
+ case SQL:
+ runSQL(queryRequest.getPlan());
+ break;
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private void parseAndRunLogicalPlan(String json) {
+ try {
+ LogicalPlan logicalPlan = context.getPlanReader().readLogicalPlan(json);
+ PhysicalPlan physicalPlan = convert(logicalPlan);
+ runPhysicalPlan(physicalPlan);
+ } catch (IOException e) {
+ fail("Failure while parsing logical plan.", e);
+ }
+ }
+
+ private void parseAndRunPhysicalPlan(String json) {
+ try {
+ PhysicalPlan plan = context.getPlanReader().readPhysicalPlan(json);
+ runPhysicalPlan(plan);
+ } catch (IOException e) {
+ fail("Failure while parsing physical plan.", e);
+ }
+ }
+
+ private void runPhysicalPlan(PhysicalPlan plan) {
+
+ PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next();
+ MakeFragmentsVisitor makeFragmentsVisitor = new MakeFragmentsVisitor();
+ Fragment rootFragment;
+ try {
+ rootFragment = rootOperator.accept(makeFragmentsVisitor, null);
+ } catch (FragmentSetupException e) {
+ fail("Failure while fragmenting query.", e);
+ return;
+ }
+ PlanningSet planningSet = StatsCollector.collectStats(rootFragment);
+ SimpleParallelizer parallelizer = new SimpleParallelizer();
+
+ try {
+ QueryWorkUnit work = parallelizer.getFragments(context.getCurrentEndpoint(), queryId, context.getActiveEndpoints(), context.getPlanReader(), rootFragment, planningSet, 10);
+
+ List<PlanFragment> leafFragments = Lists.newArrayList();
+
+ // store fragments in distributed grid.
+ for (PlanFragment f : work.getFragments()) {
+ if (f.getLeafFragment()) {
+ leafFragments.add(f);
+ } else {
+ context.getCache().storeFragment(f);
+ }
+ }
+
+ fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments);
+
+
+ } catch (ExecutionSetupException e) {
+ fail("Failure while setting up query.", e);
+ }
+
+ }
+
+ private void runSQL(String json) {
+ throw new UnsupportedOperationException();
+ }
+
+ private PhysicalPlan convert(LogicalPlan plan) {
+ throw new UnsupportedOperationException();
+ }
+
+ public QueryResult getResult(UserClientConnection connection, RequestResults req) {
+
+ return null;
+ }
+
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ @Override
+ public void close() throws IOException {
+ }
+
+ QueryState getQueryState(){
+ return this.state.getState();
+ }
+
+ public boolean rootCoorespondsTo(FragmentHandle handle){
+ throw new UnsupportedOperationException();
+ }
+
+ class ForemanManagerListener{
+ void fail(String message, Throwable t) {
+ ForemanManagerListener.this.fail(message, t);
+ }
+
+ void cleanupAndSendResult(QueryResult result){
+ ForemanManagerListener.this.cleanupAndSendResult(result);
+ }
+
+ }
+
+
+
+ @Override
+ public int compareTo(Object o) {
+ return o.hashCode() - o.hashCode();
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
new file mode 100644
index 0000000..d906ba2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentStatusListener.java
@@ -0,0 +1,26 @@
+/*******************************************************************************
+ * 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.drill.exec.work.foreman;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+
+public interface FragmentStatusListener {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatusListener.class);
+
+ public void statusUpdate(FragmentStatus status);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
new file mode 100644
index 0000000..20797b8
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/RunningFragmentManager.java
@@ -0,0 +1,266 @@
+/*******************************************************************************
+ * 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.drill.exec.work.foreman;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.RootExec;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
+import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserProtos.QueryResult;
+import org.apache.drill.exec.proto.UserProtos.QueryResult.QueryState;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.RpcOutcomeListener;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.work.AbstractFragmentRunnerListener;
+import org.apache.drill.exec.work.EndpointListener;
+import org.apache.drill.exec.work.FragmentRunner;
+import org.apache.drill.exec.work.WorkManager.WorkerBee;
+import org.apache.drill.exec.work.batch.IncomingBuffers;
+import org.apache.drill.exec.work.foreman.Foreman.ForemanManagerListener;
+import org.apache.drill.exec.work.fragment.LocalFragmentHandler;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Each Foreman holds its own fragment manager. This manages the events associated with execution of a particular query across all fragments.
+ */
+class RunningFragmentManager implements FragmentStatusListener{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RunningFragmentManager.class);
+
+ public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be
+ private final TunnelManager tun;
+ private ForemanManagerListener foreman;
+ private AtomicInteger remainingFragmentCount;
+ private FragmentRunner rootRunner;
+
+ public RunningFragmentManager(ForemanManagerListener foreman, TunnelManager tun) {
+ super();
+ this.foreman = foreman;
+ this.tun = tun;
+ this.remainingFragmentCount = new AtomicInteger(0);
+ }
+
+ public void runFragments(WorkerBee bee, PlanFragment rootFragment, FragmentRoot rootOperator, UserClientConnection rootClient, List<PlanFragment> leafFragments) throws ExecutionSetupException{
+ remainingFragmentCount.set(leafFragments.size()+1);
+
+ // set up the root framgnet first so we'll have incoming buffers available.
+ {
+ IncomingBuffers buffers = new IncomingBuffers(rootOperator);
+
+ FragmentContext rootContext = new FragmentContext(bee.getContext(), rootFragment.getHandle(), rootClient, buffers);
+ RootExec rootExec = ImplCreator.getExec(rootContext, rootOperator);
+ // add fragment to local node.
+ map.put(rootFragment.getHandle(), new FragmentData(rootFragment.getHandle(), null, true));
+ rootRunner = new FragmentRunner(rootContext, rootExec, new RootFragmentManager(rootContext, rootFragment));
+ LocalFragmentHandler handler = new LocalFragmentHandler(rootFragment.getHandle(), buffers, rootRunner);
+ if(buffers.isDone()){
+ bee.addFragmentRunner(handler.getRunnable());
+ }else{
+ bee.getContext().getBitCom().registerIncomingBatchHandler(handler);
+ }
+
+ }
+
+ // send remote fragments.
+ for (PlanFragment f : leafFragments) {
+ sendRemoteFragment(f);
+ }
+
+ }
+
+ private void sendRemoteFragment(PlanFragment fragment){
+ map.put(fragment.getHandle(), new FragmentData(fragment.getHandle(), fragment.getAssignment(), false));
+ FragmentSubmitListener listener = new FragmentSubmitListener(fragment.getAssignment(), fragment);
+ tun.get(fragment.getAssignment()).sendFragment(fragment).addLightListener(listener);
+ }
+
+
+ @Override
+ public void statusUpdate(FragmentStatus status) {
+
+ switch(status.getState()){
+ case AWAITING_ALLOCATION:
+ updateStatus(status);
+ break;
+ case CANCELLED:
+ // we don't care about cancellation messages since we're the only entity that should drive cancellations.
+ break;
+ case FAILED:
+ fail(status);
+ break;
+ case FINISHED:
+ finished(status);
+ break;
+ case RUNNING:
+ updateStatus(status);
+ break;
+ default:
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private void updateStatus(FragmentStatus status){
+ map.get(status.getHandle()).setStatus(status);
+ }
+
+ private void finished(FragmentStatus status){
+ updateStatus(status);
+ int remaining = remainingFragmentCount.decrementAndGet();
+ if(remaining == 0){
+ QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.COMPLETED).build();
+ foreman.cleanupAndSendResult(result);
+ }
+ }
+
+ private void fail(FragmentStatus status){
+ updateStatus(status);
+ stopQuery();
+ QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.FAILED).build();
+ foreman.cleanupAndSendResult(result);
+ }
+
+
+ private void stopQuery(){
+ // Stop all queries with a currently active status.
+// for(FragmentData data: map.values()){
+// FragmentHandle handle = data.getStatus().getHandle();
+// switch(data.getStatus().getState()){
+// case SENDING:
+// case AWAITING_ALLOCATION:
+// case RUNNING:
+// if(data.isLocal()){
+// rootRunner.cancel();
+// }else{
+// tun.get(data.getEndpoint()).cancelFragment(handle).addLightListener(new CancelListener(data.endpoint, handle));
+// }
+// break;
+// default:
+// break;
+// }
+// }
+ }
+
+ public void cancel(){
+ stopQuery();
+ }
+
+ private class CancelListener extends EndpointListener<Ack, FragmentHandle>{
+
+ public CancelListener(DrillbitEndpoint endpoint, FragmentHandle handle) {
+ super(endpoint, handle);
+ }
+
+ @Override
+ public void failed(RpcException ex) {
+ logger.error("Failure while attempting to cancel fragment {} on endpoint {}.", value, endpoint, ex);
+ }
+
+ @Override
+ public void success(Ack value) {
+ if(!value.getOk()){
+ logger.warn("Remote node {} responded negative on cancellation request for fragment {}.", endpoint, value);
+ }
+ // do nothing.
+ }
+
+ };
+
+ public RpcOutcomeListener<Ack> getSubmitListener(DrillbitEndpoint endpoint, PlanFragment value){
+ return new FragmentSubmitListener(endpoint, value);
+ }
+
+
+
+ private class FragmentSubmitListener extends EndpointListener<Ack, PlanFragment>{
+
+ public FragmentSubmitListener(DrillbitEndpoint endpoint, PlanFragment value) {
+ super(endpoint, value);
+ }
+
+ @Override
+ public void failed(RpcException ex) {
+ stopQuery();
+ }
+
+ }
+
+
+ private class FragmentData{
+ private final boolean isLocal;
+ private volatile FragmentStatus status;
+ private volatile long lastStatusUpdate = 0;
+ private final DrillbitEndpoint endpoint;
+
+ public FragmentData(FragmentHandle handle, DrillbitEndpoint endpoint, boolean isLocal) {
+ super();
+ this.status = FragmentStatus.newBuilder().setHandle(handle).setState(FragmentState.SENDING).build();
+ this.endpoint = endpoint;
+ this.isLocal = isLocal;
+ }
+
+ public void setStatus(FragmentStatus status){
+ this.status = status;
+ lastStatusUpdate = System.currentTimeMillis();
+ }
+
+ public FragmentStatus getStatus() {
+ return status;
+ }
+
+ public boolean isLocal() {
+ return isLocal;
+ }
+
+ public long getLastStatusUpdate() {
+ return lastStatusUpdate;
+ }
+
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+
+ }
+
+ private class RootFragmentManager extends AbstractFragmentRunnerListener{
+
+ private RootFragmentManager(FragmentContext context, PlanFragment fragment){
+ super(context);
+ }
+
+ @Override
+ protected void statusChange(FragmentHandle handle, FragmentStatus status) {
+ RunningFragmentManager.this.updateStatus(status);
+ }
+
+
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
new file mode 100644
index 0000000..ad3534c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/TunnelManager.java
@@ -0,0 +1,53 @@
+/*******************************************************************************
+ * 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.drill.exec.work.foreman;
+
+import java.util.Map;
+
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.bit.BitCom;
+import org.apache.drill.exec.rpc.bit.BitTunnel;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Keeps a local list of tunnels associated with a particular Foreman.
+ */
+public class TunnelManager {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TunnelManager.class);
+
+ private final BitCom com;
+ private Map<DrillbitEndpoint, BitTunnel> tunnels = Maps.newHashMap();
+
+ public TunnelManager(BitCom com){
+ this.com = com;
+ }
+
+ public BitTunnel get(DrillbitEndpoint ep){
+ BitTunnel bt = tunnels.get(ep);
+ if(bt == null){
+ bt = com.getTunnel(ep);
+ tunnels.put(ep, bt);
+ }
+
+ return bt;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
new file mode 100644
index 0000000..b4e9308
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/work/fragment/IncomingFragmentHandler.java
@@ -0,0 +1,49 @@
+/*******************************************************************************
+ * 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.drill.exec.work.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.record.RawFragmentBatch;
+import org.apache.drill.exec.rpc.RemoteConnection.ConnectionThrottle;
+import org.apache.drill.exec.work.FragmentRunner;
+
+/**
+ * Handles incoming fragments as they arrive, routing them as apporpriate.
+ */
+public interface IncomingFragmentHandler {
+
+ /**
+ * Handle the next incoming fragment.
+ * @param throttle
+ * @param batch
+ * @return True if the fragment has enough incoming data to be able to be run.
+ * @throws FragmentSetupException
+ */
+ public abstract boolean handle(ConnectionThrottle throttle, RawFragmentBatch batch) throws FragmentSetupException;
+
+ /**
+ * Get the fragment runner for this incoming fragment. Note, this can only be requested once.
+ * @return
+ */
+ public abstract FragmentRunner getRunnable();
+
+ public abstract void cancel();
+ public boolean isDone();
+ public abstract FragmentHandle getHandle();
+}
\ No newline at end of file
[08/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
index 7b76810..94904a1 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ExecPlanner.java
@@ -17,10 +17,10 @@
******************************************************************************/
package org.apache.drill.exec.planner;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.work.QueryWorkUnit;
/**
@@ -30,5 +30,5 @@ import org.apache.drill.exec.ops.QueryContext;
public interface ExecPlanner {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExecPlanner.class);
- public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException;
+ public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws ExecutionSetupException;
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
deleted file mode 100644
index 51b0691..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentMaterializer.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.AbstractStore;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-public class FragmentMaterializer extends AbstractPhysicalVisitor<PhysicalOperator, FragmentMaterializer.IndexedFragmentNode, FragmentSetupException>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentMaterializer.class);
-
-
- @Override
- public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws FragmentSetupException {
- if(exchange == iNode.getNode().getSendingExchange()){
-
- // this is a sending exchange.
- PhysicalOperator child = exchange.getChild();
- return exchange.getSender(iNode.getMinorFragmentId(), child);
-
- }else{
- // receiving exchange.
- return exchange.getReceiver(iNode.getMinorFragmentId());
- }
- }
-
- @Override
- public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws FragmentSetupException {
- return scan.getSpecificScan(iNode.getMinorFragmentId());
- }
-
- @Override
- public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws FragmentSetupException {
- PhysicalOperator child = store.getChild();
- return store.getSpecificStore(child, iNode.getMinorFragmentId());
- }
-
- @Override
- public PhysicalOperator visitUnknown(PhysicalOperator op, IndexedFragmentNode iNode) throws FragmentSetupException {
- return op;
- }
-
- public static class IndexedFragmentNode{
- final FragmentWrapper info;
- final int minorFragmentId;
-
- public IndexedFragmentNode(int minorFragmentId, FragmentWrapper info) {
- super();
- this.info = info;
- this.minorFragmentId = minorFragmentId;
- }
-
- public FragmentNode getNode() {
- return info.getNode();
- }
-
- public int getMinorFragmentId() {
- return minorFragmentId;
- }
-
- public FragmentWrapper getInfo() {
- return info;
- }
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
deleted file mode 100644
index f53240e..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentNode.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-import com.google.common.collect.Lists;
-
-public class FragmentNode implements Iterable<FragmentNode.ExchangeFragmentPair>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentNode.class);
-
- private PhysicalOperator root;
- private Exchange sendingExchange;
- private final List<ExchangeFragmentPair> receivingExchangePairs = Lists.newLinkedList();
- private FragmentStats stats = new FragmentStats();
-
- public void addOperator(PhysicalOperator o){
- if(root == null){
- root = o;
- }
- }
-
- public void addSendExchange(Exchange e) throws FragmentSetupException{
- if(sendingExchange != null) throw new FragmentSetupException("Fragment was trying to add a second SendExchange. ");
- sendingExchange = e;
- }
-
- public void addReceiveExchange(Exchange e, FragmentNode fragment){
- this.receivingExchangePairs.add(new ExchangeFragmentPair(e, fragment));
- }
-
- @Override
- public Iterator<ExchangeFragmentPair> iterator() {
- return this.receivingExchangePairs.iterator();
- }
-
- public List<ExchangeFragmentPair> getReceivingExchangePairs() {
- return receivingExchangePairs;
- }
-
- public PhysicalOperator getRoot() {
- return root;
- }
-
- public Exchange getSendingExchange() {
- return sendingExchange;
- }
-
- public <T, V> T accept(FragmentVisitor<T, V> visitor, V extra){
- return visitor.visit(this, extra);
- }
-
- public FragmentStats getStats(){
- return stats;
- }
-
- public class ExchangeFragmentPair {
- private Exchange exchange;
- private FragmentNode node;
- public ExchangeFragmentPair(Exchange exchange, FragmentNode node) {
- super();
- this.exchange = exchange;
- this.node = node;
- }
- public Exchange getExchange() {
- return exchange;
- }
- public FragmentNode getNode() {
- return node;
- }
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + FragmentNode.this.hashCode();
- result = prime * result + ((exchange == null) ? 0 : exchange.hashCode());
- result = prime * result + ((node == null) ? 0 : node.hashCode());
- return result;
- }
-
- }
-
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + ((receivingExchangePairs == null) ? 0 : receivingExchangePairs.hashCode());
- result = prime * result + ((root == null) ? 0 : root.hashCode());
- result = prime * result + ((sendingExchange == null) ? 0 : sendingExchange.hashCode());
- result = prime * result + ((stats == null) ? 0 : stats.hashCode());
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj) return true;
- if (obj == null) return false;
- if (getClass() != obj.getClass()) return false;
- FragmentNode other = (FragmentNode) obj;
- if (receivingExchangePairs == null) {
- if (other.receivingExchangePairs != null) return false;
- } else if (!receivingExchangePairs.equals(other.receivingExchangePairs)) return false;
- if (root == null) {
- if (other.root != null) return false;
- } else if (!root.equals(other.root)) return false;
- if (sendingExchange == null) {
- if (other.sendingExchange != null) return false;
- } else if (!sendingExchange.equals(other.sendingExchange)) return false;
- if (stats == null) {
- if (other.stats != null) return false;
- } else if (!stats.equals(other.stats)) return false;
- return true;
- }
-
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
deleted file mode 100644
index 3f7c3a9..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentPlanningSet.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import java.util.Iterator;
-import java.util.Map;
-
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.ops.QueryContext;
-
-public class FragmentPlanningSet implements Iterable<FragmentWrapper>{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentPlanningSet.class);
-
- private Map<FragmentNode, FragmentWrapper> fragmentMap;
- private int majorFragmentIdIndex = 0;
- private QueryContext context;
-
- public FragmentPlanningSet(QueryContext context){
- this.context = context;
- }
-
- public void setStats(FragmentNode node, FragmentStats stats){
- get(node).setStats(stats);
- }
-
- public void addAffinity(FragmentNode n, DrillbitEndpoint endpoint, float affinity){
- get(n).addEndpointAffinity(endpoint, affinity);
- }
-
- public void setWidth(FragmentNode n, int width){
- get(n).setWidth(width);
- }
-
- private FragmentWrapper get(FragmentNode node){
- FragmentWrapper info = fragmentMap.get(node);
- if(info == null) info = new FragmentWrapper(node, majorFragmentIdIndex++);
- return info;
- }
-
- @Override
- public Iterator<FragmentWrapper> iterator() {
- return this.fragmentMap.values().iterator();
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
deleted file mode 100644
index d551aa4..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentRunnable.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import io.netty.buffer.ByteBuf;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.CancelableQuery;
-import org.apache.drill.exec.foreman.StatusProvider;
-import org.apache.drill.exec.metrics.SingleThreadNestedCounter;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.FragmentConverter;
-import org.apache.drill.exec.ops.FragmentRoot;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus;
-import org.apache.drill.exec.proto.ExecProtos.FragmentStatus.FragmentState;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-import org.apache.drill.exec.server.DrillbitContext;
-
-import com.yammer.metrics.MetricRegistry;
-import com.yammer.metrics.Timer;
-
-/**
- * Responsible for running a single fragment on a single Drillbit. Listens/responds to status request and cancellation
- * messages.
- */
-public class FragmentRunnable implements Runnable, CancelableQuery, StatusProvider {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentRunnable.class);
-
- private final AtomicInteger state = new AtomicInteger(FragmentState.AWAITING_ALLOCATION_VALUE);
- private final FragmentRoot root;
- private final FragmentContext context;
-
- public FragmentRunnable(DrillbitContext dbContext, long fragmentId) throws FragmentSetupException {
- PlanFragment fragment = dbContext.getCache().getFragment(fragmentId);
- if (fragment == null) throw new FragmentSetupException(String.format("The provided fragment id [%d] was unknown.", fragmentId));
- this.context = new FragmentContext(dbContext, fragment);
- this.root = FragmentConverter.getFragment(this.context);
- }
-
- @Override
- public FragmentStatus getStatus() {
- return FragmentStatus.newBuilder() //
- .setBatchesCompleted(context.batchesCompleted.get()) //
- .setDataProcessed(context.dataProcessed.get()) //
- .setMemoryUse(context.getAllocator().getAllocatedMemory()) //
- .build();
- }
-
- @Override
- public boolean cancel(long queryId) {
- if (context.getFragment().getQueryId() == queryId) {
- state.set(FragmentState.CANCELLED_VALUE);
- return true;
- }
- return false;
- }
-
- private void fail(Throwable cause){
- context.fail(cause);
- state.set(FragmentState.FAILED_VALUE);
- }
-
- @Override
- public void run() {
- if(!updateState(FragmentState.AWAITING_ALLOCATION, FragmentState.RUNNING, false)){
- fail(new RuntimeException(String.format("Run was called when fragment was in %s state. FragmentRunnables should only be started when they are currently in awaiting allocation state.", FragmentState.valueOf(state.get()))));
- return;
- }
-
- Timer.Context t = context.fragmentTime.time();
-
- // setup the query.
- try{
- root.setup();
- }catch(FragmentSetupException e){
-
- context.fail(e);
- return;
- }
-
- // run the query.
- try{
- while(state.get() == FragmentState.RUNNING_VALUE){
- if(!root.next()){
- updateState(FragmentState.RUNNING, FragmentState.FINISHED, false);
- }
- }
- t.stop();
- }catch(Exception ex){
- fail(ex);
- }
-
- }
-
- private boolean updateState(FragmentState current, FragmentState update, boolean exceptionOnFailure) {
- boolean success = state.compareAndSet(current.getNumber(), update.getNumber());
- if (!success && exceptionOnFailure) {
- context.fail(new RuntimeException(String.format(
- "State was different than expected. Attempting to update state from %s to %s however current state was %s.",
- current.name(), update.name(), FragmentState.valueOf(state.get()))));
- return false;
- }
-
- return true;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
deleted file mode 100644
index 168072a..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentScheduler.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.exec.foreman.ResourceRequest;
-import org.apache.drill.exec.foreman.ResourceRequest.ResourceAllocation;
-
-import com.google.common.util.concurrent.ListenableFutureTask;
-
-public class FragmentScheduler {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentScheduler.class);
-
- public void getRunningResources(ResourceRequest resources, Runnable listener){
- // request the resource.
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
deleted file mode 100644
index 512b5d0..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStats.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.OperatorCost;
-
-public class FragmentStats {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStats.class);
-
- private int maxWidth = Integer.MAX_VALUE;
- private float networkCost;
- private float diskCost;
- private float memoryCost;
- private float cpuCost;
-
- public void addMaxWidth(int width){
- maxWidth = Math.min(maxWidth, width);
- }
-
- public void addCost(OperatorCost cost){
- networkCost += cost.getNetwork();
- diskCost += cost.getDisk();
- memoryCost += cost.getMemory();
- cpuCost += cost.getCpu();
- }
-
- public int getMaxWidth() {
- return maxWidth;
- }
-
- public float getNetworkCost() {
- return networkCost;
- }
-
- public float getDiskCost() {
- return diskCost;
- }
-
- public float getMemoryCost() {
- return memoryCost;
- }
-
- public float getCpuCost() {
- return cpuCost;
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
deleted file mode 100644
index a0dcde3..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentStatsCollector.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Scan;
-import org.apache.drill.common.physical.pop.base.Store;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
-
-import com.google.common.base.Preconditions;
-
-public class FragmentStatsCollector implements FragmentVisitor<Void, FragmentStats> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentStatsCollector.class);
-
- //private HashMap<FragmentNode, FragmentStats> nodeStats = Maps.newHashMap();
- private final StatsCollector opCollector = new StatsCollector();
- private final FragmentPlanningSet planningSet;
-
- public FragmentStatsCollector(FragmentPlanningSet planningSet){
- this.planningSet = planningSet;
- }
-
- @Override
- public Void visit(FragmentNode n, FragmentStats stats) {
- Preconditions.checkNotNull(stats);
- Preconditions.checkNotNull(n);
-
- n.getRoot().accept(opCollector, stats);
-
- // sending exchange.
- Exchange sending = n.getSendingExchange();
- if (sending != null) {
- stats.addCost(sending.getAggregateSendCost());
- stats.addMaxWidth(sending.getMaxSendWidth());
- }
-
- // receivers...
- for (ExchangeFragmentPair child : n) {
- // add exchange receive cost.
- Exchange receivingExchange = child.getExchange();
- stats.addCost(receivingExchange.getAggregateReceiveCost());
-
- FragmentStats childStats = new FragmentStats();
- FragmentNode childNode = child.getNode();
- childNode.accept(this, childStats);
- }
-
- // store the stats for later use.
- planningSet.setStats(n, stats);
-
- return null;
- }
-
- public void collectStats(FragmentNode rootFragment) {
- FragmentStats s = new FragmentStats();
- rootFragment.accept(this, s);
- }
-
- private class StatsCollector extends AbstractPhysicalVisitor<Void, FragmentStats, RuntimeException> {
-
- @Override
- public Void visitExchange(Exchange exchange, FragmentStats stats) throws RuntimeException {
- // don't do anything here since we'll add the exchange costs elsewhere. We also don't want navigate across
- // exchanges since they are separate fragments.
- return null;
- }
-
- @Override
- public Void visitScan(Scan<?> scan, FragmentStats stats) {
- stats.addMaxWidth(scan.getReadEntries().size());
- return super.visitScan(scan, stats);
- }
-
- @Override
- public Void visitStore(Store store, FragmentStats stats) {
- stats.addMaxWidth(store.getMaxWidth());
- return super.visitStore(store, stats);
- }
-
- @Override
- public Void visitUnknown(PhysicalOperator op, FragmentStats stats) {
- stats.addCost(op.getCost());
- for (PhysicalOperator child : op) {
- child.accept(this, stats);
- }
- return null;
- }
-
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
deleted file mode 100644
index 12d2b9f..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentVisitor.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-public interface FragmentVisitor<T, V> {
- public T visit(FragmentNode n, V extra);
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
deleted file mode 100644
index a1e4f81..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentWrapper.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ThreadLocalRandom;
-
-import org.apache.commons.lang.NotImplementedException;
-import org.apache.drill.common.physical.EndpointAffinity;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.planner.FragmentNode.ExchangeFragmentPair;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-public class FragmentWrapper {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWrapper.class);
-
- private final FragmentNode node;
- private final int majorFragmentId;
- private int width = -1;
- private FragmentStats stats;
- private boolean endpointsAssigned;
- private Map<DrillbitEndpoint, EndpointAffinity> endpointAffinity = Maps.newHashMap();
-
- // a list of assigned endpoints. Technically, there could repeated endpoints in this list if we'd like to assign the
- // same fragment multiple times to the same endpoint.
- private List<DrillbitEndpoint> endpoints = Lists.newLinkedList();
-
- public FragmentWrapper(FragmentNode node, int majorFragmentId) {
- this.majorFragmentId = majorFragmentId;
- this.node = node;
- }
-
- public FragmentStats getStats() {
- return stats;
- }
-
- public void setStats(FragmentStats stats) {
- this.stats = stats;
- }
-
- public void addEndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
- Preconditions.checkState(!endpointsAssigned);
- EndpointAffinity ea = endpointAffinity.get(endpoint);
- if (ea == null) {
- ea = new EndpointAffinity(endpoint);
- endpointAffinity.put(endpoint, ea);
- }
-
- ea.addAffinity(affinity);
- endpointAffinity.put(endpoint, ea);
- }
-
- public int getMajorFragmentId() {
- return majorFragmentId;
- }
-
- public int getWidth() {
- return width;
- }
-
- public void setWidth(int width) {
- Preconditions.checkState(width == -1);
- this.width = width;
- }
-
- public FragmentNode getNode() {
- return node;
- }
-
- public void assignEndpoints(Collection<DrillbitEndpoint> allPossible) {
- Preconditions.checkState(!endpointsAssigned);
-
- endpointsAssigned = true;
-
- List<EndpointAffinity> values = Lists.newArrayList();
- values.addAll(endpointAffinity.values());
-
- if(values.size() == 0){
- final int div = allPossible.size();
- int start = ThreadLocalRandom.current().nextInt(div);
- // round robin with random start.
- for(int i = start; i < start + width; i++){
- endpoints.add(values.get(i % div).getEndpoint());
- }
- }else if(values.size() < width){
- throw new NotImplementedException("Haven't implemented a scenario where we have some node affinity but the affinity list is smaller than the expected width.");
- }else{
- // get nodes with highest affinity.
- Collections.sort(values);
- values = Lists.reverse(values);
- for (int i = 0; i < width; i++) {
- endpoints.add(values.get(i).getEndpoint());
- }
- }
-
- node.getSendingExchange().setupSenders(endpoints);
- for (ExchangeFragmentPair e : node.getReceivingExchangePairs()) {
- e.getExchange().setupReceivers(endpoints);
- }
- }
-
- public DrillbitEndpoint getAssignedEndpoint(int minorFragmentId) {
- Preconditions.checkState(endpointsAssigned);
- return this.endpoints.get(minorFragmentId);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
deleted file mode 100644
index 5f67617..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/FragmentingPhysicalVisitor.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.exec.exception.FragmentSetupException;
-
-/**
- * Responsible for breaking a plan into its constituent Fragments.
- */
-public class FragmentingPhysicalVisitor extends AbstractPhysicalVisitor<FragmentNode, FragmentNode, FragmentSetupException> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentingPhysicalVisitor.class);
-
- private FragmentNode rootFragment = new FragmentNode();
-
- public FragmentingPhysicalVisitor(){
- }
-
-
- @Override
- public FragmentNode visitExchange(Exchange exchange, FragmentNode value) throws FragmentSetupException {
-// logger.debug("Visiting Exchange {}", exchange);
- if(value == null) throw new FragmentSetupException("The simple fragmenter was called without a FragmentBuilder value. This will only happen if the initial call to SimpleFragmenter is by a Exchange node. This should never happen since an Exchange node should never be the root node of a plan.");
- FragmentNode next = getNextBuilder();
- value.addReceiveExchange(exchange, next);
- next.addSendExchange(exchange);
- exchange.getChild().accept(this, getNextBuilder());
- return value;
- }
-
- @Override
- public FragmentNode visitUnknown(PhysicalOperator op, FragmentNode value) throws FragmentSetupException{
-// logger.debug("Visiting Other {}", op);
- value = ensureBuilder(value);
- value.addOperator(op);
- for(PhysicalOperator child : op){
- child.accept(this, value);
- }
- return value;
- }
-
- private FragmentNode ensureBuilder(FragmentNode value) throws FragmentSetupException{
- if(value != null){
- return value;
- }else{
- return rootFragment;
- }
- }
-
- public FragmentNode getNextBuilder(){
- return new FragmentNode();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
deleted file mode 100644
index d1c85cb..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/MaterializedFragment.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.OperatorCost;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-public class MaterializedFragment {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaterializedFragment.class);
-
- final PhysicalOperator root;
- final DrillbitEndpoint endpoint;
- final long queryId;
- final int majorFragmentId;
- final int minorFragmentId;
- final OperatorCost cost;
-
- public MaterializedFragment(PhysicalOperator root, DrillbitEndpoint endpoint, long queryId, int majorFragmentId,
- int minorFragmentId, OperatorCost cost) {
- super();
- this.root = root;
- this.endpoint = endpoint;
- this.queryId = queryId;
- this.majorFragmentId = majorFragmentId;
- this.minorFragmentId = minorFragmentId;
- this.cost = cost;
- }
-
- public PhysicalOperator getRoot() {
- return root;
- }
-
- public DrillbitEndpoint getEndpoint() {
- return endpoint;
- }
-
- public long getQueryId() {
- return queryId;
- }
-
- public int getMajorFragmentId() {
- return majorFragmentId;
- }
-
- public int getMinorFragmentId() {
- return minorFragmentId;
- }
-
- public OperatorCost getCost() {
- return cost;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
index ff31cd9..3c2df61 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/PhysicalPlanReader.java
@@ -19,29 +19,74 @@ package org.apache.drill.exec.planner;
import java.io.IOException;
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.StorageEngineRegistry;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.exec.coord.DrillbitEndpointSerDe;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.FragmentLeaf;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalOperatorUtil;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.record.MajorTypeSerDe;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectReader;
+import com.fasterxml.jackson.databind.module.SimpleModule;
public class PhysicalPlanReader {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanReader.class);
- private final ObjectReader reader;
+ private final ObjectReader physicalPlanReader;
+ private final ObjectMapper mapper;
+ private final ObjectReader operatorReader;
+ private final ObjectReader logicalPlanReader;
- public PhysicalPlanReader(ObjectMapper mapper, DrillbitEndpoint endpoint) {
+ public PhysicalPlanReader(DrillConfig config, ObjectMapper mapper, final DrillbitEndpoint endpoint) {
+
+ // Endpoint serializer/deserializer.
+ SimpleModule deserModule = new SimpleModule("PhysicalOperatorModule") //
+ .addSerializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.Se()) //
+ .addDeserializer(DrillbitEndpoint.class, new DrillbitEndpointSerDe.De()) //
+ .addSerializer(MajorType.class, new MajorTypeSerDe.Se())
+ .addDeserializer(MajorType.class, new MajorTypeSerDe.De());
+
+
+ mapper.registerModule(deserModule);
+ mapper.registerSubtypes(PhysicalOperatorUtil.getSubTypes(config));
InjectableValues injectables = new InjectableValues.Std() //
.addValue(DrillbitEndpoint.class, endpoint); //
- this.reader = mapper.reader(PhysicalPlan.class).with(injectables);
+
+ this.mapper = mapper;
+ this.physicalPlanReader = mapper.reader(PhysicalPlan.class).with(injectables);
+ this.operatorReader = mapper.reader(PhysicalOperator.class).with(injectables);
+ this.logicalPlanReader = mapper.reader(LogicalPlan.class).with(injectables);
+ }
+
+ public String writeJson(PhysicalOperator op) throws JsonProcessingException{
+ return mapper.writeValueAsString(op);
+ }
+
+ public PhysicalPlan readPhysicalPlan(String json) throws JsonProcessingException, IOException {
+ logger.debug("Reading physical plan {}", json);
+ return physicalPlanReader.readValue(json);
}
- public PhysicalPlan read(String json) throws JsonProcessingException, IOException {
- return reader.readValue(json);
+ public FragmentRoot readFragmentOperator(String json) throws JsonProcessingException, IOException {
+ logger.debug("Attempting to read {}", json);
+ PhysicalOperator op = operatorReader.readValue(json);
+ if(op instanceof FragmentLeaf){
+ return (FragmentRoot) op;
+ }else{
+ throw new UnsupportedOperationException(String.format("The provided json fragment doesn't have a FragmentRoot as its root operator. The operator was %s.", op.getClass().getCanonicalName()));
+ }
}
+ public LogicalPlan readLogicalPlan(String json) throws JsonProcessingException, IOException{
+ logger.debug("Reading logical plan {}", json);
+ return logicalPlanReader.readValue(json);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
deleted file mode 100644
index ff81d90..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/ScanFinder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import org.apache.drill.common.physical.pop.base.AbstractPhysicalVisitor;
-import org.apache.drill.common.physical.pop.base.Exchange;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.Store;
-
-public class ScanFinder extends AbstractPhysicalVisitor<Boolean, Void, RuntimeException> {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanFinder.class);
-
- private static final ScanFinder finder = new ScanFinder();
-
- private ScanFinder(){}
-
- @Override
- public Boolean visitExchange(Exchange exchange, Void value) throws RuntimeException {
- return false;
- }
-
- @Override
- public Boolean visitStore(Store store, Void value) throws RuntimeException {
- return true;
- }
-
- @Override
- public Boolean visitUnknown(PhysicalOperator op, Void value) throws RuntimeException {
- for(PhysicalOperator child : op){
- if(child.accept(this, null)) return true;
- }
- return false;
- }
-
- public static boolean containsScan(PhysicalOperator op){
- return op.accept(finder, null);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
index d1c3add..824b2e2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleExecPlanner.java
@@ -17,14 +17,16 @@
******************************************************************************/
package org.apache.drill.exec.planner;
-import java.util.List;
-
-import org.apache.drill.common.physical.PhysicalPlan;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.fragment.Fragment;
+import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
+import org.apache.drill.exec.planner.fragment.PlanningSet;
+import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
+import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.work.QueryWorkUnit;
/**
* Parallelization is based on available nodes with source or target data. Nodes that are "overloaded" are excluded from execution.
@@ -32,22 +34,20 @@ import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
public class SimpleExecPlanner implements ExecPlanner{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleExecPlanner.class);
- private FragmentingPhysicalVisitor fragmenter = new FragmentingPhysicalVisitor();
+ private MakeFragmentsVisitor fragmenter = new MakeFragmentsVisitor();
private SimpleParallelizer parallelizer = new SimpleParallelizer();
@Override
- public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws FragmentSetupException {
+ public QueryWorkUnit getWorkUnit(QueryContext context, PhysicalPlan plan, int maxWidth) throws ExecutionSetupException {
// get the root physical operator and split the plan into sub fragments.
PhysicalOperator root = plan.getSortedOperators(false).iterator().next();
- FragmentNode fragmentRoot = root.accept(fragmenter, null);
+ Fragment fragmentRoot = root.accept(fragmenter, null);
// generate a planning set and collect stats.
- FragmentPlanningSet planningSet = new FragmentPlanningSet(context);
- FragmentStatsCollector statsCollector = new FragmentStatsCollector(planningSet);
- statsCollector.collectStats(fragmentRoot);
+ PlanningSet planningSet = StatsCollector.collectStats(fragmentRoot);
- return parallelizer.getFragments(context, fragmentRoot, planningSet, maxWidth);
+ return parallelizer.getFragments(context.getCurrentEndpoint(), context.getQueryId(), context.getActiveEndpoints(), context.getPlanReader(), fragmentRoot, planningSet, maxWidth);
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
deleted file mode 100644
index a52abaa..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/SimpleParallelizer.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.planner;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.exception.FragmentSetupException;
-import org.apache.drill.exec.foreman.QueryWorkUnit;
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.planner.FragmentMaterializer.IndexedFragmentNode;
-import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
-
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.Lists;
-
-public class SimpleParallelizer {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
-
- private final FragmentMaterializer materializer = new FragmentMaterializer();
-
- /**
- * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
- * beyond the global max width.
- *
- * @param context
- * The current QueryContext.
- * @param planningSet
- * The set of queries with collected statistics that we'll work with.
- * @param globalMaxWidth
- * The maximum level or paralellization any stage of the query can do. Note that while this might be the
- * number of active Drillbits, realistically, this could be well beyond that number of we want to do things
- * like speed results return.
- * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
- * @throws FragmentSetupException
- */
- public QueryWorkUnit getFragments(QueryContext context, FragmentNode rootNode, FragmentPlanningSet planningSet,
- int globalMaxWidth) throws FragmentSetupException {
- assignEndpoints(context.getActiveEndpoints(), planningSet, globalMaxWidth);
- return generateWorkUnit(context.getQueryId(), context.getMapper(), rootNode, planningSet);
- }
-
- private QueryWorkUnit generateWorkUnit(long queryId, ObjectMapper mapper, FragmentNode rootNode,
- FragmentPlanningSet planningSet) throws FragmentSetupException {
-
- List<PlanFragment> fragments = Lists.newArrayList();
-
- PlanFragment rootFragment = null;
-
- // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
- // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
- for (FragmentWrapper info : planningSet) {
-
- FragmentNode node = info.getNode();
- FragmentStats stats = node.getStats();
- PhysicalOperator abstractRoot = node.getRoot();
- boolean isRootNode = rootNode == node;
-
- if (isRootNode && info.getWidth() != 1)
- throw new FragmentSetupException(
- String
- .format(
- "Failure while trying to setup fragment. The root fragment must always have parallelization one. In the current case, the width was set to %d.",
- info.getWidth()));
- // a fragment is self driven if it doesn't rely on any other exchanges.
- boolean selfDriven = node.getReceivingExchangePairs().size() == 0;
-
- // Create a minorFragment for each major fragment.
- for (int minorFragmentId = 0; minorFragmentId < info.getWidth(); minorFragmentId++) {
- IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, info);
- PhysicalOperator root = abstractRoot.accept(materializer, iNode);
-
- // get plan as JSON
- String plan;
- try {
- plan = mapper.writeValueAsString(root);
- } catch (JsonProcessingException e) {
- throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
- }
-
- PlanFragment fragment = PlanFragment.newBuilder() //
- .setCpuCost(stats.getCpuCost()) //
- .setDiskCost(stats.getDiskCost()) //
- .setMemoryCost(stats.getMemoryCost()) //
- .setNetworkCost(stats.getNetworkCost()) //
- .setFragmentJson(plan) //
- .setMinorFragmentId(minorFragmentId) //
- .setMajorFragmentId(info.getMajorFragmentId()).setQueryId(queryId) //
- .setAssignment(info.getAssignedEndpoint(minorFragmentId)).setSelfDriven(selfDriven).build();
-
- if (isRootNode) {
- rootFragment = fragment;
- } else {
- fragments.add(fragment);
- }
- }
- }
-
- return new QueryWorkUnit(rootFragment, fragments);
-
- }
-
- private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, FragmentPlanningSet planningSet,
- int globalMaxWidth) {
- // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
- // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
- // could be based on endpoint load)
- for (FragmentWrapper info : planningSet) {
-
- FragmentStats stats = info.getStats();
-
- // figure out width.
- int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
- float diskCost = stats.getDiskCost();
-
- // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
- // of tasks or the maximum width of the fragment.
- if (diskCost < width) {
- width = (int) diskCost;
- }
-
- if (width < 1) width = 1;
- info.setWidth(width);
-
- // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
- info.assignEndpoints(allNodes);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
new file mode 100644
index 0000000..3284086
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Fragment.java
@@ -0,0 +1,150 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Lists;
+
+public class Fragment implements Iterable<Fragment.ExchangeFragmentPair>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Fragment.class);
+
+ private PhysicalOperator root;
+ private Exchange sendingExchange;
+ private final List<ExchangeFragmentPair> receivingExchangePairs = Lists.newLinkedList();
+ private Stats stats = new Stats();
+
+ public void addOperator(PhysicalOperator o){
+ if(root == null){
+ root = o;
+ }
+ }
+
+ public void addSendExchange(Exchange e) throws FragmentSetupException{
+ if(sendingExchange != null) throw new FragmentSetupException("Fragment was trying to add a second SendExchange. ");
+ addOperator(e);
+ sendingExchange = e;
+ }
+
+ public void addReceiveExchange(Exchange e, Fragment fragment){
+ this.receivingExchangePairs.add(new ExchangeFragmentPair(e, fragment));
+ }
+
+ @Override
+ public Iterator<ExchangeFragmentPair> iterator() {
+ return this.receivingExchangePairs.iterator();
+ }
+
+ public List<ExchangeFragmentPair> getReceivingExchangePairs() {
+ return receivingExchangePairs;
+ }
+
+ public PhysicalOperator getRoot() {
+ return root;
+ }
+
+ public Exchange getSendingExchange() {
+ return sendingExchange;
+ }
+
+// public <T, V> T accept(FragmentVisitor<T, V> visitor, V extra){
+// return visitor.visit(this, extra);
+// }
+
+ public Stats getStats(){
+ return stats;
+ }
+
+ public class ExchangeFragmentPair {
+ private Exchange exchange;
+ private Fragment node;
+ public ExchangeFragmentPair(Exchange exchange, Fragment node) {
+ super();
+ this.exchange = exchange;
+ this.node = node;
+ }
+ public Exchange getExchange() {
+ return exchange;
+ }
+ public Fragment getNode() {
+ return node;
+ }
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((exchange == null) ? 0 : exchange.hashCode());
+ result = prime * result + ((node == null) ? 0 : node.hashCode());
+ return result;
+ }
+ @Override
+ public String toString() {
+ return "ExchangeFragmentPair [exchange=" + exchange + "]";
+ }
+
+
+
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((receivingExchangePairs == null) ? 0 : receivingExchangePairs.hashCode());
+ result = prime * result + ((root == null) ? 0 : root.hashCode());
+ result = prime * result + ((sendingExchange == null) ? 0 : sendingExchange.hashCode());
+ result = prime * result + ((stats == null) ? 0 : stats.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) return true;
+ if (obj == null) return false;
+ if (getClass() != obj.getClass()) return false;
+ Fragment other = (Fragment) obj;
+ if (receivingExchangePairs == null) {
+ if (other.receivingExchangePairs != null) return false;
+ } else if (!receivingExchangePairs.equals(other.receivingExchangePairs)) return false;
+ if (root == null) {
+ if (other.root != null) return false;
+ } else if (!root.equals(other.root)) return false;
+ if (sendingExchange == null) {
+ if (other.sendingExchange != null) return false;
+ } else if (!sendingExchange.equals(other.sendingExchange)) return false;
+ if (stats == null) {
+ if (other.stats != null) return false;
+ } else if (!stats.equals(other.stats)) return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "FragmentNode [root=" + root + ", sendingExchange=" + sendingExchange + ", receivingExchangePairs="
+ + receivingExchangePairs + ", stats=" + stats + "]";
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
new file mode 100644
index 0000000..0cba2ec
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/FragmentVisitor.java
@@ -0,0 +1,23 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+
+public interface FragmentVisitor<T, V> {
+ public T visit(Fragment n, V extra);
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
new file mode 100644
index 0000000..4188435
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MakeFragmentsVisitor.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+
+/**
+ * Responsible for breaking a plan into its constituent Fragments.
+ */
+public class MakeFragmentsVisitor extends AbstractPhysicalVisitor<Fragment, Fragment, FragmentSetupException> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MakeFragmentsVisitor.class);
+
+
+ public MakeFragmentsVisitor(){
+ }
+
+ @Override
+ public Fragment visitExchange(Exchange exchange, Fragment value) throws FragmentSetupException {
+// logger.debug("Visiting Exchange {}", exchange);
+ if(value == null) throw new FragmentSetupException("The simple fragmenter was called without a FragmentBuilder value. This will only happen if the initial call to SimpleFragmenter is by a Exchange node. This should never happen since an Exchange node should never be the root node of a plan.");
+ Fragment next = getNextBuilder();
+ value.addReceiveExchange(exchange, next);
+ next.addSendExchange(exchange);
+ exchange.getChild().accept(this, next);
+ return value;
+ }
+
+ @Override
+ public Fragment visitOp(PhysicalOperator op, Fragment value) throws FragmentSetupException{
+// logger.debug("Visiting Other {}", op);
+ value = ensureBuilder(value);
+ value.addOperator(op);
+ for(PhysicalOperator child : op){
+ child.accept(this, value);
+ }
+ return value;
+ }
+
+ private Fragment ensureBuilder(Fragment value) throws FragmentSetupException{
+ if(value != null){
+ return value;
+ }else{
+ return getNextBuilder();
+ }
+ }
+
+ public Fragment getNextBuilder(){
+ return new Fragment();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
new file mode 100644
index 0000000..e292cd2
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/MaterializedFragment.java
@@ -0,0 +1,69 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+public class MaterializedFragment {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MaterializedFragment.class);
+
+ final PhysicalOperator root;
+ final DrillbitEndpoint endpoint;
+ final long queryId;
+ final int majorFragmentId;
+ final int minorFragmentId;
+ final OperatorCost cost;
+
+ public MaterializedFragment(PhysicalOperator root, DrillbitEndpoint endpoint, long queryId, int majorFragmentId,
+ int minorFragmentId, OperatorCost cost) {
+ super();
+ this.root = root;
+ this.endpoint = endpoint;
+ this.queryId = queryId;
+ this.majorFragmentId = majorFragmentId;
+ this.minorFragmentId = minorFragmentId;
+ this.cost = cost;
+ }
+
+ public PhysicalOperator getRoot() {
+ return root;
+ }
+
+ public DrillbitEndpoint getEndpoint() {
+ return endpoint;
+ }
+
+ public long getQueryId() {
+ return queryId;
+ }
+
+ public int getMajorFragmentId() {
+ return majorFragmentId;
+ }
+
+ public int getMinorFragmentId() {
+ return minorFragmentId;
+ }
+
+ public OperatorCost getCost() {
+ return cost;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
new file mode 100644
index 0000000..9fee586
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Materializer.java
@@ -0,0 +1,107 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+
+import com.google.common.collect.Lists;
+
+public class Materializer extends AbstractPhysicalVisitor<PhysicalOperator, Materializer.IndexedFragmentNode, ExecutionSetupException>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Materializer.class);
+
+
+ @Override
+ public PhysicalOperator visitExchange(Exchange exchange, IndexedFragmentNode iNode) throws ExecutionSetupException {
+ if(exchange == iNode.getNode().getSendingExchange()){
+
+ // this is a sending exchange.
+ PhysicalOperator child = exchange.getChild().accept(this, iNode);
+ PhysicalOperator materializedSender = exchange.getSender(iNode.getMinorFragmentId(), child);
+ logger.debug("Visit sending exchange, materialized {} with child {}.", materializedSender, child);
+ return materializedSender;
+
+ }else{
+ // receiving exchange.
+ PhysicalOperator materializedReceiver = exchange.getReceiver(iNode.getMinorFragmentId());
+ logger.debug("Visit receiving exchange, materialized receiver: {}.", materializedReceiver);
+ return materializedReceiver;
+ }
+ }
+
+ @Override
+ public PhysicalOperator visitScan(Scan<?> scan, IndexedFragmentNode iNode) throws ExecutionSetupException {
+ return scan.getSpecificScan(iNode.getMinorFragmentId());
+ }
+
+ @Override
+ public PhysicalOperator visitStore(Store store, IndexedFragmentNode iNode) throws ExecutionSetupException {
+ PhysicalOperator child = store.getChild().accept(this, iNode);
+
+ try {
+ PhysicalOperator o = store.getSpecificStore(child, iNode.getMinorFragmentId());
+ logger.debug("New materialized store node {} with child {}", o, child);
+ return o;
+ } catch (PhysicalOperatorSetupException e) {
+ throw new FragmentSetupException("Failure while generating a specific Store materialization.");
+ }
+ }
+
+ @Override
+ public PhysicalOperator visitOp(PhysicalOperator op, IndexedFragmentNode iNode) throws ExecutionSetupException {
+ logger.debug("Visiting catch all: {}", op);
+ List<PhysicalOperator> children = Lists.newArrayList();
+ for(PhysicalOperator child : op){
+ children.add(child.accept(this, iNode));
+ }
+ return op.getNewWithChildren(children);
+ }
+
+ public static class IndexedFragmentNode{
+ final Wrapper info;
+ final int minorFragmentId;
+
+ public IndexedFragmentNode(int minorFragmentId, Wrapper info) {
+ super();
+ this.info = info;
+ this.minorFragmentId = minorFragmentId;
+ }
+
+ public Fragment getNode() {
+ return info.getNode();
+ }
+
+ public int getMinorFragmentId() {
+ return minorFragmentId;
+ }
+
+ public Wrapper getInfo() {
+ return info;
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
new file mode 100644
index 0000000..fb04255
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/PlanningSet.java
@@ -0,0 +1,66 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.collect.Maps;
+
+public class PlanningSet implements Iterable<Wrapper>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningSet.class);
+
+ private Map<Fragment, Wrapper> fragmentMap = Maps.newHashMap();
+ private int majorFragmentIdIndex = 0;
+
+ PlanningSet(){
+ }
+
+ public void addAffinity(Fragment n, DrillbitEndpoint endpoint, float affinity){
+ get(n).addEndpointAffinity(endpoint, affinity);
+ }
+
+ public void setWidth(Fragment n, int width){
+ get(n).setWidth(width);
+ }
+
+ Wrapper get(Fragment node){
+ Wrapper wrapper = fragmentMap.get(node);
+ if(wrapper == null){
+ wrapper = new Wrapper(node, majorFragmentIdIndex++);
+ fragmentMap.put(node, wrapper);
+ }
+ return wrapper;
+ }
+
+ @Override
+ public Iterator<Wrapper> iterator() {
+ return this.fragmentMap.values().iterator();
+ }
+
+ @Override
+ public String toString() {
+ return "FragmentPlanningSet:\n" + fragmentMap.values() + "]";
+ }
+
+
+
+}
[07/13] Update typing system. Update RPC system. Add Fragmenting
Implementation. Working single node. Distributed failing due to threading
issues.
Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
new file mode 100644
index 0000000..fc03a23
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/SimpleParallelizer.java
@@ -0,0 +1,163 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.exception.FragmentSetupException;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.planner.fragment.Materializer.IndexedFragmentNode;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.PlanFragment;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.work.QueryWorkUnit;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class SimpleParallelizer {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleParallelizer.class);
+
+ private final Materializer materializer = new Materializer();
+
+ /**
+ * Generate a set of assigned fragments based on the provided planningSet. Do not allow parallelization stages to go
+ * beyond the global max width.
+ *
+ * @param context
+ * The current QueryContext.
+ * @param planningSet
+ * The set of queries with collected statistics that we'll work with.
+ * @param globalMaxWidth
+ * The maximum level or paralellization any stage of the query can do. Note that while this might be the
+ * number of active Drillbits, realistically, this could be well beyond that number of we want to do things
+ * like speed results return.
+ * @return The list of generatoe PlanFragment protobuf objects to be assigned out to the individual nodes.
+ * @throws FragmentSetupException
+ */
+ public QueryWorkUnit getFragments(DrillbitEndpoint foremanNode, QueryId queryId, Collection<DrillbitEndpoint> activeEndpoints, PhysicalPlanReader reader, Fragment rootNode, PlanningSet planningSet,
+ int globalMaxWidth) throws ExecutionSetupException {
+ assignEndpoints(activeEndpoints, planningSet, globalMaxWidth);
+ return generateWorkUnit(foremanNode, queryId, reader, rootNode, planningSet);
+ }
+
+ private QueryWorkUnit generateWorkUnit(DrillbitEndpoint foremanNode, QueryId queryId, PhysicalPlanReader reader, Fragment rootNode,
+ PlanningSet planningSet) throws ExecutionSetupException {
+
+ List<PlanFragment> fragments = Lists.newArrayList();
+
+ PlanFragment rootFragment = null;
+ FragmentRoot rootOperator = null;
+
+ // now we generate all the individual plan fragments and associated assignments. Note, we need all endpoints
+ // assigned before we can materialize, so we start a new loop here rather than utilizing the previous one.
+ for (Wrapper wrapper : planningSet) {
+ Fragment node = wrapper.getNode();
+ Stats stats = node.getStats();
+ final PhysicalOperator physicalOperatorRoot = node.getRoot();
+ boolean isRootNode = rootNode == node;
+
+ if (isRootNode && wrapper.getWidth() != 1)
+ throw new FragmentSetupException(
+ String.format(
+ "Failure while trying to setup fragment. The root fragment must always have parallelization one. In the current case, the width was set to %d.",
+ wrapper.getWidth()));
+ // a fragment is self driven if it doesn't rely on any other exchanges.
+ boolean isLeafFragment = node.getReceivingExchangePairs().size() == 0;
+
+ // Create a minorFragment for each major fragment.
+ for (int minorFragmentId = 0; minorFragmentId < wrapper.getWidth(); minorFragmentId++) {
+ IndexedFragmentNode iNode = new IndexedFragmentNode(minorFragmentId, wrapper);
+ PhysicalOperator op = physicalOperatorRoot.accept(materializer, iNode);
+ Preconditions.checkArgument(op instanceof FragmentRoot);
+ FragmentRoot root = (FragmentRoot) op;
+
+ // get plan as JSON
+ String plan;
+ try {
+ plan = reader.writeJson(root);
+ } catch (JsonProcessingException e) {
+ throw new FragmentSetupException("Failure while trying to convert fragment into json.", e);
+ }
+
+ FragmentHandle handle = FragmentHandle //
+ .newBuilder() //
+ .setMajorFragmentId(wrapper.getMajorFragmentId()) //
+ .setMinorFragmentId(minorFragmentId) //
+ .setQueryId(queryId) //
+ .build();
+ PlanFragment fragment = PlanFragment.newBuilder() //
+ .setCpuCost(stats.getCpuCost()) //
+ .setDiskCost(stats.getDiskCost()) //
+ .setForeman(foremanNode) //
+ .setMemoryCost(stats.getMemoryCost()) //
+ .setNetworkCost(stats.getNetworkCost()) //
+ .setFragmentJson(plan) //
+ .setHandle(handle) //
+ .setAssignment(wrapper.getAssignedEndpoint(minorFragmentId)) //
+ .setLeafFragment(isLeafFragment) //
+ .build();
+
+ if (isRootNode) {
+ rootFragment = fragment;
+ rootOperator = root;
+ } else {
+ fragments.add(fragment);
+ }
+ }
+ }
+
+ return new QueryWorkUnit(rootOperator, rootFragment, fragments);
+
+ }
+
+ private void assignEndpoints(Collection<DrillbitEndpoint> allNodes, PlanningSet planningSet,
+ int globalMaxWidth) throws PhysicalOperatorSetupException {
+ // First we determine the amount of parallelization for a fragment. This will be between 1 and maxWidth based on
+ // cost. (Later could also be based on cluster operation.) then we decide endpoints based on affinity (later this
+ // could be based on endpoint load)
+ for (Wrapper wrapper : planningSet) {
+
+ Stats stats = wrapper.getStats();
+
+ // figure out width.
+ int width = Math.min(stats.getMaxWidth(), globalMaxWidth);
+ float diskCost = stats.getDiskCost();
+ logger.debug("Frag max width: {} and diskCost: {}", stats.getMaxWidth(), diskCost);
+
+ // TODO: right now we'll just assume that each task is cost 1 so we'll set the breadth at the lesser of the number
+ // of tasks or the maximum width of the fragment.
+ if (diskCost < width) {
+ width = (int) diskCost;
+ }
+
+ if (width < 1) width = 1;
+ logger.debug("Setting width {} on fragment {}", width, wrapper);
+ wrapper.setWidth(width);
+ // figure out endpoint assignments. also informs the exchanges about their respective endpoints.
+ wrapper.assignEndpoints(allNodes);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
new file mode 100644
index 0000000..729b2f1
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Stats.java
@@ -0,0 +1,70 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.OperatorCost;
+
+public class Stats {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Stats.class);
+
+ private int maxWidth = Integer.MAX_VALUE;
+ private float networkCost;
+ private float diskCost;
+ private float memoryCost;
+ private float cpuCost;
+
+ public void addMaxWidth(int maxWidth){
+ this.maxWidth = Math.min(this.maxWidth, maxWidth);
+ }
+
+ public void addCost(OperatorCost cost){
+ networkCost += cost.getNetwork();
+ diskCost += cost.getDisk();
+ memoryCost += cost.getMemory();
+ cpuCost += cost.getCpu();
+ }
+
+ public int getMaxWidth() {
+ return maxWidth;
+ }
+
+ public float getNetworkCost() {
+ return networkCost;
+ }
+
+ public float getDiskCost() {
+ return diskCost;
+ }
+
+ public float getMemoryCost() {
+ return memoryCost;
+ }
+
+ public float getCpuCost() {
+ return cpuCost;
+ }
+
+ @Override
+ public String toString() {
+ return "FragmentStats [maxWidth=" + maxWidth + ", networkCost=" + networkCost + ", diskCost=" + diskCost
+ + ", memoryCost=" + memoryCost + ", cpuCost=" + cpuCost + "]";
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
new file mode 100644
index 0000000..d53a78c
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/StatsCollector.java
@@ -0,0 +1,106 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.HasAffinity;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.planner.AbstractOpWrapperVisitor;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+
+import com.google.common.base.Preconditions;
+
+public class StatsCollector {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StatsCollector.class);
+
+ private final static OpStatsCollector opStatCollector = new OpStatsCollector();
+
+ private StatsCollector() {
+ };
+
+ private static void visit(PlanningSet planningSet, Fragment n) {
+ Preconditions.checkNotNull(planningSet);
+ Preconditions.checkNotNull(n);
+
+ Wrapper wrapper = planningSet.get(n);
+ n.getRoot().accept(opStatCollector, wrapper);
+ logger.debug("Set stats to {}", wrapper.getStats());
+ // receivers...
+ for (ExchangeFragmentPair child : n) {
+ // get the fragment node that feeds this node.
+ Fragment childNode = child.getNode();
+ visit(planningSet, childNode);
+ }
+
+ }
+
+ public static PlanningSet collectStats(Fragment rootFragment) {
+ PlanningSet fps = new PlanningSet();
+ visit(fps, rootFragment);
+ return fps;
+ }
+
+ private static class OpStatsCollector extends AbstractOpWrapperVisitor<Void, RuntimeException> {
+
+ @Override
+ public Void visitSendingExchange(Exchange exchange, Wrapper wrapper) throws RuntimeException {
+ Stats stats = wrapper.getStats();
+ stats.addCost(exchange.getAggregateSendCost());
+ stats.addMaxWidth(exchange.getMaxSendWidth());
+ return super.visitSendingExchange(exchange, wrapper);
+ }
+
+ @Override
+ public Void visitReceivingExchange(Exchange exchange, Wrapper wrapper) throws RuntimeException {
+ wrapper.getStats().addCost(exchange.getAggregateReceiveCost());
+ // no traversal since it would cross fragment boundary.
+ return null;
+ }
+
+ @Override
+ public Void visitScan(Scan<?> scan, Wrapper wrapper) {
+ Stats stats = wrapper.getStats();
+ stats.addMaxWidth(scan.getReadEntries().size());
+ return super.visitScan(scan, wrapper);
+ }
+
+ @Override
+ public Void visitStore(Store store, Wrapper wrapper) {
+ Stats stats = wrapper.getStats();
+ stats.addMaxWidth(store.getMaxWidth());
+ return super.visitStore(store, wrapper);
+ }
+
+ @Override
+ public Void visitOp(PhysicalOperator op, Wrapper wrapper) {
+ if(op instanceof HasAffinity){
+ wrapper.addEndpointAffinity(((HasAffinity)op).getOperatorAffinity());
+ }
+ Stats stats = wrapper.getStats();
+ stats.addCost(op.getCost());
+ for (PhysicalOperator child : op) {
+ child.accept(this, wrapper);
+ }
+ return null;
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
new file mode 100644
index 0000000..0dfcb62
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/planner/fragment/Wrapper.java
@@ -0,0 +1,186 @@
+/*******************************************************************************
+ * 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.drill.exec.planner.fragment;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.Exchange;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Scan;
+import org.apache.drill.exec.physical.base.Store;
+import org.apache.drill.exec.planner.fragment.Fragment.ExchangeFragmentPair;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * A wrapping class that allows us to add additional information to each fragment node for planning purposes.
+ */
+public class Wrapper {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Wrapper.class);
+
+ private final Fragment node;
+ private final int majorFragmentId;
+ private int width = -1;
+ private final Stats stats;
+ private boolean endpointsAssigned;
+ private Map<DrillbitEndpoint, EndpointAffinity> endpointAffinity = Maps.newHashMap();
+
+ // a list of assigned endpoints. Technically, there could repeated endpoints in this list if we'd like to assign the
+ // same fragment multiple times to the same endpoint.
+ private List<DrillbitEndpoint> endpoints = Lists.newLinkedList();
+
+ public Wrapper(Fragment node, int majorFragmentId) {
+ this.majorFragmentId = majorFragmentId;
+ this.node = node;
+ this.stats = new Stats();
+ }
+
+ public Stats getStats() {
+ return stats;
+ }
+
+ public void addEndpointAffinity(List<EndpointAffinity> affinities){
+ Preconditions.checkState(!endpointsAssigned);
+ for(EndpointAffinity ea : affinities){
+ addEndpointAffinity(ea.getEndpoint(), ea.getAffinity());
+ }
+ }
+
+ public void addEndpointAffinity(DrillbitEndpoint endpoint, float affinity) {
+ Preconditions.checkState(!endpointsAssigned);
+ EndpointAffinity ea = endpointAffinity.get(endpoint);
+ if (ea == null) {
+ ea = new EndpointAffinity(endpoint);
+ endpointAffinity.put(endpoint, ea);
+ }
+
+ ea.addAffinity(affinity);
+ }
+
+ public int getMajorFragmentId() {
+ return majorFragmentId;
+ }
+
+ public int getWidth() {
+ return width;
+ }
+
+ public void setWidth(int width) {
+ Preconditions.checkState(this.width == -1);
+ this.width = width;
+ }
+
+ public Fragment getNode() {
+ return node;
+ }
+
+ private class AssignEndpointsToScanAndStore extends AbstractPhysicalVisitor<Void, List<DrillbitEndpoint>, PhysicalOperatorSetupException>{
+
+
+ @Override
+ public Void visitExchange(Exchange exchange, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+ if(exchange == node.getSendingExchange()){
+ return visitOp(exchange, value);
+ }
+ // stop on receiver exchange.
+ return null;
+ }
+
+ @Override
+ public Void visitScan(Scan<?> scan, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+ scan.applyAssignments(value);
+ return super.visitScan(scan, value);
+ }
+
+ @Override
+ public Void visitStore(Store store, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+ store.applyAssignments(value);
+ return super.visitStore(store, value);
+ }
+
+ @Override
+ public Void visitOp(PhysicalOperator op, List<DrillbitEndpoint> value) throws PhysicalOperatorSetupException {
+ return visitChildren(op, value);
+ }
+
+ }
+
+ public void assignEndpoints(Collection<DrillbitEndpoint> allPossible) throws PhysicalOperatorSetupException {
+ Preconditions.checkState(!endpointsAssigned);
+
+ endpointsAssigned = true;
+
+ List<EndpointAffinity> values = Lists.newArrayList();
+ values.addAll(endpointAffinity.values());
+
+ if (values.size() == 0) {
+ List<DrillbitEndpoint> all = Lists.newArrayList(allPossible);
+ final int div = allPossible.size();
+ int start = ThreadLocalRandom.current().nextInt(div);
+ // round robin with random start.
+ for (int i = start; i < start + width; i++) {
+ endpoints.add(all.get(i % div));
+ }
+ } else if (values.size() < width) {
+ throw new NotImplementedException(
+ "Haven't implemented a scenario where we have some node affinity but the affinity list is smaller than the expected width.");
+ } else {
+ // get nodes with highest affinity.
+ Collections.sort(values);
+ values = Lists.reverse(values);
+ for (int i = 0; i < width; i++) {
+ endpoints.add(values.get(i).getEndpoint());
+ }
+ }
+
+ // Set scan and store endpoints.
+ AssignEndpointsToScanAndStore visitor = new AssignEndpointsToScanAndStore();
+ node.getRoot().accept(visitor, endpoints);
+
+ // Set the endpoints for this (one at most) sending exchange.
+ if (node.getSendingExchange() != null) {
+ node.getSendingExchange().setupSenders(majorFragmentId, endpoints);
+ }
+
+ // Set the endpoints for each incoming exchange within this fragment.
+ for (ExchangeFragmentPair e : node.getReceivingExchangePairs()) {
+ e.getExchange().setupReceivers(majorFragmentId, endpoints);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "FragmentWrapper [majorFragmentId=" + majorFragmentId + ", width=" + width + ", stats=" + stats + "]";
+ }
+
+ public DrillbitEndpoint getAssignedEndpoint(int minorFragmentId) {
+ Preconditions.checkState(endpointsAssigned);
+ return this.endpoints.get(minorFragmentId);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
deleted file mode 100644
index 562d109..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/NWayOrderingReceiver.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.pop.receiver;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractReceiver;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("nway-ordering-receiver")
-public class NWayOrderingReceiver extends AbstractReceiver{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(NWayOrderingReceiver.class);
-
- @Override
- public List<DrillbitEndpoint> getProvidingEndpoints() {
- return null;
- }
-
- @Override
- public boolean supportsOutOfOrderExchange() {
- return false;
- }
-
- @Override
- public int getSenderCount() {
- return 0;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
- return null;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
deleted file mode 100644
index 487c645..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/receiver/RandomReceiver.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.pop.receiver;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractReceiver;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("receiver-random")
-public class RandomReceiver extends AbstractReceiver{
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RandomReceiver.class);
-
- @Override
- public List<DrillbitEndpoint> getProvidingEndpoints() {
- return null;
- }
-
- @Override
- public boolean supportsOutOfOrderExchange() {
- return false;
- }
-
- @Override
- public int getSenderCount() {
- return 0;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
- return null;
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
deleted file mode 100644
index b0fb51c..0000000
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/pop/sender/HashPartitionSender.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*******************************************************************************
- * 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.drill.exec.pop.sender;
-
-import java.util.List;
-
-import org.apache.drill.common.physical.pop.base.AbstractSender;
-import org.apache.drill.common.physical.pop.base.PhysicalOperator;
-import org.apache.drill.common.physical.pop.base.PhysicalVisitor;
-import org.apache.drill.common.proto.CoordinationProtos.DrillbitEndpoint;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-
-@JsonTypeName("hash-partition-sender")
-public class HashPartitionSender extends AbstractSender {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPartitionSender.class);
-
- public HashPartitionSender(PhysicalOperator child) {
- super(child);
- }
-
-
- @Override
- public List<DrillbitEndpoint> getDestinations() {
- return null;
- }
-
- @Override
- public <T, X, E extends Throwable> T accept(PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
- return null;
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
index d3e4b23..05b1cc7 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/BatchSchema.java
@@ -17,25 +17,19 @@
******************************************************************************/
package org.apache.drill.exec.record;
-import java.util.Collections;
import java.util.Iterator;
import java.util.List;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.exception.SchemaChangeException;
import org.apache.drill.exec.record.vector.ValueVector;
-import com.carrotsearch.hppc.IntObjectOpenHashMap;
-import com.google.common.collect.Lists;
-public class BatchSchema implements Iterable<MaterializedField>{
+public class BatchSchema implements Iterable<MaterializedField> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BatchSchema.class);
-
+
private final List<MaterializedField> fields;
- private final boolean hasSelectionVector;
-
- private BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
+ final boolean hasSelectionVector;
+
+ BatchSchema(boolean hasSelectionVector, List<MaterializedField> fields) {
this.fields = fields;
this.hasSelectionVector = hasSelectionVector;
}
@@ -45,88 +39,16 @@ public class BatchSchema implements Iterable<MaterializedField>{
return fields.iterator();
}
- public void addAnyField(short fieldId, boolean nullable, ValueMode mode){
- addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
+ public static SchemaBuilder newBuilder() {
+ return new SchemaBuilder();
}
-
- public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass){
- fields.add(new MaterializedField(fieldId, type, nullable, mode, valueClass));
+
+ @Override
+ public String toString() {
+ return "BatchSchema [fields=" + fields + ", hasSelectionVector=" + hasSelectionVector + "]";
}
+
- /**
- * Builder to build BatchSchema. Can have a supporting expected object. If the expected Schema object is defined, the builder will always check that this schema is a equal or more materialized version of the current schema.
- */
- public class BatchSchemaBuilder{
- private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
- private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
-
- private boolean hasSelectionVector;
-
- public BatchSchemaBuilder(BatchSchema expected){
- for(MaterializedField f: expected){
- expectedFields.put(f.getFieldId(), f);
- }
- hasSelectionVector = expected.hasSelectionVector;
- }
-
- public BatchSchemaBuilder(){
- }
-
-
- /**
- * Add a field where we don't have type information. In this case, DataType will be set to LATEBIND and valueClass will be set to null.
- * @param fieldId The desired fieldId. Should be unique for this BatchSchema.
- * @param nullable Whether this field supports nullability.
- * @param mode
- * @throws SchemaChangeException
- */
- public void addLateBindField(short fieldId, boolean nullable, ValueMode mode) throws SchemaChangeException{
- addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
- }
-
- public void setSelectionVector(boolean hasSelectionVector){
- this.hasSelectionVector = hasSelectionVector;
- }
-
- private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
- MaterializedField f = new MaterializedField(fieldId, type, nullable, mode, valueClass);
- if(expectedFields != null){
- if(!expectedFields.containsKey(f.getFieldId())) throw new SchemaChangeException(String.format("You attempted to add a field for Id An attempt was made to add a duplicate fieldId to the schema. The offending fieldId was %d", fieldId));
- f.checkMaterialization(expectedFields.lget());
- }
- fields.put(f.getFieldId(), f);
- }
-
- public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
- if(fields.containsKey(fieldId)) throw new SchemaChangeException(String.format("An attempt was made to add a duplicate fieldId to the schema. The offending fieldId was %d", fieldId));
- setTypedField(fieldId, type, nullable, mode, valueClass);
- }
-
- public void replaceTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) throws SchemaChangeException{
- if(!fields.containsKey(fieldId)) throw new SchemaChangeException(String.format("An attempt was made to replace a field in the schema, however the schema does not currently contain that field id. The offending fieldId was %d", fieldId));
- setTypedField(fieldId, type, nullable, mode, valueClass);
- }
-
-// public void addVector(ValueVector<?> v){
-//
-// }
-//
-// public void replaceVector(ValueVector<?> oldVector, ValueVector<?> newVector){
-//
-// }
-
-
- public BatchSchema buildAndClear() throws SchemaChangeException{
- // check if any fields are unaccounted for.
-
- List<MaterializedField> fieldList = Lists.newArrayList();
- for(MaterializedField f : fields.values){
- if(f != null) fieldList.add(f);
- }
- Collections.sort(fieldList);
- return new BatchSchema(this.hasSelectionVector, fieldList);
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
new file mode 100644
index 0000000..c19065d
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/FragmentWritableBatch.java
@@ -0,0 +1,59 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+
+public class FragmentWritableBatch{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentWritableBatch.class);
+
+ private final ByteBuf[] buffers;
+ private final FragmentRecordBatch header;
+
+ public FragmentWritableBatch(boolean isLast, QueryId queryId, int sendMajorFragmentId, int sendMinorFragmentId, int receiveMajorFragmentId, int receiveMinorFragmentId, WritableBatch batch){
+ this.buffers = batch.getBuffers();
+ FragmentHandle handle = FragmentHandle //
+ .newBuilder() //
+ .setMajorFragmentId(receiveMajorFragmentId) //
+ .setMinorFragmentId(receiveMinorFragmentId) //
+ .setQueryId(queryId) //
+ .build();
+ this.header = FragmentRecordBatch //
+ .newBuilder() //
+ .setIsLastBatch(isLast) //
+ .setDef(batch.getDef()) //
+ .setHandle(handle) //
+ .setSendingMajorFragmentId(sendMajorFragmentId) //
+ .setSendingMinorFragmentId(sendMinorFragmentId) //
+ .build();
+ }
+
+ public ByteBuf[] getBuffers(){
+ return buffers;
+ }
+
+ public FragmentRecordBatch getHeader() {
+ return header;
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
index 403c7a3..d820e0e 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/InvalidValueAccessor.java
@@ -17,7 +17,7 @@
******************************************************************************/
package org.apache.drill.exec.record;
-import org.apache.drill.exec.exception.ExecutionSetupException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
public class InvalidValueAccessor extends ExecutionSetupException{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InvalidValueAccessor.class);
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
new file mode 100644
index 0000000..718396e
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MajorTypeSerDe.java
@@ -0,0 +1,116 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.MinorType;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
+
+public class MajorTypeSerDe {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MajorTypeSerDe.class);
+
+
+ public static class De extends StdDeserializer<MajorType> {
+
+ public De() {
+ super(MajorType.class);
+ }
+
+ @Override
+ public MajorType deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+ JsonProcessingException {
+ return jp.readValueAs(MajorTypeHolder.class).getMajorType();
+ }
+
+
+ }
+
+
+ public static class Se extends StdSerializer<MajorType> {
+
+ public Se() {
+ super(MajorType.class);
+ }
+
+ @Override
+ public void serialize(MajorType value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
+ JsonGenerationException {
+ MajorTypeHolder holder = MajorTypeHolder.get(value);
+ jgen.writeObject(holder);
+ }
+
+ }
+
+ @JsonInclude(Include.NON_NULL)
+ public static class MajorTypeHolder{
+ @JsonProperty("type") public MinorType minorType;
+ public DataMode mode;
+ public Integer width;
+ public Integer precision;
+ public Integer scale;
+
+ @JsonCreator
+ public MajorTypeHolder(@JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) {
+ super();
+ this.minorType = minorType;
+ this.mode = mode;
+ this.width = width;
+ this.precision = precision;
+ this.scale = scale;
+ }
+
+ private MajorTypeHolder(){}
+
+ @JsonIgnore
+ public MajorType getMajorType(){
+ MajorType.Builder b = MajorType.newBuilder();
+ b.setMode(mode);
+ b.setMinorType(minorType);
+ if(precision != null) b.setPrecision(precision);
+ if(width != null) b.setWidth(width);
+ if(scale != null) b.setScale(scale);
+ return b.build();
+ }
+
+ public static MajorTypeHolder get(MajorType mt){
+ MajorTypeHolder h = new MajorTypeHolder();
+ h.minorType = mt.getMinorType();
+ h.mode = mt.getMode();
+ if(mt.hasPrecision()) h.precision = mt.getPrecision();
+ if(mt.hasScale()) h.scale = mt.getScale();
+ if(mt.hasWidth()) h.width = mt.getWidth();
+ return h;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index 2e941a2..09427ef 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -17,67 +17,152 @@
******************************************************************************/
package org.apache.drill.exec.record;
-import org.apache.drill.common.expression.types.DataType;
-import org.apache.drill.common.physical.RecordField.ValueMode;
-import org.apache.drill.exec.exception.SchemaChangeException;
-
-public class MaterializedField implements Comparable<MaterializedField>{
- private int fieldId;
- private DataType type;
- private boolean nullable;
- private ValueMode mode;
- private Class<?> valueClass;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.DataMode;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.SchemaDefProtos.MajorType;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
+import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
+import org.apache.drill.exec.record.vector.TypeHelper;
+
+public class MaterializedField implements Comparable<MaterializedField> {
+ private final FieldDef def;
+
+ public MaterializedField(FieldDef def) {
+ this.def = def;
+ }
+
+ public static MaterializedField create(FieldDef def){
+ return new MaterializedField(def);
+ }
+
+ public static MaterializedField create(SchemaPath path, int fieldId, int parentId, MajorType type) {
+ FieldDef.Builder b = FieldDef.newBuilder();
+ b.setFieldId(fieldId);
+ b.setMajorType(type);
+ addSchemaPathToFieldDef(path, b);
+ b.setParentId(parentId);
+ return create(b.build());
+ }
+
+ private static void addSchemaPathToFieldDef(SchemaPath path, FieldDef.Builder builder) {
+ for (PathSegment p = path.getRootSegment();; p = p.getChild()) {
+ NamePart.Builder b = NamePart.newBuilder();
+ if (p.isArray()) {
+ b.setType(Type.ARRAY);
+ } else {
+ b.setName(p.getNameSegment().getPath().toString());
+ b.setType(Type.NAME);
+ }
+ builder.addName(b.build());
+ if(p.isLastPath()) break;
+ }
+ }
+
+ public FieldDef getDef() {
+ return def;
+ }
- public MaterializedField(int fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass) {
- super();
- this.fieldId = fieldId;
- this.type = type;
- this.nullable = nullable;
- this.mode = mode;
- this.valueClass = valueClass;
+ public String getName(){
+ StringBuilder sb = new StringBuilder();
+ boolean first = true;
+ for(NamePart np : def.getNameList()){
+ if(np.getType() == Type.ARRAY){
+ sb.append("[]");
+ }else{
+ if(first){
+ first = false;
+ }else{
+ sb.append(".");
+ }
+ sb.append(np.getName());
+
+ }
+ }
+ return sb.toString();
+ }
+
+ public int getWidth() {
+ return def.getMajorType().getWidth();
}
public int getFieldId() {
- return fieldId;
+ return def.getFieldId();
}
- public DataType getType() {
- return type;
+ public MajorType getType() {
+ return def.getMajorType();
}
public boolean isNullable() {
- return nullable;
+ return def.getMajorType().getMode() == DataMode.OPTIONAL;
}
- public ValueMode getMode() {
- return mode;
+ public DataMode getDataMode() {
+ return def.getMajorType().getMode();
}
public Class<?> getValueClass() {
- return valueClass;
+ return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
}
- private void check(String name, Object val1, Object expected) throws SchemaChangeException{
- if(expected.equals(val1)) return;
- throw new SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name, val1, name, expected);
- }
-
- public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
- if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s", this.type, expected.type);
- if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
- check("fieldId", this.fieldId, expected.fieldId);
- check("nullability", this.nullable, expected.nullable);
- check("valueMode", this.mode, expected.mode);
- }
+ public boolean matches(SchemaPath path) {
+ Iterator<NamePart> iter = def.getNameList().iterator();
+
+ for (PathSegment p = path.getRootSegment();; p = p.getChild()) {
+ if(p == null) break;
+ if (!iter.hasNext()) return false;
+ NamePart n = iter.next();
+
+ if (p.isArray()) {
+ if (n.getType() == Type.ARRAY) continue;
+ return false;
+ } else {
+ if (p.getNameSegment().getPath().equals(n.getName())) continue;
+ return false;
+ }
+
+ }
+ // we've reviewed all path segments. confirm that we don't have any extra name parts.
+ return !iter.hasNext();
- public MaterializedField getNullableVersion(Class<?> valueClass){
- return new MaterializedField(fieldId, type, true, mode, valueClass);
}
-
+
+ // private void check(String name, Object val1, Object expected) throws SchemaChangeException{
+ // if(expected.equals(val1)) return;
+ // throw new
+ // SchemaChangeException("Expected and actual field definitions don't match. Actual %s: %s, expected %s: %s", name,
+ // val1, name, expected);
+ // }
+
+ // public void checkMaterialization(MaterializedField expected) throws SchemaChangeException{
+ // if(this.type == expected.type || expected.type == DataType.LATEBIND) throw new
+ // SchemaChangeException("Expected and actual field definitions don't match. Actual DataType: %s, expected DataTypes: %s",
+ // this.type, expected.type);
+ // if(expected.valueClass != null) check("valueClass", this.valueClass, expected.valueClass);
+ // check("fieldId", this.fieldId, expected.fieldId);
+ // check("nullability", this.nullable, expected.nullable);
+ // check("valueMode", this.mode, expected.mode);
+ // }
+ //
+ // public MaterializedField getNullableVersion(Class<?> valueClass){
+ // return new MaterializedField(path, fieldId, type, true, mode, valueClass);
+ // }
+
@Override
public int compareTo(MaterializedField o) {
- return Integer.compare(this.fieldId, o.fieldId);
+ return Integer.compare(this.getFieldId(), o.getFieldId());
}
-
+
+ @Override
+ public String toString() {
+ return "MaterializedField [" + def.toString() + "]";
+ }
+
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
new file mode 100644
index 0000000..c244cea
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatch.java
@@ -0,0 +1,44 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import org.apache.drill.exec.proto.ExecProtos.FragmentRecordBatch;
+
+public class RawFragmentBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RawFragmentBatch.class);
+
+ final FragmentRecordBatch header;
+ final ByteBuf body;
+
+ public RawFragmentBatch(FragmentRecordBatch header, ByteBuf body) {
+ super();
+ this.header = header;
+ this.body = body;
+ }
+
+ public FragmentRecordBatch getHeader() {
+ return header;
+ }
+
+ public ByteBuf getBody() {
+ return body;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
new file mode 100644
index 0000000..08b0e11
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RawFragmentBatchProvider.java
@@ -0,0 +1,27 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import org.apache.drill.exec.ops.FragmentContext;
+
+public interface RawFragmentBatchProvider {
+
+ public RawFragmentBatch getNext();
+ public void kill(FragmentContext context);
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
index eca62bb..3e4ded2 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatch.java
@@ -34,8 +34,9 @@ public interface RecordBatch {
NONE, // No more records were found.
OK, // A new range of records have been provided.
OK_NEW_SCHEMA, // A full collection of records
- STOP // Informs parent nodes that the query has terminated. In this case, a consumer can consume their QueryContext
+ STOP, // Informs parent nodes that the query has terminated. In this case, a consumer can consume their QueryContext
// to understand the current state of things.
+ NOT_YET // used by batches that haven't received incoming data yet.
}
/**
@@ -81,5 +82,11 @@ public interface RecordBatch {
* @return An IterOutcome describing the result of the iteration.
*/
public IterOutcome next();
+
+ /**
+ * Get a writable version of this batch. Takes over owernship of existing buffers.
+ * @return
+ */
+ public WritableBatch getWritableBatch();
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
new file mode 100644
index 0000000..d990198
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -0,0 +1,143 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.vector.TypeHelper;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.IntObjectCursor;
+
+public class RecordBatchLoader implements Iterable<IntObjectCursor<ValueVector<?>>>{
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RecordBatchLoader.class);
+
+ private IntObjectOpenHashMap<ValueVector<?>> vectors = new IntObjectOpenHashMap<ValueVector<?>>();
+ private final BufferAllocator allocator;
+ private int recordCount;
+ private BatchSchema schema;
+
+ public RecordBatchLoader(BufferAllocator allocator) {
+ super();
+ this.allocator = allocator;
+ }
+
+ /**
+ * Load a record batch from a single buffer.
+ *
+ * @param def
+ * The definition for the record batch.
+ * @param buf
+ * The buffer that holds the data ssociated with the record batch
+ * @return Whether or not the schema changed since the previous load.
+ * @throws SchemaChangeException
+ */
+ public boolean load(RecordBatchDef def, ByteBuf buf) throws SchemaChangeException {
+// logger.debug("Loading record batch with def {} and data {}", def, buf);
+ this.recordCount = def.getRecordCount();
+ boolean schemaChanged = false;
+
+ IntObjectOpenHashMap<ValueVector<?>> newVectors = new IntObjectOpenHashMap<ValueVector<?>>();
+
+ List<FieldMetadata> fields = def.getFieldList();
+
+ int bufOffset = 0;
+ for (FieldMetadata fmd : fields) {
+ FieldDef fieldDef = fmd.getDef();
+ ValueVector<?> v = vectors.remove(fieldDef.getFieldId());
+ if (v != null) {
+ if (v.getField().getDef().equals(fieldDef)) {
+ v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+ newVectors.put(fieldDef.getFieldId(), v);
+ continue;
+ } else {
+ v.close();
+ v = null;
+ }
+ }
+ // if we arrive here, either the metadata didn't match, or we didn't find a vector.
+ schemaChanged = true;
+ MaterializedField m = new MaterializedField(fieldDef);
+ v = TypeHelper.getNewVector(m, allocator);
+ v.setTo(fmd, buf.slice(bufOffset, fmd.getBufferLength()));
+ newVectors.put(fieldDef.getFieldId(), v);
+ }
+
+ if(!vectors.isEmpty()){
+ schemaChanged = true;
+ for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+ cursor.value.close();
+ }
+
+ }
+
+ if(schemaChanged){
+ // rebuild the schema.
+ SchemaBuilder b = BatchSchema.newBuilder();
+ for(IntObjectCursor<ValueVector<?>> cursor : newVectors){
+ b.addField(cursor.value.getField());
+ }
+ b.setSelectionVector(false);
+ this.schema = b.build();
+ }
+ vectors = newVectors;
+ return schemaChanged;
+
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T extends ValueVector<T>> T getValueVector(int fieldId, Class<T> clazz) throws InvalidValueAccessor {
+ ValueVector<?> v = vectors.get(fieldId);
+ assert v != null;
+ if (v.getClass() != clazz)
+ throw new InvalidValueAccessor(String.format(
+ "Failure while reading vector. Expected vector class of %s but was holding vector class %s.",
+ clazz.getCanonicalName(), v.getClass().getCanonicalName()));
+ return (T) v;
+ }
+
+ public int getRecordCount() {
+ return recordCount;
+ }
+
+
+ public WritableBatch getWritableBatch(){
+ return WritableBatch.get(recordCount, vectors);
+ }
+
+ @Override
+ public Iterator<IntObjectCursor<ValueVector<?>>> iterator() {
+ return this.vectors.iterator();
+ }
+
+ public BatchSchema getSchema(){
+ return schema;
+ }
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
new file mode 100644
index 0000000..1e25b1a
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/SchemaBuilder.java
@@ -0,0 +1,127 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.cursors.ObjectCursor;
+import com.google.common.collect.Lists;
+
+/**
+ * A reusable builder that supports the creation of BatchSchemas. Can have a supporting expected object. If the expected Schema object is defined, the
+ * builder will always check that this schema is a equal or more materialized version of the current schema.
+ */
+public class SchemaBuilder {
+ private IntObjectOpenHashMap<MaterializedField> fields = new IntObjectOpenHashMap<MaterializedField>();
+ private IntObjectOpenHashMap<MaterializedField> expectedFields = new IntObjectOpenHashMap<MaterializedField>();
+
+ private boolean hasSelectionVector;
+
+ public SchemaBuilder(BatchSchema expected) {
+ for (MaterializedField f : expected) {
+ expectedFields.put(f.getFieldId(), f);
+ }
+ hasSelectionVector = expected.hasSelectionVector;
+ }
+
+ SchemaBuilder() {
+ }
+
+ /**
+ * Add a field where we don't have type information. In this case, DataType will be set to LATEBIND and valueClass
+ * will be set to null.
+ *
+ * @param fieldId
+ * The desired fieldId. Should be unique for this BatchSchema.
+ * @param nullable
+ * Whether this field supports nullability.
+ * @param mode
+ * @throws SchemaChangeException
+ */
+// public void addLateBindField(short fieldId, boolean nullable, ValueMode mode) throws SchemaChangeException {
+// addTypedField(fieldId, DataType.LATEBIND, nullable, mode, Void.class);
+// }
+
+ public void setSelectionVector(boolean hasSelectionVector) {
+ this.hasSelectionVector = hasSelectionVector;
+ }
+
+
+// private void setTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+// throws SchemaChangeException {
+// MaterializedField f = new MaterializedField(fieldId, type, nullable, mode, valueClass);
+// if (expectedFields != null) {
+// if (!expectedFields.containsKey(f.getFieldId()))
+// throw new SchemaChangeException(
+// String
+// .format(
+// "You attempted to add a field for Id An attempt was made to add a duplicate fieldId to the schema. The offending fieldId was %d",
+// fieldId));
+// f.checkMaterialization(expectedFields.lget());
+// }
+// fields.put(f.getFieldId(), f);
+// }
+
+ public void addField(MaterializedField f){
+ fields.put(f.getFieldId(), f);
+ }
+
+// public void addTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+// throws SchemaChangeException {
+// if (fields.containsKey(fieldId))
+// throw new SchemaChangeException(String.format(
+// "An attempt was made to add a duplicate fieldId to the schema. The offending fieldId was %d", fieldId));
+// setTypedField(fieldId, type, nullable, mode, valueClass);
+// }
+//
+// public void replaceTypedField(short fieldId, DataType type, boolean nullable, ValueMode mode, Class<?> valueClass)
+// throws SchemaChangeException {
+// if (!fields.containsKey(fieldId))
+// throw new SchemaChangeException(
+// String.format("An attempt was made to replace a field in the schema, however the schema does " +
+// "not currently contain that field id. The offending fieldId was %d", fieldId));
+// setTypedField(fieldId, type, nullable, mode, valueClass);
+// }
+
+ public void removeField(short fieldId) throws SchemaChangeException{
+ MaterializedField f = fields.remove(fieldId);
+ if(f == null) throw new SchemaChangeException("You attempted to remove an nonexistent field.");
+ }
+
+ /**
+ * Generate a new BatchSchema object based on the current state of the builder.
+ * @return
+ * @throws SchemaChangeException
+ */
+ public BatchSchema build() throws SchemaChangeException {
+ // check if any fields are unaccounted for.
+
+ List<MaterializedField> fieldList = Lists.newArrayList();
+ for (ObjectCursor<MaterializedField> f : fields.values()) {
+ if (f != null) fieldList.add(f.value);
+ }
+ Collections.sort(fieldList);
+ return new BatchSchema(this.hasSelectionVector, fieldList);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
new file mode 100644
index 0000000..788c731
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/WritableBatch.java
@@ -0,0 +1,108 @@
+/*******************************************************************************
+ * 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.drill.exec.record;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.List;
+
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.proto.UserBitShared.RecordBatchDef;
+import org.apache.drill.exec.record.vector.ValueVector;
+
+import com.carrotsearch.hppc.IntObjectOpenHashMap;
+import com.carrotsearch.hppc.procedures.IntObjectProcedure;
+import com.google.common.collect.Lists;
+
+/**
+ * A specialized version of record batch that can moves out buffers and preps them for writing.
+ */
+public class WritableBatch {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(WritableBatch.class);
+
+ private final RecordBatchDef def;
+ private final ByteBuf[] buffers;
+
+ public WritableBatch(RecordBatchDef def, List<ByteBuf> buffers) {
+ logger.debug("Created new writable batch with def {} and buffers {}", def, buffers);
+ this.def = def;
+ this.buffers = buffers.toArray(new ByteBuf[buffers.size()]);
+ }
+
+ public WritableBatch(RecordBatchDef def, ByteBuf[] buffers) {
+ super();
+ this.def = def;
+ this.buffers = buffers;
+ }
+
+
+ public RecordBatchDef getDef(){
+ return def;
+ }
+ public ByteBuf[] getBuffers(){
+ return buffers;
+ }
+
+// public static WritableBatch get(ValueVector<?>[] vectors){
+// WritableCreator c = new WritableCreator();
+// for(int i =0; i < vectors.length; i++){
+// c.apply(i, vectors[i]);
+// }
+// return c.get();
+// }
+//
+
+ public static WritableBatch get(int recordCount, IntObjectOpenHashMap<ValueVector<?>> fields){
+ WritableCreator creator = new WritableCreator(recordCount);
+ fields.forEach(creator);
+ return creator.get();
+
+ }
+
+ private static class WritableCreator implements IntObjectProcedure<ValueVector<?>>{
+
+ List<ByteBuf> buffers = Lists.newArrayList();
+ List<FieldMetadata> metadata = Lists.newArrayList();
+ private int recordCount;
+
+
+ public WritableCreator(int recordCount) {
+ super();
+ this.recordCount = recordCount;
+ }
+
+ @Override
+ public void apply(int key, ValueVector<?> value) {
+ metadata.add(value.getMetadata());
+ for(ByteBuf b : value.getBuffers()){
+ buffers.add(b);
+ b.retain();
+ }
+ // allocate new buffer to release hold on old buffer.
+ value.allocateNew(value.capacity());
+ }
+
+
+ public WritableBatch get(){
+ RecordBatchDef batchDef = RecordBatchDef.newBuilder().addAllField(metadata).setRecordCount(recordCount).build();
+ WritableBatch b = new WritableBatch(batchDef, buffers);
+ return b;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
index 912e02d..b32f067 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/AbstractFixedValueVector.java
@@ -20,6 +20,8 @@ package org.apache.drill.exec.record.vector;
import io.netty.buffer.ByteBuf;
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
+import org.apache.drill.exec.record.MaterializedField;
/**
* Abstract class that fixed value vectors are derived from.
@@ -27,12 +29,12 @@ import org.apache.drill.exec.memory.BufferAllocator;
abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> extends BaseValueVector<T> {
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractFixedValueVector.class);
- private final int widthInBits;
+ protected final int widthInBits;
protected int longWords = 0;
-
- public AbstractFixedValueVector(int fieldId, BufferAllocator allocator, int widthInBits) {
- super(fieldId, allocator);
+
+ public AbstractFixedValueVector(MaterializedField field, BufferAllocator allocator, int widthInBits) {
+ super(field, allocator);
this.widthInBits = widthInBits;
}
@@ -56,5 +58,16 @@ abstract class AbstractFixedValueVector<T extends AbstractFixedValueVector<T>> e
longWords = 0;
}
+ @Override
+ public void setRecordCount(int recordCount) {
+ this.data.writerIndex(recordCount*(widthInBits/8));
+ super.setRecordCount(recordCount);
+ }
+
+
+
+
+
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
index 8d524b2..b001add 100644
--- a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/BaseValueVector.java
@@ -19,20 +19,25 @@ package org.apache.drill.exec.record.vector;
import io.netty.buffer.ByteBuf;
+import java.util.Random;
+
import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserBitShared.FieldMetadata;
import org.apache.drill.exec.record.DeadBuf;
+import org.apache.drill.exec.record.MaterializedField;
public abstract class BaseValueVector<T extends BaseValueVector<T>> implements ValueVector<T>{
static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseValueVector.class);
protected final BufferAllocator allocator;
protected ByteBuf data = DeadBuf.DEAD_BUFFER;
- protected int valueCount = 0;
- protected final int fieldId;
+ protected int maxValueCount = 0;
+ protected final MaterializedField field;
+ private int recordCount;
- public BaseValueVector(int fieldId, BufferAllocator allocator) {
+ public BaseValueVector(MaterializedField field, BufferAllocator allocator) {
this.allocator = allocator;
- this.fieldId = fieldId;
+ this.field = field;
}
public final void allocateNew(int valueCount){
@@ -42,35 +47,42 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
resetAllocation(valueCount, newBuf);
}
- protected abstract int getAllocationSize(int valueCount);
+ protected abstract int getAllocationSize(int maxValueCount);
protected abstract void childResetAllocation(int valueCount, ByteBuf buf);
protected abstract void childCloneMetadata(T other);
protected abstract void childClear();
- protected final void resetAllocation(int valueCount, ByteBuf buf){
+ /**
+ * Update the current buffer allocation utilize the provided allocation.
+ * @param valueCount
+ * @param buf
+ */
+ protected final void resetAllocation(int maxValueCount, ByteBuf buf){
clear();
- this.valueCount = valueCount;
+ buf.retain();
+ this.maxValueCount = maxValueCount;
this.data = buf;
- childResetAllocation(valueCount, buf);
+ childResetAllocation(maxValueCount, buf);
}
public final void cloneMetadata(T other){
- other.valueCount = this.valueCount;
+ other.maxValueCount = this.maxValueCount;
}
+
@Override
public final void cloneInto(T vector) {
- vector.allocateNew(valueCount);
+ vector.allocateNew(maxValueCount);
data.writeBytes(vector.data);
cloneMetadata(vector);
- childResetAllocation(valueCount, vector.data);
+ childResetAllocation(maxValueCount, vector.data);
}
@Override
public final void transferTo(T vector) {
vector.data = this.data;
cloneMetadata(vector);
- childResetAllocation(valueCount, data);
+ childResetAllocation(maxValueCount, data);
clear();
}
@@ -78,7 +90,7 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
if(this.data != DeadBuf.DEAD_BUFFER){
this.data.release();
this.data = DeadBuf.DEAD_BUFFER;
- this.valueCount = 0;
+ this.maxValueCount = 0;
}
childClear();
}
@@ -88,8 +100,8 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
*
* @return
*/
- public int size() {
- return valueCount;
+ public int capacity() {
+ return maxValueCount;
}
@Override
@@ -98,8 +110,48 @@ public abstract class BaseValueVector<T extends BaseValueVector<T>> implements V
}
@Override
- public ByteBuf getBuffer() {
- return data;
+ public ByteBuf[] getBuffers() {
+ return new ByteBuf[]{data};
+ }
+
+ public MaterializedField getField(){
+ return field;
+ }
+
+
+ public int getRecordCount() {
+ return recordCount;
+ }
+
+ public void setRecordCount(int recordCount) {
+ this.recordCount = recordCount;
+ }
+
+ @Override
+ public FieldMetadata getMetadata() {
+ int len = 0;
+ for(ByteBuf b : getBuffers()){
+ len += b.writerIndex();
+ }
+ return FieldMetadata.newBuilder().setDef(getField().getDef()).setValueCount(getRecordCount()).setBufferLength(len).build();
+ }
+
+ @Override
+ public void setTo(FieldMetadata metadata, ByteBuf data) {
+// logger.debug("Updating value vector to {}, {}", metadata, data);
+ clear();
+ resetAllocation(metadata.getValueCount(), data);
+ }
+
+ @Override
+ public void randomizeData() {
+ if(this.data != DeadBuf.DEAD_BUFFER){
+ Random r = new Random();
+ for(int i =0; i < data.capacity()-8; i+=8){
+ data.setLong(i, r.nextLong());
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e57a8d6d/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
----------------------------------------------------------------------
diff --git a/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
new file mode 100644
index 0000000..533e3bd
--- /dev/null
+++ b/sandbox/prototype/exec/java-exec/src/main/java/org/apache/drill/exec/record/vector/Bit.java
@@ -0,0 +1,168 @@
+/*******************************************************************************
+ * 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.drill.exec.record.vector;
+
+import org.apache.drill.common.expression.types.DataType;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.RecordField.ValueMode;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.hadoop.io.SequenceFile;
+
+/**
+ * Describes a vector which holds a number of true/false values.
+ */
+public class Bit extends AbstractFixedValueVector<Bit> {
+ static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Bit.class);
+
+ public Bit(MaterializedField field, BufferAllocator allocator) {
+ super(field, allocator, 1);
+ }
+
+
+// /** Returns true or false for the specified bit index.
+// * The index should be less than the OpenBitSet size
+// */
+// public boolean get(int index) {
+// assert index >= 0 && index < this.valueCount;
+// int i = index >> 3; // div 8
+// // signed shift will keep a negative index and force an
+// // array-index-out-of-bounds-exception, removing the need for an explicit check.
+// int bit = index & 0x3f; // mod 64
+// long bitmask = 1L << bit;
+// return (data.getLong(i) & bitmask) != 0;
+// }
+
+ public int getBit(int index) {
+
+ assert index >= 0 && index < this.maxValueCount;
+ int i = 8*(index >> 6); // div 8
+ int bit = index & 0x3f; // mod 64
+ return ((int) (data.getLong(i) >>> bit)) & 0x01;
+ }
+
+ /** Sets the bit at the specified index.
+ * The index should be less than the OpenBitSet size.
+ */
+ public void set(int index) {
+ assert index >= 0 && index < this.maxValueCount;
+ int wordNum = index >> 3;
+ int bit = index & 0x3f;
+ long bitmask = 1L << bit;
+ data.setLong(wordNum, data.getLong(wordNum) | bitmask);
+ }
+
+ public void clear(int index) {
+ assert index >= 0 && index < this.maxValueCount;
+ int wordNum = index >> 3;
+ int bit = index & 0x03f;
+ long bitmask = 1L << bit;
+ data.setLong(wordNum, data.getLong(wordNum) & ~bitmask);
+ }
+
+
+
+ /** Clears a range of bits. Clearing past the end does not change the size of the set.
+ *
+ * @param startBitIndex lower index
+ * @param lastBitIndex one-past the last bit to clear
+ */
+ private void clear2(int startBitIndex, int lastBitIndex) {
+ if (lastBitIndex <= startBitIndex) return;
+
+ int firstWordStart = (startBitIndex>>3);
+ if (firstWordStart >= this.longWords) return;
+
+ // since endIndex is one past the end, this is index of the last
+ // word to be changed.
+ int lastWordStart = ((lastBitIndex-1)>>3);
+
+ long startmask = -1L << startBitIndex;
+ long endmask = -1L >>> -lastBitIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+
+ // invert masks since we are clearing
+ startmask = ~startmask;
+ endmask = ~endmask;
+
+ if (firstWordStart == lastWordStart) {
+ data.setLong(firstWordStart, data.getLong(firstWordStart) & (startmask | endmask));
+ return;
+ }
+ data.setLong(firstWordStart, data.getLong(firstWordStart) & startmask);
+
+ int middle = Math.min(this.longWords, lastWordStart);
+
+ for(int i =firstWordStart+8; i < middle; i += 8){
+ data.setLong(i, 0L);
+ }
+ if (lastWordStart < this.longWords) {
+ data.setLong(lastWordStart, data.getLong(lastWordStart) & endmask);
+ }
+ }
+
+ public void setAllFalse(){
+ clear(0, maxValueCount);
+ }
+
+
+ public void clear(int startIndex, int endIndex) {
+ if (endIndex <= startIndex) return;
+
+ int startWord = (startIndex >> 6);
+ if (startWord >= longWords) return;
+
+ // since endIndex is one past the end, this is index of the last
+ // word to be changed.
+ int endWord = ((endIndex - 1) >> 6);
+
+ long startmask = -1L << startIndex;
+ long endmask = -1L >>> -endIndex; // 64-(endIndex&0x3f) is the same as -endIndex due to wrap
+
+ // invert masks since we are clearing
+ startmask = ~startmask;
+ endmask = ~endmask;
+
+ int startWordPos = startWord * 8;
+ if (startWord == endWord) {
+ data.setLong(startWordPos, data.getLong(startWordPos) & (startmask | endmask));
+ return;
+ }
+
+ int endWordPos = endWord * 8;
+
+ data.setLong(startWordPos, data.getLong(startWordPos) & startmask);
+
+ int middle = Math.min(longWords, endWord)*8;
+
+
+ for(int i =startWordPos+8; i < middle; i += 8){
+ data.setLong(i, 0L);
+ }
+
+ if (endWordPos < startWordPos) {
+ data.setLong(endWordPos, data.getLong(endWordPos) & endmask);
+ }
+ }
+
+
+ @Override
+ public Object getObject(int index) {
+ return this.getBit(index);
+ }
+
+
+}