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/07/20 03:57:40 UTC

[19/53] [abbrv] git commit: Update typing system. Update RPC system. Add Fragmenting Implementation. Working single node. Distributed failing due to threading issues.

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/master
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 +-
 .../exceptions/ExecutionSetupException.java     |  44 +++
 .../PhysicalOperatorSetupException.java         |  45 +++
 .../drill/common/expression/SchemaPath.java     |   2 +-
 .../drill/common/expression/types/DataType.java |  11 +
 .../apache/drill/common/optimize/Optimizer.java |  45 ---
 .../common/physical/DataValidationMode.java     |  24 --
 .../drill/common/physical/EndpointAffinity.java |  60 ----
 .../apache/drill/common/physical/FieldSet.java  |  80 ------
 .../drill/common/physical/OperatorCost.java     |  61 -----
 .../drill/common/physical/PhysicalPlan.java     |  95 -------
 .../apache/drill/common/physical/ReadEntry.java |  25 --
 .../drill/common/physical/RecordField.java      |  60 ----
 .../drill/common/physical/WriteEntry.java       |  22 --
 .../drill/common/physical/pop/Filter.java       |  52 ----
 .../physical/pop/PartitionToRandomExchange.java |  92 -------
 .../drill/common/physical/pop/Project.java      |  53 ----
 .../drill/common/physical/pop/Screen.java       |  77 ------
 .../apache/drill/common/physical/pop/Sort.java  |  57 ----
 .../common/physical/pop/base/AbstractBase.java  |  56 ----
 .../physical/pop/base/AbstractExchange.java     |  68 -----
 .../pop/base/AbstractPhysicalVisitor.java       |  80 ------
 .../physical/pop/base/AbstractReceiver.java     |  32 ---
 .../common/physical/pop/base/AbstractScan.java  |  62 -----
 .../physical/pop/base/AbstractSender.java       |  29 --
 .../physical/pop/base/AbstractSingle.java       |  48 ----
 .../common/physical/pop/base/AbstractStore.java |  42 ---
 .../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 --
 .../physical/pop/base/PhysicalOperator.java     |  59 ----
 .../physical/pop/base/PhysicalOperatorUtil.java |  34 ---
 .../physical/pop/base/PhysicalVisitor.java      |  43 ---
 .../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 --
 .../drill/common/physical/MockScanPOP.java      |  69 -----
 .../drill/common/physical/MockStorePOP.java     |  62 -----
 .../common/physical/ParsePhysicalPlan.java      |  41 ---
 .../common/src/test/resources/drill-module.conf |   1 -
 .../src/test/resources/physical_test1.json      |  33 ---
 sandbox/prototype/exec/java-exec/pom.xml        |   4 +-
 .../java/org/apache/drill/exec/ByteReorder.java |  54 ----
 .../drill/exec/cache/DistributedCache.java      |  14 +-
 .../org/apache/drill/exec/cache/HazelCache.java |  87 ++----
 .../org/apache/drill/exec/cache/LocalCache.java |  55 ++++
 .../apache/drill/exec/cache/ProtoBufImpl.java   |  50 ++++
 .../apache/drill/exec/cache/ProtoBufWrap.java   |  69 +++++
 .../org/apache/drill/exec/cache/ProtoMap.java   |  52 ++++
 .../exec/cache/TemplatizedLogicalPlan.java      |  22 --
 .../exec/cache/TemplatizedPhysicalPlan.java     |  22 --
 .../apache/drill/exec/client/DrillClient.java   |  73 ++++-
 .../drill/exec/coord/ClusterCoordinator.java    |   9 +-
 .../exec/coord/DrillServiceInstanceHelper.java  |   4 +-
 .../drill/exec/coord/DrillbitEndpointSerDe.java |  65 +++++
 .../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 +
 .../drill/exec/foreman/CancelableQuery.java     |  22 --
 .../drill/exec/foreman/ExecutionPlanner.java    |  24 --
 .../org/apache/drill/exec/foreman/Foreman.java  |  39 ---
 .../drill/exec/foreman/QueryWorkUnit.java       |  54 ----
 .../drill/exec/foreman/ResourceRequest.java     |  30 --
 .../drill/exec/foreman/StatusProvider.java      |  24 --
 .../drill/exec/memory/BufferAllocator.java      |   5 +-
 .../exec/metrics/SingleThreadNestedCounter.java |  22 +-
 .../ops/FilteringRecordBatchTransformer.java    |  58 ----
 .../apache/drill/exec/ops/FragmentContext.java  |  54 +++-
 .../drill/exec/ops/FragmentConverter.java       |  30 --
 .../org/apache/drill/exec/ops/FragmentRoot.java |  37 ---
 .../apache/drill/exec/ops/OperatorFactory.java  |  22 --
 .../apache/drill/exec/ops/OutputMutator.java    |  28 --
 .../org/apache/drill/exec/ops/QueryContext.java |  27 +-
 .../org/apache/drill/exec/ops/ScanBatch.java    | 157 -----------
 .../drill/exec/ops/StreamingRecordBatch.java    |  25 --
 .../exec/ops/exchange/ExchangeRecordBatch.java  |  22 --
 .../exec/ops/exchange/RecordBatchSender.java    |  24 --
 .../exec/ops/filter/FilterRecordBatch.java      | 109 --------
 .../exec/ops/filter/SelectionVectorUpdater.java |  80 ------
 .../drill/exec/opt/IdentityOptimizer.java       |   3 +-
 .../org/apache/drill/exec/opt/Optimizer.java    |  45 +++
 .../drill/exec/physical/DataValidationMode.java |  24 ++
 .../drill/exec/physical/EndpointAffinity.java   |  60 ++++
 .../drill/exec/physical/OperatorCost.java       |  66 +++++
 .../drill/exec/physical/PhysicalPlan.java       |  94 +++++++
 .../apache/drill/exec/physical/ReadEntry.java   |  33 +++
 .../apache/drill/exec/physical/RecordField.java |  60 ++++
 .../apache/drill/exec/physical/WriteEntry.java  |  22 ++
 .../drill/exec/physical/base/AbstractBase.java  |  43 +++
 .../exec/physical/base/AbstractExchange.java    |  90 ++++++
 .../physical/base/AbstractPhysicalVisitor.java  | 124 +++++++++
 .../exec/physical/base/AbstractReceiver.java    |  63 +++++
 .../drill/exec/physical/base/AbstractScan.java  |  84 ++++++
 .../exec/physical/base/AbstractSender.java      |  53 ++++
 .../exec/physical/base/AbstractSingle.java      |  68 +++++
 .../drill/exec/physical/base/AbstractStore.java |  36 +++
 .../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 +++
 .../apache/drill/exec/physical/base/Leaf.java   |  25 ++
 .../exec/physical/base/PhysicalOperator.java    |  80 ++++++
 .../physical/base/PhysicalOperatorUtil.java     |  34 +++
 .../exec/physical/base/PhysicalVisitor.java     |  61 +++++
 .../drill/exec/physical/base/Receiver.java      |  51 ++++
 .../apache/drill/exec/physical/base/Root.java   |  24 ++
 .../apache/drill/exec/physical/base/Scan.java   |  36 +++
 .../apache/drill/exec/physical/base/Sender.java |  44 +++
 .../apache/drill/exec/physical/base/Size.java   |  48 ++++
 .../apache/drill/exec/physical/base/Store.java  |  74 +++++
 .../drill/exec/physical/config/Filter.java      |  75 +++++
 .../physical/config/HashPartitionSender.java    |  58 ++++
 .../physical/config/HashToRandomExchange.java   |  86 ++++++
 .../exec/physical/config/MockRecordReader.java  | 113 ++++++++
 .../physical/config/MockScanBatchCreator.java   |  49 ++++
 .../drill/exec/physical/config/MockScanPOP.java | 193 +++++++++++++
 .../exec/physical/config/MockStorageEngine.java |  58 ++++
 .../exec/physical/config/MockStorePOP.java      |  75 +++++
 .../exec/physical/config/PartitionRange.java    |  47 ++++
 .../drill/exec/physical/config/Project.java     |  72 +++++
 .../exec/physical/config/RandomReceiver.java    |  83 ++++++
 .../drill/exec/physical/config/RangeSender.java |  72 +++++
 .../drill/exec/physical/config/Screen.java      | 106 ++++++++
 .../exec/physical/config/SingleSender.java      |  78 ++++++
 .../apache/drill/exec/physical/config/Sort.java |  86 ++++++
 .../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 ++
 .../exec/physical/impl/PhysicalConfig.java      |  29 ++
 .../physical/impl/RandomReceiverCreator.java    |  46 ++++
 .../drill/exec/physical/impl/RootCreator.java   |  31 +++
 .../drill/exec/physical/impl/RootExec.java      |  40 +++
 .../drill/exec/physical/impl/ScanBatch.java     | 172 ++++++++++++
 .../drill/exec/physical/impl/ScreenCreator.java |  90 ++++++
 .../exec/physical/impl/SingleSenderCreator.java |  89 ++++++
 .../exec/physical/impl/WireRecordBatch.java     |  99 +++++++
 .../impl/materialize/QueryWritableBatch.java    |  46 ++++
 .../impl/materialize/RecordMaterializer.java    |  31 +++
 .../materialize/VectorRecordMaterializer.java   |  52 ++++
 .../exec/planner/AbstractOpWrapperVisitor.java  |  45 +++
 .../apache/drill/exec/planner/ExecPlanner.java  |   8 +-
 .../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 ---
 .../drill/exec/planner/FragmentStats.java       |  63 -----
 .../exec/planner/FragmentStatsCollector.java    | 109 --------
 .../drill/exec/planner/FragmentVisitor.java     |  22 --
 .../drill/exec/planner/FragmentWrapper.java     | 127 ---------
 .../planner/FragmentingPhysicalVisitor.java     |  71 -----
 .../exec/planner/MaterializedFragment.java      |  69 -----
 .../drill/exec/planner/PhysicalPlanReader.java  |  63 ++++-
 .../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 +++++
 .../exec/planner/fragment/Materializer.java     | 107 ++++++++
 .../exec/planner/fragment/PlanningSet.java      |  66 +++++
 .../planner/fragment/SimpleParallelizer.java    | 163 +++++++++++
 .../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 ----
 .../exec/pop/sender/HashPartitionSender.java    |  49 ----
 .../apache/drill/exec/record/BatchSchema.java   | 102 +------
 .../exec/record/FragmentWritableBatch.java      |  59 ++++
 .../drill/exec/record/InvalidValueAccessor.java |   2 +-
 .../drill/exec/record/MajorTypeSerDe.java       | 116 ++++++++
 .../drill/exec/record/MaterializedField.java    | 167 +++++++++---
 .../drill/exec/record/RawFragmentBatch.java     |  44 +++
 .../exec/record/RawFragmentBatchProvider.java   |  27 ++
 .../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 +-
 .../exec/record/vector/BaseValueVector.java     |  86 ++++--
 .../apache/drill/exec/record/vector/Bit.java    | 168 ++++++++++++
 .../drill/exec/record/vector/BitVector.java     | 166 -----------
 .../drill/exec/record/vector/ByteVector.java    |  48 ----
 .../apache/drill/exec/record/vector/Fixed1.java |  43 +++
 .../drill/exec/record/vector/Fixed12.java       |  35 +++
 .../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 ++++
 .../drill/exec/record/vector/FixedLen.java      |  45 +++
 .../drill/exec/record/vector/Int16Vector.java   |  52 ----
 .../drill/exec/record/vector/Int32Vector.java   |  52 ----
 .../exec/record/vector/NullableFixed4.java      |  37 +++
 .../exec/record/vector/NullableInt32Vector.java |  47 ----
 .../exec/record/vector/NullableValueVector.java |  36 ++-
 .../drill/exec/record/vector/RepeatMap.java     |  57 ++++
 .../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 +++-
 .../drill/exec/record/vector/VarLen1.java       |  36 +++
 .../drill/exec/record/vector/VarLen2.java       |  36 +++
 .../drill/exec/record/vector/VarLen4.java       |  36 +++
 .../exec/record/vector/VariableVector.java      |  35 ++-
 .../exec/rpc/AbstractHandshakeHandler.java      |  57 ++++
 .../java/org/apache/drill/exec/rpc/Acks.java    |  27 ++
 .../org/apache/drill/exec/rpc/BasicClient.java  | 137 ++++++++--
 .../exec/rpc/BasicClientWithConnection.java     |  64 +++++
 .../org/apache/drill/exec/rpc/BasicServer.java  |  61 ++++-
 .../drill/exec/rpc/CoordinationQueue.java       |  32 ++-
 .../apache/drill/exec/rpc/DrillRpcFuture.java   |  78 +-----
 .../drill/exec/rpc/DrillRpcFutureImpl.java      | 118 ++++++++
 .../drill/exec/rpc/InboundRpcMessage.java       |  13 +-
 .../drill/exec/rpc/OutboundRpcMessage.java      |  41 ++-
 .../apache/drill/exec/rpc/RemoteConnection.java |  45 +++
 .../org/apache/drill/exec/rpc/Response.java     |   8 +-
 .../java/org/apache/drill/exec/rpc/RpcBus.java  | 136 ++++++----
 .../org/apache/drill/exec/rpc/RpcConfig.java    | 150 ++++++++++
 .../org/apache/drill/exec/rpc/RpcDecoder.java   |  14 +-
 .../org/apache/drill/exec/rpc/RpcEncoder.java   |  15 +-
 .../drill/exec/rpc/RpcExceptionHandler.java     |  21 +-
 .../org/apache/drill/exec/rpc/RpcMessage.java   |   9 +-
 .../drill/exec/rpc/RpcOutcomeListener.java      |  28 ++
 .../exec/rpc/ZeroCopyProtobufLengthDecoder.java |  16 +-
 .../exec/rpc/bit/AvailabilityListener.java      |  22 ++
 .../apache/drill/exec/rpc/bit/BitClient.java    |  77 ++++--
 .../org/apache/drill/exec/rpc/bit/BitCom.java   |  75 ++---
 .../rpc/bit/BitComDefaultInstanceHandler.java   |  51 ++++
 .../drill/exec/rpc/bit/BitComHandler.java       | 136 ----------
 .../apache/drill/exec/rpc/bit/BitComImpl.java   | 194 +++++++------
 .../drill/exec/rpc/bit/BitConnection.java       | 168 ++++++++++++
 .../exec/rpc/bit/BitConnectionManager.java      |  80 ++++++
 .../apache/drill/exec/rpc/bit/BitRpcConfig.java |  46 ++++
 .../apache/drill/exec/rpc/bit/BitServer.java    |  61 +++--
 .../apache/drill/exec/rpc/bit/BitTunnel.java    | 215 +++++++++++++--
 .../apache/drill/exec/rpc/bit/ListenerPool.java |  56 ++++
 .../drill/exec/rpc/user/QueryResultBatch.java   |  49 ++++
 .../apache/drill/exec/rpc/user/UserClient.java  | 169 ++++++++++--
 .../exec/rpc/user/UserResultsListener.java      |  41 +++
 .../drill/exec/rpc/user/UserRpcConfig.java      |  39 +++
 .../apache/drill/exec/rpc/user/UserServer.java  |  93 +++++--
 .../drill/exec/server/BootStrapContext.java     |  68 +++++
 .../org/apache/drill/exec/server/Drillbit.java  |  65 +++--
 .../drill/exec/server/DrillbitContext.java      |  58 ++--
 .../drill/exec/server/RemoteServiceSet.java     |  59 ++++
 .../drill/exec/service/ServiceEngine.java       |  45 ++-
 .../drill/exec/store/AbstractStorageEngine.java |   2 +-
 .../apache/drill/exec/store/RecordReader.java   |   7 +-
 .../apache/drill/exec/store/StorageEngine.java  |   2 +-
 .../org/apache/drill/exec/util/AtomicState.java |  58 ++++
 .../work/AbstractFragmentRunnerListener.java    | 109 ++++++++
 .../apache/drill/exec/work/CancelableQuery.java |  22 ++
 .../drill/exec/work/EndpointListener.java       |  45 +++
 .../apache/drill/exec/work/FragmentRunner.java  | 124 +++++++++
 .../drill/exec/work/FragmentRunnerListener.java |  28 ++
 .../apache/drill/exec/work/QueryWorkUnit.java   |  64 +++++
 .../drill/exec/work/RecordOutputStream.java     |  22 ++
 .../work/RemotingFragmentRunnerListener.java    |  46 ++++
 .../apache/drill/exec/work/ResourceRequest.java |  30 ++
 .../apache/drill/exec/work/RootNodeDriver.java  |  25 ++
 .../apache/drill/exec/work/StatusProvider.java  |  24 ++
 .../org/apache/drill/exec/work/WorkManager.java | 168 ++++++++++++
 .../work/batch/AbstractFragmentCollector.java   |  84 ++++++
 .../drill/exec/work/batch/BatchCollector.java   |  32 +++
 .../drill/exec/work/batch/BitComHandler.java    |  41 +++
 .../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 +++++++++++++++++++
 .../work/foreman/FragmentStatusListener.java    |  26 ++
 .../work/foreman/RunningFragmentManager.java    | 266 ++++++++++++++++++
 .../drill/exec/work/foreman/TunnelManager.java  |  53 ++++
 .../work/fragment/IncomingFragmentHandler.java  |  49 ++++
 .../work/fragment/LocalFragmentHandler.java     |  69 +++++
 .../work/fragment/RemoteFragmentHandler.java    | 123 +++++++++
 .../apache/drill/exec/work/user/UserWorker.java |  72 +++++
 .../src/main/protobuf/Coordination.proto        |  26 ++
 .../src/main/protobuf/ExecutionProtos.proto     |  55 ++--
 .../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 ++++
 .../apache/drill/exec/DrillSystemTestBase.java  |   7 +
 .../exec/client/DrillClientSystemTest.java      |  25 +-
 .../exec/compile/TestClassCompilationTypes.java |   5 +-
 .../exec/physical/config/ParsePhysicalPlan.java |  42 +++
 .../physical/impl/DistributedFragmentRun.java   |  53 ++++
 .../exec/physical/impl/SimpleFragmentRun.java   | 100 +++++++
 .../apache/drill/exec/pop/CheckFragmenter.java  |  70 ++---
 .../drill/exec/pop/CheckInjectionValue.java     |  12 +-
 .../apache/drill/exec/pop/FragmentChecker.java  |  66 +++++
 .../apache/drill/exec/pop/PopUnitTestBase.java  |  71 +++++
 .../drill/exec/rpc/user/RunRemoteQuery.java     |  41 ---
 .../apache/drill/exec/rpc/user/UserRpcTest.java | 107 --------
 .../apache/drill/exec/server/TestBitRpc.java    |  84 ++++++
 .../drill/exec/store/MockRecordConfig.java      |  46 ----
 .../drill/exec/store/MockRecordReader.java      | 108 --------
 .../drill/exec/store/MockStorageEngine.java     |  54 ----
 .../src/test/resources/drill-module.conf        |   5 +-
 .../java-exec/src/test/resources/logback.xml    |  10 +-
 .../src/test/resources/physical_screen.json     |   5 +-
 .../test/resources/physical_simpleexchange.json |  36 ++-
 .../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;
-  }
-  
-}