cross merge fullstack_release_candidate into trunk
git-svn-id: https://hyracks.googlecode.com/svn/trunk@3208 123451ca-8445-de46-9d55-352943316053
diff --git a/fullstack/hyracks/hyracks-api/pom.xml b/fullstack/hyracks/hyracks-api/pom.xml
index 72f0d8b..6807f76 100644
--- a/fullstack/hyracks/hyracks-api/pom.xml
+++ b/fullstack/hyracks/hyracks-api/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
index a8f2fda..a4f0b29 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IInputChannel {
@@ -30,7 +30,7 @@
public void recycleBuffer(ByteBuffer buffer);
- public void open(IHyracksTaskContext ctx) throws HyracksDataException;
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException;
public void close() throws HyracksDataException;
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index e34e60d..cd2b698 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -17,6 +17,8 @@
import java.io.Serializable;
import java.util.EnumSet;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -30,6 +32,10 @@
CREATE_JOB,
GET_JOB_STATUS,
START_JOB,
+ GET_DATASET_DIRECTORY_SERIVICE_INFO,
+ GET_DATASET_RESULT_STATUS,
+ GET_DATASET_RECORD_DESCRIPTOR,
+ GET_DATASET_RESULT_LOCATIONS,
WAIT_FOR_COMPLETION,
GET_NODE_CONTROLLERS_INFO
}
@@ -156,6 +162,74 @@
}
}
+ public static class GetDatasetDirectoryServiceInfoFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_DIRECTORY_SERIVICE_INFO;
+ }
+ }
+
+ public static class GetDatasetResultStatusFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ public GetDatasetResultStatusFunction(JobId jobId, ResultSetId rsId) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_RESULT_STATUS;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+ }
+
+ public static class GetDatasetResultLocationsFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final DatasetDirectoryRecord[] knownRecords;
+
+ public GetDatasetResultLocationsFunction(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.knownRecords = knownRecords;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_RESULT_LOCATIONS;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public DatasetDirectoryRecord[] getKnownRecords() {
+ return knownRecords;
+ }
+ }
+
public static class WaitForCompletionFunction extends Function {
private static final long serialVersionUID = 1L;
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 4c06d42..2ab42c0 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -17,6 +17,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -76,6 +77,12 @@
}
@Override
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf = new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
+ return (NetworkAddress) rpci.call(ipcHandle, gddsf);
+ }
+
+ @Override
public void waitForCompletion(JobId jobId) throws Exception {
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
jobId);
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 227524c..e0fafb0 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -26,6 +26,7 @@
import org.apache.http.impl.client.DefaultHttpClient;
import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
@@ -118,6 +119,10 @@
return hci.startJob(appName, JavaSerializationUtils.serialize(acggf), jobFlags);
}
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+ return hci.getDatasetDirectoryServiceInfo();
+ }
+
@Override
public void waitForCompletion(JobId jobId) throws Exception {
hci.waitForCompletion(jobId);
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index bdbb544..6333c22 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -18,6 +18,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -100,6 +101,14 @@
throws Exception;
/**
+ * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
+ *
+ * @return {@link NetworkAddress}
+ * @throws Exception
+ */
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
+ /**
* Waits until the specified job has completed, either successfully or has
* encountered a permanent failure.
*
@@ -123,4 +132,4 @@
* @throws Exception
*/
public ClusterTopology getClusterTopology() throws Exception;
-}
\ No newline at end of file
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index ef5906e..22b0a8f 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -17,6 +17,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -35,6 +36,8 @@
public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
public void waitForCompletion(JobId jobId) throws Exception;
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index fd9218a..73b5488 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -27,10 +27,14 @@
private final NetworkAddress netAddress;
- public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress) {
+ private final NetworkAddress datasetNetworkAddress;
+
+ public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
+ NetworkAddress datasetNetworkAddress) {
this.nodeId = nodeId;
this.status = status;
this.netAddress = netAddress;
+ this.datasetNetworkAddress = datasetNetworkAddress;
}
public String getNodeId() {
@@ -44,4 +48,8 @@
public NetworkAddress getNetworkAddress() {
return netAddress;
}
+
+ public NetworkAddress getDatasetNetworkAddress() {
+ return datasetNetworkAddress;
+ }
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
index f36b7b3..0eac9a2 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/impl/JobSpecificationActivityClusterGraphGeneratorFactory.java
@@ -17,6 +17,7 @@
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.rewriter.ActivityClusterGraphRewriter;
public class JobSpecificationActivityClusterGraphGeneratorFactory implements IActivityClusterGraphGeneratorFactory {
private static final long serialVersionUID = 1L;
@@ -78,6 +79,8 @@
return new IActivityClusterGraphGenerator() {
@Override
public ActivityClusterGraph initialize() {
+ ActivityClusterGraphRewriter rewriter = new ActivityClusterGraphRewriter();
+ rewriter.rewrite(acg);
return acg;
}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index e964d66..a2ee977 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.api.context;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
@@ -28,5 +29,7 @@
public ICounterContext getCounterContext();
+ public IDatasetPartitionManager getDatasetPartitionManager();
+
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception;
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
new file mode 100644
index 0000000..ba2ff9a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResultSerializer extends Serializable {
+ /**
+ * Initializes the serializer.
+ */
+ public void init() throws HyracksDataException;
+
+ /**
+ * Method to serialize the result and append it to the provided output stream
+ *
+ * @param tAccess
+ * - A frame tuple accessor object that contains the original data to be serialized
+ * @param tIdx
+ * - Index of the tuple that should be serialized.
+ * @return true if the tuple was appended successfully, else false.
+ */
+ public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
new file mode 100644
index 0000000..1fbf00f
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+
+public interface IResultSerializerFactory extends Serializable {
+ /**
+ * Creates a result serialized appender
+ *
+ * @param printStream
+ * - A print stream object to which the serialized results will be written.
+ * @return A new instance of result serialized appender.
+ */
+ public IResultSerializer createResultSerializer(RecordDescriptor recordDesc, PrintStream printStream);
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
new file mode 100644
index 0000000..1eca502
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataflow.value;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public interface JSONSerializable {
+ /**
+ * Returns the JSON representation of the object.
+ *
+ * @return A new JSONObject instance representing this Java object.
+ */
+ public JSONObject toJSON() throws JSONException;
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java
new file mode 100644
index 0000000..6316bba
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/DatasetDirectoryRecord.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+public class DatasetDirectoryRecord implements Serializable {
+ public enum Status {
+ IDLE,
+ RUNNING,
+ SUCCESS,
+ FAILED
+ }
+
+ private static final long serialVersionUID = 1L;
+
+ private NetworkAddress address;
+
+ private boolean readEOS;
+
+ private Status status;
+
+ public DatasetDirectoryRecord() {
+ this.address = null;
+ this.readEOS = false;
+ this.status = Status.IDLE;
+ }
+
+ public void setNetworkAddress(NetworkAddress address) {
+ this.address = address;
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return address;
+ }
+
+ public void readEOS() {
+ this.readEOS = true;
+ }
+
+ public boolean hasReachedReadEOS() {
+ return readEOS;
+ }
+
+ public void start() {
+ status = Status.RUNNING;
+ }
+
+ public void writeEOS() {
+ status = Status.SUCCESS;
+ }
+
+ public void fail() {
+ status = Status.FAILED;
+ }
+
+ public Status getStatus() {
+ return status;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof DatasetDirectoryRecord)) {
+ return false;
+ }
+ return address.equals(((DatasetDirectoryRecord) o).address);
+ }
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
new file mode 100644
index 0000000..5266333
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetDirectoryService.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetDirectoryService {
+ public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+ int nPartitions, NetworkAddress networkAddress);
+
+ public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition);
+
+ public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition);
+
+ public Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException;
+
+ public DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownLocations) throws HyracksDataException;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
new file mode 100644
index 0000000..65ba1c7
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetInputChannelMonitor.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+
+public interface IDatasetInputChannelMonitor extends IInputChannelMonitor {
+ public boolean eosReached();
+
+ public boolean failed();
+
+ public int getNFramesAvailable();
+
+ public void notifyFrameRead();
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
new file mode 100644
index 0000000..ae38c7f
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionManager.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IDatasetPartitionManager {
+ public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions) throws HyracksException;
+
+ public void reportPartitionWriteCompletion(JobId jobId, ResultSetId resultSetId, int partition)
+ throws HyracksException;
+
+ public void reportPartitionFailure(JobId jobId, ResultSetId resultSetId, int partition) throws HyracksException;
+
+ public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter noc) throws HyracksException;
+
+ public IWorkspaceFileFactory getFileFactory();
+
+ public void close();
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
new file mode 100644
index 0000000..8f5ed64
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionReader.java
@@ -0,0 +1,21 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+
+public interface IDatasetPartitionReader {
+ public void writeTo(IFrameWriter writer);
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
new file mode 100644
index 0000000..42dc157
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IDatasetPartitionWriter.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IDatasetPartitionWriter extends IFrameWriter {
+ public Page returnPage() throws HyracksDataException;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
new file mode 100644
index 0000000..4a7a6b0
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDataset.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDataset {
+ public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..d49d5cd
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceConnection.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceConnection {
+ /**
+ * Gets the result status for the given result set.
+ *
+ * @param jobId
+ * ID of the job
+ * @param rsId
+ * ID of the result set
+ * @return {@link Status}
+ * @throws Exception
+ */
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+ /**
+ * Gets the IP Addresses and ports for the partition generating the result for each location.
+ *
+ * @param jobId
+ * ID of the job
+ * @param rsId
+ * ID of the result set
+ * @param knownRecords
+ * Locations that are already known to the client
+ * @return {@link NetworkAddress[]}
+ * @throws Exception
+ */
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
new file mode 100644
index 0000000..ba21a84
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetDirectoryServiceInterface.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public interface IHyracksDatasetDirectoryServiceInterface {
+ /**
+ * Gets the result status for the given result set.
+ *
+ * @param jobId
+ * ID of the job
+ * @param rsId
+ * ID of the result set
+ * @return {@link Status}
+ * @throws Exception
+ */
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception;
+
+ /**
+ * Gets the IP Addresses and ports for the partition generating the result for each location.
+ *
+ * @param jobId
+ * ID of the job
+ * @param rsId
+ * ID of the result set
+ * @param knownRecords
+ * Locations from the dataset directory that are already known to the client
+ * @return {@link NetworkAddress[]}
+ * @throws Exception
+ */
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
new file mode 100644
index 0000000..b928a49
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHyracksDatasetReader {
+ public Status getResultStatus();
+
+ public int read(ByteBuffer buffer) throws HyracksDataException;
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
new file mode 100644
index 0000000..7275dfd
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/Page.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.nio.ByteBuffer;
+
+public class Page {
+ private final ByteBuffer buffer;
+
+ public Page(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ public ByteBuffer clear() {
+ return (ByteBuffer) buffer.clear();
+ }
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
new file mode 100644
index 0000000..ae38ef3
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/ResultSetId.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.dataset;
+
+import java.io.Serializable;
+
+public class ResultSetId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final long id;
+
+ public ResultSetId(long id) {
+ this.id = id;
+ }
+
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ return (int) id;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == this) {
+ return true;
+ }
+ if (!(o instanceof ResultSetId)) {
+ return false;
+ }
+ return ((ResultSetId) o).id == id;
+ }
+
+ @Override
+ public String toString() {
+ return "RSID:" + id;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
index 6698ff7..9fb2b08 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/ActivityCluster.java
@@ -33,7 +33,7 @@
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-public final class ActivityCluster implements Serializable {
+public class ActivityCluster implements Serializable {
private static final long serialVersionUID = 1L;
private final ActivityClusterGraph acg;
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 7c523f1..1fdff0f 100644
--- a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -34,12 +34,15 @@
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
private static final long serialVersionUID = 1L;
private final List<OperatorDescriptorId> roots;
+ private final List<ResultSetId> resultSetIds;
+
private final Map<OperatorDescriptorId, IOperatorDescriptor> opMap;
private final Map<ConnectorDescriptorId, IConnectorDescriptor> connMap;
@@ -72,6 +75,7 @@
public JobSpecification() {
roots = new ArrayList<OperatorDescriptorId>();
+ resultSetIds = new ArrayList<ResultSetId>();
opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
@@ -104,6 +108,10 @@
roots.add(op.getOperatorId());
}
+ public void addResultSetId(ResultSetId rsId) {
+ resultSetIds.add(rsId);
+ }
+
public void connect(IConnectorDescriptor conn, IOperatorDescriptor producerOp, int producerPort,
IOperatorDescriptor consumerOp, int consumerPort) {
insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
@@ -208,6 +216,10 @@
return roots;
}
+ public List<ResultSetId> getResultSetIds() {
+ return resultSetIds;
+ }
+
public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
return connectorPolicyAssignmentPolicy;
}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
new file mode 100644
index 0000000..148a8a2
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.api.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class ResultSetPartitionId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId resultSetId;
+
+ private final int partition;
+
+ public ResultSetPartitionId(JobId jobId, ResultSetId resultSetId, int partition) {
+ this.jobId = jobId;
+ this.resultSetId = resultSetId;
+ this.partition = partition;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return resultSetId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((resultSetId == null) ? 0 : resultSetId.hashCode());
+ result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+ result = prime * result + partition;
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ResultSetPartitionId other = (ResultSetPartitionId) obj;
+ if (resultSetId == null) {
+ if (other.resultSetId != null)
+ return false;
+ } else if (!resultSetId.equals(other.resultSetId))
+ return false;
+ if (jobId == null) {
+ if (other.jobId != null)
+ return false;
+ } else if (!jobId.equals(other.jobId))
+ return false;
+ if (partition != other.partition)
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return jobId.toString() + ":" + resultSetId + ":" + partition;
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java
new file mode 100644
index 0000000..c6761e9
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/ActivityClusterGraphRewriter.java
@@ -0,0 +1,381 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+import edu.uci.ics.hyracks.api.rewriter.runtime.SuperActivity;
+
+/**
+ * This class rewrite the AcivityClusterGraph to eliminate
+ * all one-to-one connections and merge one-to-one connected
+ * DAGs into super activities.
+ * </p>
+ * Each super activity internally maintains a DAG and execute it at the runtime.
+ *
+ * @author yingyib
+ */
+public class ActivityClusterGraphRewriter {
+ private static String ONE_TO_ONE_CONNECTOR = "edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor";
+
+ /**
+ * rewrite an activity cluster graph to eliminate
+ * all one-to-one connections and merge one-to-one connected
+ * DAGs into super activities.
+ *
+ * @param acg
+ * the activity cluster graph
+ */
+ public void rewrite(ActivityClusterGraph acg) {
+ acg.getActivityMap().clear();
+ acg.getConnectorMap().clear();
+ Map<IActivity, SuperActivity> invertedActivitySuperActivityMap = new HashMap<IActivity, SuperActivity>();
+ for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+ rewriteIntraActivityCluster(entry.getValue(), invertedActivitySuperActivityMap);
+ }
+ for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+ rewriteInterActivityCluster(entry.getValue(), invertedActivitySuperActivityMap);
+ }
+ invertedActivitySuperActivityMap.clear();
+ }
+
+ /**
+ * rewrite the blocking relationship among activity cluster
+ *
+ * @param ac
+ * the activity cluster to be rewritten
+ */
+ private void rewriteInterActivityCluster(ActivityCluster ac,
+ Map<IActivity, SuperActivity> invertedActivitySuperActivityMap) {
+ Map<ActivityId, Set<ActivityId>> blocked2BlockerMap = ac.getBlocked2BlockerMap();
+ Map<ActivityId, ActivityId> invertedAid2SuperAidMap = new HashMap<ActivityId, ActivityId>();
+ for (Entry<IActivity, SuperActivity> entry : invertedActivitySuperActivityMap.entrySet()) {
+ invertedAid2SuperAidMap.put(entry.getKey().getActivityId(), entry.getValue().getActivityId());
+ }
+ Map<ActivityId, Set<ActivityId>> replacedBlocked2BlockerMap = new HashMap<ActivityId, Set<ActivityId>>();
+ for (Entry<ActivityId, Set<ActivityId>> entry : blocked2BlockerMap.entrySet()) {
+ ActivityId blocked = entry.getKey();
+ ActivityId replacedBlocked = invertedAid2SuperAidMap.get(blocked);
+ Set<ActivityId> blockers = entry.getValue();
+ Set<ActivityId> replacedBlockers = null;
+ if (blockers != null) {
+ replacedBlockers = new HashSet<ActivityId>();
+ for (ActivityId blocker : blockers) {
+ replacedBlockers.add(invertedAid2SuperAidMap.get(blocker));
+ ActivityCluster dependingAc = ac.getActivityClusterGraph().getActivityMap()
+ .get(invertedAid2SuperAidMap.get(blocker));
+ if (!ac.getDependencies().contains(dependingAc)) {
+ ac.getDependencies().add(dependingAc);
+ }
+ }
+ }
+ if (replacedBlockers != null) {
+ Set<ActivityId> existingBlockers = replacedBlocked2BlockerMap.get(replacedBlocked);
+ if (existingBlockers == null) {
+ replacedBlocked2BlockerMap.put(replacedBlocked, replacedBlockers);
+ } else {
+ existingBlockers.addAll(replacedBlockers);
+ replacedBlocked2BlockerMap.put(replacedBlocked, existingBlockers);
+ }
+ }
+ }
+ blocked2BlockerMap.clear();
+ blocked2BlockerMap.putAll(replacedBlocked2BlockerMap);
+ }
+
+ /**
+ * rewrite an activity cluster internally
+ *
+ * @param ac
+ * the activity cluster to be rewritten
+ */
+ private void rewriteIntraActivityCluster(ActivityCluster ac,
+ Map<IActivity, SuperActivity> invertedActivitySuperActivityMap) {
+ Map<ActivityId, IActivity> activities = ac.getActivityMap();
+ Map<ActivityId, List<IConnectorDescriptor>> activityInputMap = ac.getActivityInputMap();
+ Map<ActivityId, List<IConnectorDescriptor>> activityOutputMap = ac.getActivityOutputMap();
+ Map<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> connectorActivityMap = ac
+ .getConnectorActivityMap();
+ ActivityClusterGraph acg = ac.getActivityClusterGraph();
+ Map<ActivityId, IActivity> startActivities = new HashMap<ActivityId, IActivity>();
+ Map<ActivityId, SuperActivity> superActivities = new HashMap<ActivityId, SuperActivity>();
+ Map<ActivityId, Queue<IActivity>> toBeExpendedMap = new HashMap<ActivityId, Queue<IActivity>>();
+
+ /**
+ * Build the initial super activities
+ */
+ for (Entry<ActivityId, IActivity> entry : activities.entrySet()) {
+ ActivityId activityId = entry.getKey();
+ IActivity activity = entry.getValue();
+ if (activityInputMap.get(activityId) == null) {
+ startActivities.put(activityId, activity);
+ /**
+ * use the start activity's id as the id of the super activity
+ */
+ createNewSuperActivity(ac, superActivities, toBeExpendedMap, invertedActivitySuperActivityMap,
+ activityId, activity);
+ }
+ }
+
+ /**
+ * expand one-to-one connected activity cluster by the BFS order.
+ * after the while-loop, the original activities are partitioned
+ * into equivalent classes, one-per-super-activity.
+ */
+ Map<ActivityId, SuperActivity> clonedSuperActivities = new HashMap<ActivityId, SuperActivity>();
+ while (toBeExpendedMap.size() > 0) {
+ clonedSuperActivities.clear();
+ clonedSuperActivities.putAll(superActivities);
+ for (Entry<ActivityId, SuperActivity> entry : clonedSuperActivities.entrySet()) {
+ ActivityId superActivityId = entry.getKey();
+ SuperActivity superActivity = entry.getValue();
+
+ /**
+ * for the case where the super activity has already been swallowed
+ */
+ if (superActivities.get(superActivityId) == null) {
+ continue;
+ }
+
+ /**
+ * expend the super activity
+ */
+ Queue<IActivity> toBeExpended = toBeExpendedMap.get(superActivityId);
+ if (toBeExpended == null) {
+ /**
+ * Nothing to expand
+ */
+ continue;
+ }
+ IActivity expendingActivity = toBeExpended.poll();
+ List<IConnectorDescriptor> outputConnectors = activityOutputMap.get(expendingActivity.getActivityId());
+ if (outputConnectors != null) {
+ for (IConnectorDescriptor outputConn : outputConnectors) {
+ Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> endPoints = connectorActivityMap
+ .get(outputConn.getConnectorId());
+ IActivity newActivity = endPoints.getRight().getLeft();
+ SuperActivity existingSuperActivity = invertedActivitySuperActivityMap.get(newActivity);
+ if (outputConn.getClass().getName().contains(ONE_TO_ONE_CONNECTOR)) {
+ /**
+ * expend the super activity cluster on an one-to-one out-bound connection
+ */
+ if (existingSuperActivity == null) {
+ superActivity.addActivity(newActivity);
+ toBeExpended.add(newActivity);
+ invertedActivitySuperActivityMap.put(newActivity, superActivity);
+ } else {
+ /**
+ * the two activities already in the same super activity
+ */
+ if (existingSuperActivity == superActivity) {
+ continue;
+ }
+ /**
+ * swallow an existing super activity
+ */
+ swallowExistingSuperActivity(superActivities, toBeExpendedMap,
+ invertedActivitySuperActivityMap, superActivity, superActivityId,
+ existingSuperActivity);
+ }
+ } else {
+ if (existingSuperActivity == null) {
+ /**
+ * create new activity
+ */
+ createNewSuperActivity(ac, superActivities, toBeExpendedMap,
+ invertedActivitySuperActivityMap, newActivity.getActivityId(), newActivity);
+ }
+ }
+ }
+ }
+
+ /**
+ * remove the to-be-expended queue if it is empty
+ */
+ if (toBeExpended.size() == 0) {
+ toBeExpendedMap.remove(superActivityId);
+ }
+ }
+ }
+
+ Map<ConnectorDescriptorId, IConnectorDescriptor> connMap = ac.getConnectorMap();
+ Map<ConnectorDescriptorId, RecordDescriptor> connRecordDesc = ac.getConnectorRecordDescriptorMap();
+ Map<SuperActivity, Integer> superActivityProducerPort = new HashMap<SuperActivity, Integer>();
+ Map<SuperActivity, Integer> superActivityConsumerPort = new HashMap<SuperActivity, Integer>();
+ for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+ superActivityProducerPort.put(entry.getValue(), 0);
+ superActivityConsumerPort.put(entry.getValue(), 0);
+ }
+
+ /**
+ * create a new activity cluster to replace the old activity cluster
+ */
+ ActivityCluster newActivityCluster = new ActivityCluster(acg, ac.getId());
+ newActivityCluster.setConnectorPolicyAssignmentPolicy(ac.getConnectorPolicyAssignmentPolicy());
+ for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+ newActivityCluster.addActivity(entry.getValue());
+ acg.getActivityMap().put(entry.getKey(), newActivityCluster);
+ }
+
+ /**
+ * Setup connectors: either inside a super activity or among super activities
+ */
+ for (Entry<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> entry : connectorActivityMap
+ .entrySet()) {
+ ConnectorDescriptorId connectorId = entry.getKey();
+ Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> endPoints = entry.getValue();
+ IActivity producerActivity = endPoints.getLeft().getLeft();
+ IActivity consumerActivity = endPoints.getRight().getLeft();
+ int producerPort = endPoints.getLeft().getRight();
+ int consumerPort = endPoints.getRight().getRight();
+ RecordDescriptor recordDescriptor = connRecordDesc.get(connectorId);
+ IConnectorDescriptor conn = connMap.get(connectorId);
+ if (conn.getClass().getName().contains(ONE_TO_ONE_CONNECTOR)) {
+ /**
+ * connection edge between inner activities
+ */
+ SuperActivity residingSuperActivity = invertedActivitySuperActivityMap.get(producerActivity);
+ residingSuperActivity.connect(conn, producerActivity, producerPort, consumerActivity, consumerPort,
+ recordDescriptor);
+ } else {
+ /**
+ * connection edge between super activities
+ */
+ SuperActivity producerSuperActivity = invertedActivitySuperActivityMap.get(producerActivity);
+ SuperActivity consumerSuperActivity = invertedActivitySuperActivityMap.get(consumerActivity);
+ int producerSAPort = superActivityProducerPort.get(producerSuperActivity);
+ int consumerSAPort = superActivityConsumerPort.get(consumerSuperActivity);
+ newActivityCluster.addConnector(conn);
+ newActivityCluster.connect(conn, producerSuperActivity, producerSAPort, consumerSuperActivity,
+ consumerSAPort, recordDescriptor);
+
+ /**
+ * bridge the port
+ */
+ producerSuperActivity.setClusterOutputIndex(producerSAPort, producerActivity.getActivityId(),
+ producerPort);
+ consumerSuperActivity.setClusterInputIndex(consumerSAPort, consumerActivity.getActivityId(),
+ consumerPort);
+ acg.getConnectorMap().put(connectorId, newActivityCluster);
+
+ /**
+ * increasing the port number for the producer and consumer
+ */
+ superActivityProducerPort.put(producerSuperActivity, ++producerSAPort);
+ superActivityConsumerPort.put(consumerSuperActivity, ++consumerSAPort);
+ }
+ }
+
+ /**
+ * Set up the roots of the new activity cluster
+ */
+ for (Entry<ActivityId, SuperActivity> entry : superActivities.entrySet()) {
+ List<IConnectorDescriptor> connIds = newActivityCluster.getActivityOutputMap().get(entry.getKey());
+ if (connIds == null || connIds.size() == 0) {
+ newActivityCluster.addRoot(entry.getValue());
+ }
+ }
+
+ /**
+ * set up the blocked2Blocker mapping, which will be updated in the rewriteInterActivityCluster call
+ */
+ newActivityCluster.getBlocked2BlockerMap().putAll(ac.getBlocked2BlockerMap());
+
+ /**
+ * replace the old activity cluster with the new activity cluster
+ */
+ acg.getActivityClusterMap().put(ac.getId(), newActivityCluster);
+ }
+
+ /**
+ * Create a new super activity
+ *
+ * @param acg
+ * the activity cluster
+ * @param superActivities
+ * the map from activity id to current super activities
+ * @param toBeExpendedMap
+ * the map from an existing super activity to its BFS expansion queue of the original activities
+ * @param invertedActivitySuperActivityMap
+ * the map from the original activities to their hosted super activities
+ * @param activityId
+ * the activity id for the new super activity, which is the first added acitivty's id in the super activity
+ * @param activity
+ * the first activity added to the new super activity
+ */
+ private void createNewSuperActivity(ActivityCluster acg, Map<ActivityId, SuperActivity> superActivities,
+ Map<ActivityId, Queue<IActivity>> toBeExpendedMap,
+ Map<IActivity, SuperActivity> invertedActivitySuperActivityMap, ActivityId activityId, IActivity activity) {
+ SuperActivity superActivity = new SuperActivity(acg.getActivityClusterGraph(), acg.getId(), activityId);
+ superActivities.put(activityId, superActivity);
+ superActivity.addActivity(activity);
+ Queue<IActivity> toBeExpended = new LinkedList<IActivity>();
+ toBeExpended.add(activity);
+ toBeExpendedMap.put(activityId, toBeExpended);
+ invertedActivitySuperActivityMap.put(activity, superActivity);
+ }
+
+ /**
+ * One super activity swallows another existing super activity.
+ *
+ * @param superActivities
+ * the map from activity id to current super activities
+ * @param toBeExpendedMap
+ * the map from an existing super activity to its BFS expansion queue of the original activities
+ * @param invertedActivitySuperActivityMap
+ * the map from the original activities to their hosted super activities
+ * @param superActivity
+ * the "swallowing" super activity
+ * @param superActivityId
+ * the activity id for the "swallowing" super activity, which is also the first added acitivty's id in the super activity
+ * @param existingSuperActivity
+ * an existing super activity which is to be swallowed by the "swallowing" super activity
+ */
+ private void swallowExistingSuperActivity(Map<ActivityId, SuperActivity> superActivities,
+ Map<ActivityId, Queue<IActivity>> toBeExpendedMap,
+ Map<IActivity, SuperActivity> invertedActivitySuperActivityMap, SuperActivity superActivity,
+ ActivityId superActivityId, SuperActivity existingSuperActivity) {
+ ActivityId existingSuperActivityId = existingSuperActivity.getActivityId();
+ superActivities.remove(existingSuperActivityId);
+ for (Entry<ActivityId, IActivity> existingEntry : existingSuperActivity.getActivityMap().entrySet()) {
+ IActivity existingActivity = existingEntry.getValue();
+ superActivity.addActivity(existingActivity);
+ invertedActivitySuperActivityMap.put(existingActivity, superActivity);
+ }
+ Queue<IActivity> tbeQueue = toBeExpendedMap.get(superActivityId);
+ Queue<IActivity> existingTbeQueque = toBeExpendedMap.remove(existingSuperActivityId);
+ if (existingTbeQueque != null) {
+ tbeQueue.addAll(existingTbeQueque);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java
new file mode 100644
index 0000000..07b7ffc
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/OneToOneConnectedActivityCluster.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+
+/**
+ * All the connectors in an OneToOneConnectedCluster are OneToOneConnectorDescriptors.
+ *
+ * @author yingyib
+ */
+public class OneToOneConnectedActivityCluster extends ActivityCluster {
+
+ private static final long serialVersionUID = 1L;
+
+ protected final Map<Integer, Pair<ActivityId, Integer>> clusterInputIndexMap = new HashMap<Integer, Pair<ActivityId, Integer>>();
+ protected final Map<Integer, Pair<ActivityId, Integer>> clusterOutputIndexMap = new HashMap<Integer, Pair<ActivityId, Integer>>();
+ protected final Map<Pair<ActivityId, Integer>, Integer> invertedClusterOutputIndexMap = new HashMap<Pair<ActivityId, Integer>, Integer>();
+ protected final Map<Pair<ActivityId, Integer>, Integer> invertedClusterInputIndexMap = new HashMap<Pair<ActivityId, Integer>, Integer>();
+
+ public OneToOneConnectedActivityCluster(ActivityClusterGraph acg, ActivityClusterId id) {
+ super(acg, id);
+ }
+
+ /**
+ * Set up the mapping of the cluster's output channel to an internal activity and its output channel
+ *
+ * @param clusterOutputIndex
+ * the output channel index for the cluster
+ * @param activityId
+ * the id of the internal activity which produces the corresponding output
+ * @param activityOutputIndex
+ * the output channel index of the internal activity which corresponds to the output channel of the cluster of activities
+ */
+ public void setClusterOutputIndex(int clusterOutputIndex, ActivityId activityId, int activityOutputIndex) {
+ clusterOutputIndexMap.put(clusterOutputIndex, Pair.of(activityId, activityOutputIndex));
+ invertedClusterOutputIndexMap.put(Pair.of(activityId, activityOutputIndex), clusterOutputIndex);
+ }
+
+ /**
+ * get the an internal activity and its output channel of a cluster output channel
+ *
+ * @param clusterOutputIndex
+ * the output channel index for the cluster
+ * @return a pair containing the activity id of the corresponding internal activity and the output channel index
+ */
+ public Pair<ActivityId, Integer> getActivityIdOutputIndex(int clusterOutputIndex) {
+ return clusterOutputIndexMap.get(clusterOutputIndex);
+ }
+
+ /**
+ * Set up the mapping of the cluster's input channel to an internal activity and input output channel
+ *
+ * @param clusterInputIndex
+ * the input channel index for the cluster
+ * @param activityId
+ * the id of the internal activity which consumes the corresponding input
+ * @param activityInputIndex
+ * the output channel index of the internal activity which corresponds to the input channel of the cluster of activities
+ */
+ public void setClusterInputIndex(int clusterInputIndex, ActivityId activityId, int activityInputIndex) {
+ clusterInputIndexMap.put(clusterInputIndex, Pair.of(activityId, activityInputIndex));
+ invertedClusterInputIndexMap.put(Pair.of(activityId, activityInputIndex), clusterInputIndex);
+ }
+
+ /**
+ * get the an internal activity and its input channel of a cluster input channel
+ *
+ * @param clusterOutputIndex
+ * the output channel index for the cluster
+ * @return a pair containing the activity id of the corresponding internal activity and the output channel index
+ */
+ public Pair<ActivityId, Integer> getActivityIdInputIndex(int clusterInputIndex) {
+ return clusterInputIndexMap.get(clusterInputIndex);
+ }
+
+ /**
+ * Get the cluster input channel of an input-boundary activity and its input channel
+ *
+ * @param activityInputChannel
+ * the input-boundary activity and its input channel
+ * @return the cluster input channel
+ */
+ public int getClusterInputIndex(Pair<ActivityId, Integer> activityInputChannel) {
+ Integer channel = invertedClusterInputIndexMap.get(activityInputChannel);
+ return channel == null ? -1 : channel;
+ }
+
+ /**
+ * Get the cluster output channel of an input-boundary activity and its output channel
+ *
+ * @param activityOutputChannel
+ * the output-boundary activity and its output channel
+ * @return the cluster output channel
+ */
+ public int getClusterOutputIndex(Pair<ActivityId, Integer> activityOutputChannel) {
+ Integer channel = invertedClusterOutputIndexMap.get(activityOutputChannel);
+ return channel == null ? -1 : channel;
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java
new file mode 100644
index 0000000..734ff85
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivity.java
@@ -0,0 +1,177 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter.runtime;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.ActivityCluster;
+import edu.uci.ics.hyracks.api.job.ActivityClusterGraph;
+import edu.uci.ics.hyracks.api.job.ActivityClusterId;
+import edu.uci.ics.hyracks.api.rewriter.OneToOneConnectedActivityCluster;
+
+/**
+ * This class can be used to execute a DAG of activities inside which
+ * there are only one-to-one connectors.
+ *
+ * @author yingyib
+ */
+public class SuperActivity extends OneToOneConnectedActivityCluster implements IActivity {
+ private static final long serialVersionUID = 1L;
+ private final ActivityId activityId;
+
+ public SuperActivity(ActivityClusterGraph acg, ActivityClusterId id, ActivityId activityId) {
+ super(acg, id);
+ this.activityId = activityId;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
+ final Map<ActivityId, IActivity> startActivities = new HashMap<ActivityId, IActivity>();
+ Map<ActivityId, IActivity> activities = getActivityMap();
+ for (Entry<ActivityId, IActivity> entry : activities.entrySet()) {
+ /**
+ * extract start activities
+ */
+ List<IConnectorDescriptor> conns = getActivityInputMap().get(entry.getKey());
+ if (conns == null || conns.size() == 0) {
+ startActivities.put(entry.getKey(), entry.getValue());
+ }
+ }
+
+ /**
+ * wrap a RecordDescriptorProvider for the super activity
+ */
+ IRecordDescriptorProvider wrappedRecDescProvider = new IRecordDescriptorProvider() {
+
+ @Override
+ public RecordDescriptor getInputRecordDescriptor(ActivityId aid, int inputIndex) {
+ if (startActivities.get(aid) != null) {
+ /**
+ * if the activity is a start (input boundary) activity
+ */
+ int superActivityInputChannel = SuperActivity.this.getClusterInputIndex(Pair.of(aid, inputIndex));
+ if (superActivityInputChannel >= 0) {
+ return recordDescProvider.getInputRecordDescriptor(activityId, superActivityInputChannel);
+ }
+ }
+ if (SuperActivity.this.getActivityMap().get(aid) != null) {
+ /**
+ * if the activity is an internal activity of the super activity
+ */
+ IConnectorDescriptor conn = getActivityInputMap().get(aid).get(inputIndex);
+ return getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+ }
+
+ /**
+ * the following is for the case where the activity is in other SuperActivities
+ */
+ ActivityClusterGraph acg = SuperActivity.this.getActivityClusterGraph();
+ for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+ ActivityCluster ac = entry.getValue();
+ for (Entry<ActivityId, IActivity> saEntry : ac.getActivityMap().entrySet()) {
+ SuperActivity sa = (SuperActivity) saEntry.getValue();
+ if (sa.getActivityMap().get(aid) != null) {
+ List<IConnectorDescriptor> conns = sa.getActivityInputMap().get(aid);
+ if (conns != null && conns.size() >= inputIndex) {
+ IConnectorDescriptor conn = conns.get(inputIndex);
+ return sa.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+ } else {
+ int superActivityInputChannel = sa.getClusterInputIndex(Pair.of(aid, inputIndex));
+ if (superActivityInputChannel >= 0) {
+ return recordDescProvider.getInputRecordDescriptor(sa.getActivityId(),
+ superActivityInputChannel);
+ }
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public RecordDescriptor getOutputRecordDescriptor(ActivityId aid, int outputIndex) {
+ /**
+ * if the activity is an output-boundary activity
+ */
+ int superActivityOutputChannel = SuperActivity.this.getClusterOutputIndex(Pair.of(aid, outputIndex));
+ if (superActivityOutputChannel >= 0) {
+ return recordDescProvider.getOutputRecordDescriptor(activityId, superActivityOutputChannel);
+ }
+
+ if (SuperActivity.this.getActivityMap().get(aid) != null) {
+ /**
+ * if the activity is an internal activity of the super activity
+ */
+ IConnectorDescriptor conn = getActivityOutputMap().get(aid).get(outputIndex);
+ return getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+ }
+
+ /**
+ * the following is for the case where the activity is in other SuperActivities
+ */
+ ActivityClusterGraph acg = SuperActivity.this.getActivityClusterGraph();
+ for (Entry<ActivityClusterId, ActivityCluster> entry : acg.getActivityClusterMap().entrySet()) {
+ ActivityCluster ac = entry.getValue();
+ for (Entry<ActivityId, IActivity> saEntry : ac.getActivityMap().entrySet()) {
+ SuperActivity sa = (SuperActivity) saEntry.getValue();
+ if (sa.getActivityMap().get(aid) != null) {
+ List<IConnectorDescriptor> conns = sa.getActivityOutputMap().get(aid);
+ if (conns != null && conns.size() >= outputIndex) {
+ IConnectorDescriptor conn = conns.get(outputIndex);
+ return sa.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
+ } else {
+ superActivityOutputChannel = sa.getClusterOutputIndex(Pair.of(aid, outputIndex));
+ if (superActivityOutputChannel >= 0) {
+ return recordDescProvider.getOutputRecordDescriptor(sa.getActivityId(),
+ superActivityOutputChannel);
+ }
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ };
+ return new SuperActivityOperatorNodePushable(this, startActivities, ctx, wrappedRecDescProvider, partition,
+ nPartitions);
+ }
+
+ @Override
+ public ActivityId getActivityId() {
+ return activityId;
+ }
+
+ @Override
+ public String toString() {
+ return getActivityMap().values().toString();
+ }
+}
diff --git a/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
new file mode 100644
index 0000000..7d50fa0
--- /dev/null
+++ b/fullstack/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/rewriter/runtime/SuperActivityOperatorNodePushable.java
@@ -0,0 +1,195 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.rewriter.runtime;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Queue;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The runtime of a SuperActivity, which internally executes a DAG of one-to-one
+ * connected activities in a single thread.
+ *
+ * @author yingyib
+ */
+public class SuperActivityOperatorNodePushable implements IOperatorNodePushable {
+ private final Map<ActivityId, IOperatorNodePushable> operatorNodePushables = new HashMap<ActivityId, IOperatorNodePushable>();
+ private final List<IOperatorNodePushable> operatprNodePushablesBFSOrder = new ArrayList<IOperatorNodePushable>();
+ private final Map<ActivityId, IActivity> startActivities;
+ private final SuperActivity parent;
+ private final IHyracksTaskContext ctx;
+ private final IRecordDescriptorProvider recordDescProvider;
+ private final int partition;
+ private final int nPartitions;
+ private int inputArity = 0;
+
+ public SuperActivityOperatorNodePushable(SuperActivity parent, Map<ActivityId, IActivity> startActivities,
+ IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
+ this.parent = parent;
+ this.startActivities = startActivities;
+ this.ctx = ctx;
+ this.recordDescProvider = recordDescProvider;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+
+ /**
+ * initialize the writer-relationship for the internal DAG of operator
+ * node pushables
+ */
+ try {
+ init();
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ /**
+ * initialize operator node pushables in the BFS order
+ */
+ for (IOperatorNodePushable op : operatprNodePushablesBFSOrder) {
+ op.initialize();
+ }
+ }
+
+ public void init() throws HyracksDataException {
+ Map<ActivityId, IOperatorNodePushable> startOperatorNodePushables = new HashMap<ActivityId, IOperatorNodePushable>();
+ Queue<Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> childQueue = new LinkedList<Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>>();
+ List<IConnectorDescriptor> outputConnectors = null;
+
+ /**
+ * Set up the source operators
+ */
+ for (Entry<ActivityId, IActivity> entry : startActivities.entrySet()) {
+ IOperatorNodePushable opPushable = entry.getValue().createPushRuntime(ctx, recordDescProvider, partition,
+ nPartitions);
+ startOperatorNodePushables.put(entry.getKey(), opPushable);
+ operatprNodePushablesBFSOrder.add(opPushable);
+ operatorNodePushables.put(entry.getKey(), opPushable);
+ inputArity += opPushable.getInputArity();
+ outputConnectors = parent.getActivityOutputMap().get(entry.getKey());
+ if (outputConnectors != null) {
+ for (IConnectorDescriptor conn : outputConnectors) {
+ childQueue.add(parent.getConnectorActivityMap().get(conn.getConnectorId()));
+ }
+ }
+ }
+
+ /**
+ * Using BFS (breadth-first search) to construct to runtime execution
+ * DAG;
+ */
+ while (childQueue.size() > 0) {
+ /**
+ * expend the executing activities further to the downstream
+ */
+ if (outputConnectors != null && outputConnectors.size() > 0) {
+ for (IConnectorDescriptor conn : outputConnectors) {
+ if (conn != null) {
+ childQueue.add(parent.getConnectorActivityMap().get(conn.getConnectorId()));
+ }
+ }
+ }
+
+ /**
+ * construct the source to destination information
+ */
+ Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> channel = childQueue.poll();
+ ActivityId sourceId = channel.getLeft().getLeft().getActivityId();
+ int outputChannel = channel.getLeft().getRight();
+ ActivityId destId = channel.getRight().getLeft().getActivityId();
+ int inputChannel = channel.getRight().getRight();
+ IOperatorNodePushable sourceOp = operatorNodePushables.get(sourceId);
+ IOperatorNodePushable destOp = operatorNodePushables.get(destId);
+ if (destOp == null) {
+ destOp = channel.getRight().getLeft()
+ .createPushRuntime(ctx, recordDescProvider, partition, nPartitions);
+ operatprNodePushablesBFSOrder.add(destOp);
+ operatorNodePushables.put(destId, destOp);
+ }
+
+ /**
+ * construct the dataflow connection from a producer to a consumer
+ */
+ sourceOp.setOutputFrameWriter(outputChannel, destOp.getInputFrameWriter(inputChannel),
+ recordDescProvider.getInputRecordDescriptor(destId, inputChannel));
+
+ /**
+ * traverse to the child of the current activity
+ */
+ outputConnectors = parent.getActivityOutputMap().get(destId);
+ }
+ }
+
+ @Override
+ public void deinitialize() throws HyracksDataException {
+ /**
+ * de-initialize operator node pushables
+ */
+ for (IOperatorNodePushable op : operatprNodePushablesBFSOrder) {
+ op.deinitialize();
+ }
+ }
+
+ @Override
+ public int getInputArity() {
+ return inputArity;
+ }
+
+ @Override
+ public void setOutputFrameWriter(int clusterOutputIndex, IFrameWriter writer, RecordDescriptor recordDesc) {
+ /**
+ * set the right output frame writer
+ */
+ Pair<ActivityId, Integer> activityIdOutputIndex = parent.getActivityIdOutputIndex(clusterOutputIndex);
+ IOperatorNodePushable opPushable = operatorNodePushables.get(activityIdOutputIndex.getLeft());
+ opPushable.setOutputFrameWriter(activityIdOutputIndex.getRight(), writer, recordDesc);
+ }
+
+ @Override
+ public IFrameWriter getInputFrameWriter(final int index) {
+ /**
+ * get the right IFrameWriter from the cluster input index
+ */
+ Pair<ActivityId, Integer> activityIdInputIndex = parent.getActivityIdInputIndex(index);
+ IOperatorNodePushable operatorNodePushable = operatorNodePushables.get(activityIdInputIndex.getLeft());
+ IFrameWriter writer = operatorNodePushable.getInputFrameWriter(activityIdInputIndex.getRight());
+ return writer;
+ }
+
+ @Override
+ public String getDisplayName() {
+ return "Super Activity " + parent.getActivityMap().values().toString();
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-cli/pom.xml b/fullstack/hyracks/hyracks-cli/pom.xml
index 3933be0..991ce2a 100644
--- a/fullstack/hyracks/hyracks-cli/pom.xml
+++ b/fullstack/hyracks/hyracks-cli/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
@@ -41,6 +42,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-client/pom.xml b/fullstack/hyracks/hyracks-client/pom.xml
new file mode 100644
index 0000000..854a009
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/pom.xml
@@ -0,0 +1,46 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-client</artifactId>
+ <name>hyracks-client</name>
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-net</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-comm</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
new file mode 100644
index 0000000..8be4a8c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class DatasetClientContext implements IHyracksCommonContext {
+ private final int frameSize;
+
+ public DatasetClientContext(int frameSize) {
+ this.frameSize = frameSize;
+ }
+
+ @Override
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ @Override
+ public IIOManager getIOManager() {
+ return null;
+ }
+
+ @Override
+ public ByteBuffer allocateFrame() {
+ return ByteBuffer.allocate(frameSize);
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
new file mode 100644
index 0000000..6866e46
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+
+public class HyracksDataset implements IHyracksDataset {
+ private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+ private final ClientNetworkManager netManager;
+
+ private final DatasetClientContext datasetClientCtx;
+
+ public HyracksDataset(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
+ NetworkAddress ddsAddress = hcc.getDatasetDirectoryServiceInfo();
+ datasetDirectoryServiceConnection = new HyracksDatasetDirectoryServiceConnection(new String(
+ ddsAddress.getIpAddress()), ddsAddress.getPort());
+
+ netManager = new ClientNetworkManager(nReaders);
+ netManager.start();
+
+ datasetClientCtx = new DatasetClientContext(frameSize);
+ }
+
+ @Override
+ public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+ IHyracksDatasetReader reader = null;
+ try {
+ reader = new HyracksDatasetReader(datasetDirectoryServiceConnection, netManager, datasetClientCtx, jobId,
+ resultSetId);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ return reader;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..095fd7d
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.net.InetSocketAddress;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceConnection implements IHyracksDatasetDirectoryServiceConnection {
+ private final IPCSystem ipc;
+ private final IHyracksDatasetDirectoryServiceInterface ddsi;
+
+ public HyracksDatasetDirectoryServiceConnection(String ddsHost, int ddsPort) throws Exception {
+ RPCInterface rpci = new RPCInterface();
+ ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
+ ipc.start();
+ IIPCHandle ddsIpchandle = ipc.getHandle(new InetSocketAddress(ddsHost, ddsPort));
+ this.ddsi = new HyracksDatasetDirectoryServiceInterfaceRemoteProxy(ddsIpchandle, rpci);
+ }
+
+ @Override
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+ return ddsi.getDatasetResultStatus(jobId, rsId);
+ }
+
+ @Override
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception {
+ return ddsi.getDatasetResultLocations(jobId, rsId, knownRecords);
+ }
+}
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
new file mode 100644
index 0000000..47cdf97
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceInterfaceRemoteProxy implements IHyracksDatasetDirectoryServiceInterface {
+ private final IIPCHandle ipcHandle;
+
+ private final RPCInterface rpci;
+
+ public HyracksDatasetDirectoryServiceInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
+ this.ipcHandle = ipcHandle;
+ this.rpci = rpci;
+ }
+
+ @Override
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction(
+ jobId, rsId);
+ return (Status) rpci.call(ipcHandle, gdrlf);
+ }
+
+ @Override
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction(
+ jobId, rsId, knownRecords);
+ return (DatasetDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
+ }
+}
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
new file mode 100644
index 0000000..78bcf20
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -0,0 +1,255 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.dataset;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetInputChannelMonitor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+import edu.uci.ics.hyracks.comm.channels.DatasetNetworkInputChannel;
+
+// TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetReader implements IHyracksDatasetReader {
+ private static final Logger LOGGER = Logger.getLogger(HyracksDatasetReader.class.getName());
+
+ private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+ private final ClientNetworkManager netManager;
+
+ private final DatasetClientContext datasetClientCtx;
+
+ private JobId jobId;
+
+ private ResultSetId resultSetId;
+
+ private DatasetDirectoryRecord[] knownRecords;
+
+ private IDatasetInputChannelMonitor[] monitors;
+
+ private int lastReadPartition;
+
+ private IDatasetInputChannelMonitor lastMonitor;
+
+ private DatasetNetworkInputChannel resultChannel;
+
+ private static int NUM_READ_BUFFERS = 1;
+
+ public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
+ ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+ throws Exception {
+ this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
+ this.netManager = netManager;
+ this.datasetClientCtx = datasetClientCtx;
+ this.jobId = jobId;
+ this.resultSetId = resultSetId;
+ knownRecords = null;
+ monitors = null;
+ lastReadPartition = -1;
+ lastMonitor = null;
+ resultChannel = null;
+ }
+
+ @Override
+ public Status getResultStatus() {
+ Status status = null;
+ try {
+ status = datasetDirectoryServiceConnection.getDatasetResultStatus(jobId, resultSetId);
+ } catch (Exception e) {
+ // TODO(madhusudancs): Decide what to do in case of error
+ }
+ return status;
+ }
+
+ @Override
+ public int read(ByteBuffer buffer) throws HyracksDataException {
+ ByteBuffer readBuffer;
+ int readSize = 0;
+
+ if (lastReadPartition == -1) {
+ while (knownRecords == null || knownRecords[0] == null) {
+ try {
+ knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId, resultSetId,
+ knownRecords);
+ lastReadPartition = 0;
+ resultChannel = new DatasetNetworkInputChannel(netManager,
+ getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+ NUM_READ_BUFFERS);
+ lastMonitor = getMonitor(lastReadPartition);
+ resultChannel.open(datasetClientCtx);
+ resultChannel.registerMonitor(lastMonitor);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ } catch (UnknownHostException e) {
+ throw new HyracksDataException(e);
+ } catch (Exception e) {
+ // Do nothing here.
+ }
+ }
+ }
+
+ while (readSize <= 0 && !((lastReadPartition == knownRecords.length - 1) && (lastMonitor.eosReached()))) {
+ synchronized (lastMonitor) {
+ while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached()) {
+ try {
+ lastMonitor.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ if (lastMonitor.getNFramesAvailable() <= 0 && lastMonitor.eosReached()) {
+ knownRecords[lastReadPartition].readEOS();
+ if ((lastReadPartition == knownRecords.length - 1)) {
+ break;
+ } else {
+ try {
+ lastReadPartition++;
+ while (knownRecords[lastReadPartition] == null) {
+ try {
+ knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
+ resultSetId, knownRecords);
+ } catch (Exception e) {
+ // Do nothing here.
+ }
+ }
+
+ resultChannel = new DatasetNetworkInputChannel(netManager,
+ getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+ NUM_READ_BUFFERS);
+ lastMonitor = getMonitor(lastReadPartition);
+ resultChannel.open(datasetClientCtx);
+ resultChannel.registerMonitor(lastMonitor);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ } catch (UnknownHostException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ } else {
+ readBuffer = resultChannel.getNextBuffer();
+ lastMonitor.notifyFrameRead();
+ if (readBuffer != null) {
+ buffer.put(readBuffer);
+ buffer.flip();
+ readSize = buffer.limit();
+ resultChannel.recycleBuffer(readBuffer);
+ }
+ }
+ }
+
+ return readSize;
+ }
+
+ private boolean nullExists(DatasetDirectoryRecord[] locations) {
+ if (locations == null) {
+ return true;
+ }
+ for (int i = 0; i < locations.length; i++) {
+ if (locations[i] == null) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private SocketAddress getSocketAddress(DatasetDirectoryRecord addr) throws UnknownHostException {
+ NetworkAddress netAddr = addr.getNetworkAddress();
+ return new InetSocketAddress(InetAddress.getByAddress(netAddr.getIpAddress()), netAddr.getPort());
+ }
+
+ private IDatasetInputChannelMonitor getMonitor(int partition) throws HyracksException {
+ if (knownRecords == null || knownRecords[partition] == null) {
+ throw new HyracksException("Accessing monitors before the obtaining the corresponding addresses.");
+ }
+ if (monitors == null) {
+ monitors = new DatasetInputChannelMonitor[knownRecords.length];
+ }
+ if (monitors[partition] == null) {
+ monitors[partition] = new DatasetInputChannelMonitor();
+ }
+ return monitors[partition];
+ }
+
+ private class DatasetInputChannelMonitor implements IDatasetInputChannelMonitor {
+ private final AtomicInteger nAvailableFrames;
+
+ private final AtomicBoolean eos;
+
+ private final AtomicBoolean failed;
+
+ public DatasetInputChannelMonitor() {
+ nAvailableFrames = new AtomicInteger(0);
+ eos = new AtomicBoolean(false);
+ failed = new AtomicBoolean(false);
+ }
+
+ @Override
+ public synchronized void notifyFailure(IInputChannel channel) {
+ failed.set(true);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+ nAvailableFrames.addAndGet(nFrames);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyEndOfStream(IInputChannel channel) {
+ eos.set(true);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized boolean eosReached() {
+ return eos.get();
+ }
+
+ @Override
+ public synchronized boolean failed() {
+ return failed.get();
+ }
+
+ @Override
+ public synchronized int getNFramesAvailable() {
+ return nAvailableFrames.get();
+ }
+
+ @Override
+ public synchronized void notifyFrameRead() {
+ nAvailableFrames.decrementAndGet();
+ }
+
+ }
+}
diff --git a/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
new file mode 100644
index 0000000..7aef8b9
--- /dev/null
+++ b/fullstack/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.client.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class ClientNetworkManager implements IChannelConnectionFactory {
+ private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+ private final MuxDemux md;
+
+ public ClientNetworkManager(int nThreads) throws IOException {
+ /* This is a connect only socket and does not listen to any incoming connections, so pass null to
+ * localAddress and listener.
+ */
+ md = new MuxDemux(null, null, nThreads, MAX_CONNECTION_ATTEMPTS);
+ }
+
+ public void start() throws IOException {
+ md.start();
+ }
+
+ public void stop() {
+
+ }
+
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+ MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+ return mConn.openChannel();
+ }
+
+ public MuxDemuxPerformanceCounters getPerformanceCounters() {
+ return md.getPerformanceCounters();
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-comm/pom.xml b/fullstack/hyracks/hyracks-comm/pom.xml
new file mode 100644
index 0000000..c3583699
--- /dev/null
+++ b/fullstack/hyracks/hyracks-comm/pom.xml
@@ -0,0 +1,36 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-comm</artifactId>
+ <name>hyracks-comm</name>
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-net</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
similarity index 72%
copy from fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
copy to fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
index 1d5af84..fac2949 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
@@ -23,21 +23,25 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
-public class NetworkInputChannel implements IInputChannel {
- private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+public class DatasetNetworkInputChannel implements IInputChannel {
+ private static final Logger LOGGER = Logger.getLogger(DatasetNetworkInputChannel.class.getName());
- private final NetworkManager netManager;
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IChannelConnectionFactory netManager;
private final SocketAddress remoteAddress;
- private final PartitionId partitionId;
+ private final JobId jobId;
+
+ private final int partition;
private final Queue<ByteBuffer> fullQueue;
@@ -49,11 +53,12 @@
private Object attachment;
- public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
- int nBuffers) {
+ public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
+ int partition, int nBuffers) {
this.netManager = netManager;
this.remoteAddress = remoteAddress;
- this.partitionId = partitionId;
+ this.jobId = jobId;
+ this.partition = partition;
fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
this.nBuffers = nBuffers;
}
@@ -85,7 +90,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
try {
ccb = netManager.connect(remoteAddress);
} catch (Exception e) {
@@ -96,14 +101,13 @@
for (int i = 0; i < nBuffers; ++i) {
ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
}
- ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
- writeBuffer.putLong(partitionId.getJobId().getId());
- writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
- writeBuffer.putInt(partitionId.getSenderIndex());
- writeBuffer.putInt(partitionId.getReceiverIndex());
+ ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+ writeBuffer.putLong(jobId.getId());
+ writeBuffer.putInt(partition);
writeBuffer.flip();
if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+ LOGGER.fine("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
+ + ccb);
}
ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
ccb.getWriteInterface().getFullBufferAcceptor().close();
@@ -118,17 +122,17 @@
@Override
public void accept(ByteBuffer buffer) {
fullQueue.add(buffer);
- monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
+ monitor.notifyDataAvailability(DatasetNetworkInputChannel.this, 1);
}
@Override
public void close() {
- monitor.notifyEndOfStream(NetworkInputChannel.this);
+ monitor.notifyEndOfStream(DatasetNetworkInputChannel.this);
}
@Override
public void error(int ecode) {
- monitor.notifyFailure(NetworkInputChannel.this);
+ monitor.notifyFailure(DatasetNetworkInputChannel.this);
}
}
diff --git a/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
new file mode 100644
index 0000000..33179ba
--- /dev/null
+++ b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
@@ -0,0 +1,24 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.comm.channels;
+
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+
+public interface IChannelConnectionFactory {
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException;
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
similarity index 89%
rename from fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
rename to fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index 1d5af84..aa37b16 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
@@ -23,7 +23,7 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
@@ -33,7 +33,9 @@
public class NetworkInputChannel implements IInputChannel {
private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
- private final NetworkManager netManager;
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IChannelConnectionFactory netManager;
private final SocketAddress remoteAddress;
@@ -49,8 +51,8 @@
private Object attachment;
- public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
- int nBuffers) {
+ public NetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress,
+ PartitionId partitionId, int nBuffers) {
this.netManager = netManager;
this.remoteAddress = remoteAddress;
this.partitionId = partitionId;
@@ -85,7 +87,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
try {
ccb = netManager.connect(remoteAddress);
} catch (Exception e) {
@@ -96,7 +98,7 @@
for (int i = 0; i < nBuffers; ++i) {
ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
}
- ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+ ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
writeBuffer.putLong(partitionId.getJobId().getId());
writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
writeBuffer.putInt(partitionId.getSenderIndex());
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
similarity index 92%
rename from fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
rename to fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index 9024e18..812a2de 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/fullstack/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -12,14 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.Deque;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
@@ -40,9 +39,9 @@
ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
}
- public void setTaskContext(IHyracksTaskContext ctx) {
+ public void setFrameSize(int frameSize) {
for (int i = 0; i < nBuffers; ++i) {
- emptyStack.push(ByteBuffer.allocateDirect(ctx.getFrameSize()));
+ emptyStack.push(ByteBuffer.allocateDirect(frameSize));
}
}
@@ -87,7 +86,7 @@
ccb.getWriteInterface().getFullBufferAcceptor().close();
}
- void abort() {
+ public void abort() {
ccb.getWriteInterface().getFullBufferAcceptor().error(1);
synchronized (NetworkOutputChannel.this) {
aborted = true;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index c7eedb3..d644673 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 5a33891..82457fe 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -35,12 +35,17 @@
import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.topology.ClusterTopology;
import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.dataset.DatasetDirectoryService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
@@ -48,17 +53,23 @@
import edu.uci.ics.hyracks.control.cc.work.ApplicationMessageWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetDatasetDirectoryServiceInfoWork;
import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultPartitionLocationsWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultStatusWork;
import edu.uci.ics.hyracks.control.cc.work.JobStartWork;
import edu.uci.ics.hyracks.control.cc.work.JobletCleanupNotificationWork;
import edu.uci.ics.hyracks.control.cc.work.NodeHeartbeatWork;
import edu.uci.ics.hyracks.control.cc.work.RegisterNodeWork;
import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionAvailibilityWork;
import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionRequestWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterResultPartitionLocationWork;
import edu.uci.ics.hyracks.control.cc.work.RemoveDeadNodesWork;
import edu.uci.ics.hyracks.control.cc.work.ReportProfilesWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionFailureWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionWriteCompletionWork;
import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
@@ -115,6 +126,8 @@
private final DeadNodeSweeper sweeper;
+ private final IDatasetDirectoryService datasetDirectoryService;
+
private long jobCounter;
public ClusterControllerService(final CCConfig ccConfig) throws Exception {
@@ -162,6 +175,7 @@
}
};
sweeper = new DeadNodeSweeper();
+ datasetDirectoryService = new DatasetDirectoryService();
jobCounter = 0;
}
@@ -264,6 +278,10 @@
return clusterIPC;
}
+ public NetworkAddress getDatasetDirectoryServiceInfo() {
+ return new NetworkAddress(ccConfig.clientNetIpAddress.getBytes(), ccConfig.clientNetPort);
+ }
+
private class DeadNodeSweeper extends TimerTask {
@Override
public void run() {
@@ -271,6 +289,10 @@
}
}
+ public IDatasetDirectoryService getDatasetDirectoryService() {
+ return datasetDirectoryService;
+ }
+
private class HyracksClientInterfaceIPCI implements IIPCI {
@Override
public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
@@ -321,6 +343,27 @@
return;
}
+ case GET_DATASET_DIRECTORY_SERIVICE_INFO: {
+ workQueue.schedule(new GetDatasetDirectoryServiceInfoWork(ClusterControllerService.this,
+ new IPCResponder<NetworkAddress>(handle, mid)));
+ return;
+ }
+
+ case GET_DATASET_RESULT_STATUS: {
+ HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+ workQueue.schedule(new GetResultStatusWork(ClusterControllerService.this, gdrlf.getJobId(), gdrlf
+ .getResultSetId(), new IPCResponder<Status>(handle, mid)));
+ return;
+ }
+
+ case GET_DATASET_RESULT_LOCATIONS: {
+ HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+ workQueue.schedule(new GetResultPartitionLocationsWork(ClusterControllerService.this, gdrlf
+ .getJobId(), gdrlf.getResultSetId(), gdrlf.getKnownRecords(),
+ new IPCResponder<DatasetDirectoryRecord[]>(handle, mid)));
+ return;
+ }
+
case WAIT_FOR_COMPLETION: {
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
@@ -416,6 +459,28 @@
return;
}
+ case REGISTER_RESULT_PARTITION_LOCATION: {
+ CCNCFunctions.RegisterResultPartitionLocationFunction rrplf = (CCNCFunctions.RegisterResultPartitionLocationFunction) fn;
+ workQueue.schedule(new RegisterResultPartitionLocationWork(ClusterControllerService.this, rrplf
+ .getJobId(), rrplf.getResultSetId(), rrplf.getOrderedResult(), rrplf.getPartition(), rrplf
+ .getNPartitions(), rrplf.getNetworkAddress()));
+ return;
+ }
+
+ case REPORT_RESULT_PARTITION_WRITE_COMPLETION: {
+ CCNCFunctions.ReportResultPartitionWriteCompletionFunction rrplf = (CCNCFunctions.ReportResultPartitionWriteCompletionFunction) fn;
+ workQueue.schedule(new ReportResultPartitionWriteCompletionWork(ClusterControllerService.this,
+ rrplf.getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+ return;
+ }
+
+ case REPORT_RESULT_PARTITION_FAILURE: {
+ CCNCFunctions.ReportResultPartitionFailureFunction rrplf = (CCNCFunctions.ReportResultPartitionFailureFunction) fn;
+ workQueue.schedule(new ReportResultPartitionFailureWork(ClusterControllerService.this, rrplf
+ .getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+ return;
+ }
+
case APPLICATION_STATE_CHANGE_RESPONSE: {
CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index c17acd0..c96a319 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -41,6 +41,8 @@
private final NetworkAddress dataPort;
+ private final NetworkAddress datasetPort;
+
private final Set<JobId> activeJobIds;
private final String osName;
@@ -107,6 +109,14 @@
private final long[] netSignalingBytesWritten;
+ private final long[] datasetNetPayloadBytesRead;
+
+ private final long[] datasetNetPayloadBytesWritten;
+
+ private final long[] datasetNetSignalingBytesRead;
+
+ private final long[] datasetNetSignalingBytesWritten;
+
private final long[] ipcMessagesSent;
private final long[] ipcMessageBytesSent;
@@ -123,6 +133,7 @@
this.nodeController = nodeController;
ncConfig = reg.getNCConfig();
dataPort = reg.getDataPort();
+ datasetPort = reg.getDatasetPort();
activeJobIds = new HashSet<JobId>();
osName = reg.getOSName();
@@ -164,6 +175,10 @@
netPayloadBytesWritten = new long[RRD_SIZE];
netSignalingBytesRead = new long[RRD_SIZE];
netSignalingBytesWritten = new long[RRD_SIZE];
+ datasetNetPayloadBytesRead = new long[RRD_SIZE];
+ datasetNetPayloadBytesWritten = new long[RRD_SIZE];
+ datasetNetSignalingBytesRead = new long[RRD_SIZE];
+ datasetNetSignalingBytesWritten = new long[RRD_SIZE];
ipcMessagesSent = new long[RRD_SIZE];
ipcMessageBytesSent = new long[RRD_SIZE];
ipcMessagesReceived = new long[RRD_SIZE];
@@ -196,6 +211,10 @@
netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
+ datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
+ datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
+ datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
+ datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
@@ -227,6 +246,10 @@
return dataPort;
}
+ public NetworkAddress getDatasetPort() {
+ return datasetPort;
+ }
+
public JSONObject toSummaryJSON() throws JSONException {
JSONObject o = new JSONObject();
o.put("node-id", ncConfig.nodeId);
@@ -271,6 +294,10 @@
o.put("net-payload-bytes-written", netPayloadBytesWritten);
o.put("net-signaling-bytes-read", netSignalingBytesRead);
o.put("net-signaling-bytes-written", netSignalingBytesWritten);
+ o.put("dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
+ o.put("dataset-net-payload-bytes-written", datasetNetPayloadBytesWritten);
+ o.put("dataset-net-signaling-bytes-read", datasetNetSignalingBytesRead);
+ o.put("dataset-net-signaling-bytes-written", datasetNetSignalingBytesWritten);
o.put("ipc-messages-sent", ipcMessagesSent);
o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
o.put("ipc-messages-received", ipcMessagesReceived);
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
new file mode 100644
index 0000000..13d0c30
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -0,0 +1,241 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.dataset;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+/**
+ * TODO(madhusudancs): The potential perils of this global dataset directory service implementation is that, the jobs
+ * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
+ * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
+ * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
+ * job.
+ */
+public class DatasetDirectoryService implements IDatasetDirectoryService {
+ private final Map<JobId, Map<ResultSetId, ResultSetMetaData>> jobResultLocationsMap;
+
+ public DatasetDirectoryService() {
+ jobResultLocationsMap = new HashMap<JobId, Map<ResultSetId, ResultSetMetaData>>();
+ }
+
+ @Override
+ public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions, NetworkAddress networkAddress) {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ if (rsMap == null) {
+ rsMap = new HashMap<ResultSetId, ResultSetMetaData>();
+ jobResultLocationsMap.put(jobId, rsMap);
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null) {
+ resultSetMetaData = new ResultSetMetaData(orderedResult, new DatasetDirectoryRecord[nPartitions]);
+ rsMap.put(rsId, resultSetMetaData);
+ }
+
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ if (records[partition] == null) {
+ records[partition] = new DatasetDirectoryRecord();
+ }
+ records[partition].setNetworkAddress(networkAddress);
+ records[partition].start();
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) {
+ DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+ ddr.writeEOS();
+ }
+
+ @Override
+ public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
+ DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+ ddr.fail();
+ }
+
+ @Override
+ public synchronized Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException {
+ Map<ResultSetId, ResultSetMetaData> rsMap;
+ while ((rsMap = jobResultLocationsMap.get(jobId)) == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+ throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+ }
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+
+ ArrayList<Status> statuses = new ArrayList<Status>(records.length);
+ for (int i = 0; i < records.length; i++) {
+ statuses.add(records[i].getStatus());
+ }
+
+ // Default status is idle
+ Status status = Status.IDLE;
+ if (statuses.contains(Status.FAILED)) {
+ // Even if there is at least one failed entry we should return failed status.
+ return Status.FAILED;
+ } else if (statuses.contains(Status.RUNNING)) {
+ // If there are not failed entry and if there is at least one running entry we should return running status.
+ return Status.RUNNING;
+ } else {
+ // If each and every partition has reported success do we report success as the status.
+ int successCount = 0;
+ for (int i = 0; i < statuses.size(); i++) {
+ if (statuses.get(i) == Status.SUCCESS) {
+ successCount++;
+ }
+ }
+ if (successCount == statuses.size()) {
+ return Status.SUCCESS;
+ }
+ }
+ return status;
+ }
+
+ @Override
+ public synchronized DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws HyracksDataException {
+ DatasetDirectoryRecord[] newRecords;
+ while ((newRecords = updatedRecords(jobId, rsId, knownRecords)) == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ return newRecords;
+ }
+
+ public DatasetDirectoryRecord getDatasetDirectoryRecord(JobId jobId, ResultSetId rsId, int partition) {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ return records[partition];
+ }
+
+ /**
+ * Compares the records already known by the client for the given job's result set id with the records that the
+ * dataset directory service knows and if there are any newly discovered records returns a whole array with the
+ * new records filled in.
+ * This method has a very convoluted logic. Here is the explanation of how it works.
+ * If the ordering constraint has to be enforced, the method obtains the first null record in the known records in
+ * the order of the partitions. It always traverses the array in the first to last order!
+ * If known records array or the first element in that array is null in the but the record for that partition now
+ * known to the directory service, the method fills in that record in the array and returns the array back.
+ * However, if the first known null record is not a first element in the array, by induction, all the previous
+ * known records should be known already be known to client and none of the records for the partitions ahead is
+ * known by the client yet. So, we check if the client has reached the end of stream for the partition corresponding
+ * to the record before the first known null record, i.e. the last known non-null record. If not, we just return
+ * null because we cannot expose any new locations until the client reaches end of stream for the last known record.
+ * If the client has reached the end of stream record for the last known non-null record, we check if the next record
+ * is discovered by the dataset directory service and if so, we fill it in the records array and return it back or
+ * send null otherwise.
+ * If the ordering is not required, we are free to return any newly discovered records back, so we just check if
+ * arrays are equal and if they are not we send the entire new updated array.
+ *
+ * @param jobId
+ * - Id of the job for which the directory records should be retrieved.
+ * @param rsId
+ * - Id of the result set for which the directory records should be retrieved.
+ * @param knownRecords
+ * - An array of directory records that the client is already aware of.
+ * @return
+ * - Returns null if there aren't any newly discovered partitions enforcing the ordering constraint
+ * @throws HyracksDataException
+ * TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n) iterations for
+ * every check. This already looks very expensive.
+ */
+ private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords)
+ throws HyracksDataException {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ if (rsMap == null) {
+ return null;
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+ throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+ }
+
+ boolean ordered = resultSetMetaData.getOrderedResult();
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ /* If ordering is required, we should expose the dataset directory records only in the order, otherwise
+ * we can simply check if there are any newly discovered records and send the whole array back if there are.
+ */
+ if (ordered) {
+ // Iterate over the known records and find the last record which is not null.
+ int i = 0;
+ for (i = 0; i < records.length; i++) {
+ if (knownRecords == null) {
+ if (records[0] != null) {
+ knownRecords = new DatasetDirectoryRecord[records.length];
+ knownRecords[0] = records[0];
+ return knownRecords;
+ }
+ return null;
+ }
+ if (knownRecords[i] == null) {
+ if ((i == 0 || knownRecords[i - 1].hasReachedReadEOS()) && records[i] != null) {
+ knownRecords[i] = records[i];
+ return knownRecords;
+ }
+ return null;
+ }
+ }
+ } else {
+ if (!Arrays.equals(records, knownRecords)) {
+ return records;
+ }
+ }
+ return null;
+ }
+
+ private class ResultSetMetaData {
+ private final boolean ordered;
+
+ private final DatasetDirectoryRecord[] records;
+
+ public ResultSetMetaData(boolean ordered, DatasetDirectoryRecord[] records) {
+ this.ordered = ordered;
+ this.records = records;
+ }
+
+ public boolean getOrderedResult() {
+ return ordered;
+ }
+
+ public DatasetDirectoryRecord[] getRecords() {
+ return records;
+ }
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
new file mode 100644
index 0000000..3ac6acc
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetDatasetDirectoryServiceInfoWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final IResultCallback<NetworkAddress> callback;
+
+ public GetDatasetDirectoryServiceInfoWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
+ this.ccs = ccs;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ try {
+ NetworkAddress addr = ccs.getDatasetDirectoryServiceInfo();
+ callback.setValue(addr);
+ } catch (Exception e) {
+ callback.setException(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 2f23a2c..a787b9f 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -39,7 +39,8 @@
Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
- result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort()));
+ result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort(), e
+ .getValue().getDatasetPort()));
}
callback.setValue(result);
}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
new file mode 100644
index 0000000..fd1d418
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultPartitionLocationsWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final DatasetDirectoryRecord[] knownRecords;
+
+ private final IResultCallback<DatasetDirectoryRecord[]> callback;
+
+ public GetResultPartitionLocationsWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.knownRecords = knownRecords;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ final IDatasetDirectoryService dds = ccs.getDatasetDirectoryService();
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ DatasetDirectoryRecord[] partitionLocations = dds.getResultPartitionLocations(jobId, rsId,
+ knownRecords);
+ callback.setValue(partitionLocations);
+ } catch (HyracksDataException e) {
+ callback.setException(e);
+ }
+ }
+ });
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
new file mode 100644
index 0000000..d2dadf5
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultStatusWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final IResultCallback<Status> callback;
+
+ public GetResultStatusWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ IResultCallback<Status> callback) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ try {
+ Status status = ccs.getDatasetDirectoryService().getResultStatus(jobId, rsId);
+ callback.setValue(status);
+ } catch (HyracksDataException e) {
+ callback.setException(e);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index b6a33cd..b062d33 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -16,6 +16,7 @@
import java.util.EnumSet;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
new file mode 100644
index 0000000..f86e924
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class RegisterResultPartitionLocationWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final boolean orderedResult;
+
+ private final int partition;
+
+ private final int nPartitions;
+
+ private final NetworkAddress networkAddress;
+
+ public RegisterResultPartitionLocationWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ boolean orderedResult, int partition, int nPartitions, NetworkAddress networkAddress) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.orderedResult = orderedResult;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+ nPartitions, networkAddress);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition + " NPartitions@" + nPartitions
+ + " ResultPartitionLocation@" + networkAddress + " OrderedResult@" + orderedResult;
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
new file mode 100644
index 0000000..4aea41e
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionFailureWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionFailureWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId, int partition) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().reportResultPartitionFailure(jobId, rsId, partition);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
new file mode 100644
index 0000000..313b730
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.work;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionWriteCompletionWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionWriteCompletionWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ int partition) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
index ff9d8a0..8e94269 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
@@ -58,6 +58,10 @@
var netPayloadBytesWritten = result['net-payload-bytes-written'];
var netSignalingBytesRead = result['net-signaling-bytes-read'];
var netSignalingBytesWritten = result['net-signaling-bytes-written'];
+ var datasetNetPayloadBytesRead = result['dataset-net-payload-bytes-read'];
+ var datasetNetPayloadBytesWritten = result['dataset-net-payload-bytes-written'];
+ var datasetNetSignalingBytesRead = result['dataset-net-signaling-bytes-read'];
+ var datasetNetSignalingBytesWritten = result['dataset-net-signaling-bytes-written'];
var ipcMessagesSent = result['ipc-messages-sent'];
var ipcMessageBytesSent = result['ipc-message-bytes-sent'];
var ipcMessagesReceived = result['ipc-messages-received'];
@@ -117,8 +121,12 @@
}
if (i < sysLoad.length - 1) {
netPayloadReadBWArray.push([ i, computeRate(netPayloadBytesRead, rrdPtr) ]);
+ netPayloadReadBWArray.push([ i, computeRate(datasetNetPayloadBytesRead, rrdPtr) ]);
netPayloadWriteBWArray.push([ i, computeRate(netPayloadBytesWritten, rrdPtr) ]);
+ netPayloadWriteBWArray.push([ i, computeRate(datasetNetPayloadBytesWritten, rrdPtr) ]);
netSignalingReadBWArray.push([ i, computeRate(netSignalingBytesRead, rrdPtr) ]);
+ netSignalingReadBWArray.push([ i, computeRate(datasetNetSignalingBytesRead, rrdPtr) ]);
+ netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
ipcMessageSendRateArray.push([ i, computeRate(ipcMessagesSent, rrdPtr) ]);
ipcMessageBytesSendRateArray.push([ i, computeRate(ipcMessageBytesSent, rrdPtr) ]);
@@ -229,4 +237,4 @@
}
fetchData();
-});
\ No newline at end of file
+});
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml b/fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
index 096f2e1..ce1298e 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 0c5bb2f..55e4479 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -16,7 +16,9 @@
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
@@ -46,6 +48,13 @@
public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+ public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+ int nPartitions, NetworkAddress networkAddress) throws Exception;
+
+ public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
+ public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 167eb4b..5071bc9 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -37,6 +37,9 @@
@Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
public String dataIPAddress;
+ @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+ public String datasetIPAddress;
+
@Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
public String ioDevices = System.getProperty("java.io.tmpdir");
@@ -55,6 +58,9 @@
@Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
public int maxMemory = -1;
+ @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+ public int resultManagerMemory = -1;
+
public void toCommandLine(List<String> cList) {
cList.add("-cc-host");
cList.add(ccHost);
@@ -66,6 +72,7 @@
cList.add(nodeId);
cList.add("-data-ip-address");
cList.add(dataIPAddress);
+ cList.add(datasetIPAddress);
cList.add("-iodevices");
cList.add(ioDevices);
cList.add("-dcache-client-servers");
@@ -80,5 +87,7 @@
cList.add(String.valueOf(nNetThreads));
cList.add("-max-memory");
cList.add(String.valueOf(maxMemory));
+ cList.add("-result-manager-memory");
+ cList.add(String.valueOf(resultManagerMemory));
}
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index 91cfecf..a897602 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -33,6 +33,8 @@
private final NetworkAddress dataPort;
+ private final NetworkAddress datasetPort;
+
private final String osName;
private final String arch;
@@ -60,13 +62,14 @@
private final HeartbeatSchema hbSchema;
public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
- String osName, String arch, String osVersion, int nProcessors, String vmName, String vmVersion,
- String vmVendor, String classpath, String libraryPath, String bootClasspath, List<String> inputArguments,
- Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
+ NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors, String vmName,
+ String vmVersion, String vmVendor, String classpath, String libraryPath, String bootClasspath,
+ List<String> inputArguments, Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
this.ncAddress = ncAddress;
this.nodeId = nodeId;
this.ncConfig = ncConfig;
this.dataPort = dataPort;
+ this.datasetPort = datasetPort;
this.osName = osName;
this.arch = arch;
this.osVersion = osVersion;
@@ -98,6 +101,10 @@
return dataPort;
}
+ public NetworkAddress getDatasetPort() {
+ return datasetPort;
+ }
+
public String getOSName() {
return osName;
}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index 1dba3bc..663c68a 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -37,6 +37,10 @@
public long netPayloadBytesWritten;
public long netSignalingBytesRead;
public long netSignalingBytesWritten;
+ public long datasetNetPayloadBytesRead;
+ public long datasetNetPayloadBytesWritten;
+ public long datasetNetSignalingBytesRead;
+ public long datasetNetSignalingBytesWritten;
public long ipcMessagesSent;
public long ipcMessageBytesSent;
public long ipcMessagesReceived;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 557a8cb..b506b12 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -36,6 +36,7 @@
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -68,6 +69,9 @@
REPORT_PROFILE,
REGISTER_PARTITION_PROVIDER,
REGISTER_PARTITION_REQUEST,
+ REGISTER_RESULT_PARTITION_LOCATION,
+ REPORT_RESULT_PARTITION_WRITE_COMPLETION,
+ REPORT_RESULT_PARTITION_FAILURE,
APPLICATION_STATE_CHANGE_RESPONSE,
NODE_REGISTRATION_RESULT,
@@ -438,6 +442,127 @@
}
}
+ public static class RegisterResultPartitionLocationFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final boolean orderedResult;
+
+ private final int partition;
+
+ private final int nPartitions;
+
+ private NetworkAddress networkAddress;
+
+ public RegisterResultPartitionLocationFunction(JobId jobId, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions, NetworkAddress networkAddress) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.orderedResult = orderedResult;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REGISTER_RESULT_PARTITION_LOCATION;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public boolean getOrderedResult() {
+ return orderedResult;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public int getNPartitions() {
+ return nPartitions;
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+ }
+
+ public static class ReportResultPartitionWriteCompletionFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionWriteCompletionFunction(JobId jobId, ResultSetId rsId, int partition) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_RESULT_PARTITION_WRITE_COMPLETION;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+ }
+
+ public static class ReportResultPartitionFailureFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionFailureFunction(JobId jobId, ResultSetId rsId, int partition) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_RESULT_PARTITION_FAILURE;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+ }
+
public static class ApplicationStateChangeResponseFunction extends Function {
private static final long serialVersionUID = 1L;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bbaab4e..091a5d2 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -16,7 +16,9 @@
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.base.IClusterController;
@@ -95,6 +97,28 @@
}
@Override
+ public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+ int nPartitions, NetworkAddress networkAddress) throws Exception {
+ CCNCFunctions.RegisterResultPartitionLocationFunction fn = new CCNCFunctions.RegisterResultPartitionLocationFunction(
+ jobId, rsId, orderedResult, partition, nPartitions, networkAddress);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
+ public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+ CCNCFunctions.ReportResultPartitionWriteCompletionFunction fn = new CCNCFunctions.ReportResultPartitionWriteCompletionFunction(
+ jobId, rsId, partition);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
+ public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+ CCNCFunctions.ReportResultPartitionFailureFunction fn = new CCNCFunctions.ReportResultPartitionFailureFunction(
+ jobId, rsId, partition);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
nodeId, appName, status);
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index 9ecc083..c44cec9 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
@@ -40,6 +41,11 @@
<artifactId>hyracks-net</artifactId>
<version>0.2.3-SNAPSHOT</version>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-comm</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
</dependencies>
<reporting>
<plugins>
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0195143..8a36dac 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -45,6 +45,7 @@
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
@@ -61,7 +62,9 @@
import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.WorkQueue;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+import edu.uci.ics.hyracks.control.nc.dataset.DatasetPartitionManager;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
@@ -94,6 +97,10 @@
private final NetworkManager netManager;
+ private final IDatasetPartitionManager datasetPartitionManager;
+
+ private final DatasetNetworkManager datasetNetworkManager;
+
private final WorkQueue queue;
private final Timer timer;
@@ -140,7 +147,11 @@
throw new Exception("id not set");
}
partitionManager = new PartitionManager(this);
- netManager = new NetworkManager(getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
+ netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
+
+ datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory);
+ datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
+ datasetPartitionManager, ncConfig.nNetThreads);
queue = new WorkQueue();
jobletMap = new Hashtable<JobId, Joblet>();
@@ -205,6 +216,7 @@
LOGGER.log(Level.INFO, "Starting NodeControllerService");
ipc.start();
netManager.start();
+ datasetNetworkManager.start();
IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -213,10 +225,11 @@
}
HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
- osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
- runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
- .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
- runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
+ datasetNetworkManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean
+ .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
+ .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
+ .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
+ runtimeMXBean.getSystemProperties(), hbSchema));
synchronized (this) {
while (registrationPending) {
@@ -247,8 +260,10 @@
LOGGER.log(Level.INFO, "Stopping NodeControllerService");
executor.shutdownNow();
partitionManager.close();
+ datasetPartitionManager.close();
heartbeatTask.cancel();
netManager.stop();
+ datasetNetworkManager.stop();
queue.stop();
LOGGER.log(Level.INFO, "Stopped NodeControllerService");
}
@@ -273,6 +288,10 @@
return netManager;
}
+ public DatasetNetworkManager getDatasetNetworkManager() {
+ return datasetNetworkManager;
+ }
+
public PartitionManager getPartitionManager() {
return partitionManager;
}
@@ -297,8 +316,7 @@
return queue;
}
- private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
- String ipaddrStr = ncConfig.dataIPAddress;
+ private static InetAddress getIpAddress(String ipaddrStr) throws Exception {
ipaddrStr = ipaddrStr.trim();
Pattern pattern = Pattern.compile("(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})");
Matcher m = pattern.matcher(ipaddrStr);
@@ -355,6 +373,12 @@
hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
+ MuxDemuxPerformanceCounters datasetNetPC = datasetNetworkManager.getPerformanceCounters();
+ hbData.datasetNetPayloadBytesRead = datasetNetPC.getPayloadBytesRead();
+ hbData.datasetNetPayloadBytesWritten = datasetNetPC.getPayloadBytesWritten();
+ hbData.datasetNetSignalingBytesRead = datasetNetPC.getSignalingBytesRead();
+ hbData.datasetNetSignalingBytesWritten = datasetNetPC.getSignalingBytesWritten();
+
IPCPerformanceCounters ipcPC = ipc.getPerformanceCounters();
hbData.ipcMessagesSent = ipcPC.getMessageSentCount();
hbData.ipcMessageBytesSent = ipcPC.getMessageBytesSent();
@@ -459,6 +483,10 @@
}
}
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return datasetPartitionManager;
+ }
+
public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
ccs.sendApplicationMessageToCC(data, appName, nodeId);
}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index eba3ec9..5a3e9dd 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -34,6 +34,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -348,6 +349,11 @@
}
@Override
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return ncs.getDatasetPartitionManager();
+ }
+
+ @Override
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
.getApplicationName(), nodeId);
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
new file mode 100644
index 0000000..cecd677
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
@@ -0,0 +1,237 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetMemoryManager {
+ private final Set<Page> availPages;
+
+ private final LeastRecentlyUsedList leastRecentlyUsedList;
+
+ private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
+
+ private final static int FRAME_SIZE = 32768;
+
+ public DatasetMemoryManager(int availableMemory) {
+ availPages = new HashSet<Page>();
+
+ // Atleast have one page for temporarily storing the results.
+ if (availableMemory <= 0)
+ availableMemory = FRAME_SIZE;
+
+ while (availableMemory >= FRAME_SIZE) {
+ /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
+ * instead of direct ByteBuffer.allocate()?
+ */
+ availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+ availableMemory -= FRAME_SIZE;
+ }
+
+ leastRecentlyUsedList = new LeastRecentlyUsedList();
+ resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
+ }
+
+ public Page requestPage(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter dpw)
+ throws OutOfMemoryError, HyracksDataException {
+ Page page;
+ if (availPages.isEmpty()) {
+ page = evictPage();
+ } else {
+ page = getAvailablePage();
+ }
+
+ page.clear();
+
+ /*
+ * It is extremely important to update the reference after obtaining the page because, in the cases where memory
+ * manager is allocated only one page of memory, the front of the LRU list should not be created by the
+ * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
+ * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
+ */
+ PartitionNode pn = updateReference(resultSetPartitionId, dpw);
+ pn.add(page);
+ return page;
+ }
+
+ public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
+ // When a page is referenced the dataset partition writer should already be known, so we pass null.
+ updateReference(resultSetPartitionId, null);
+ }
+
+ public int getPageSize() {
+ return FRAME_SIZE;
+ }
+
+ protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
+ leastRecentlyUsedList.add(pn);
+ resultPartitionNodesMap.put(resultSetPartitionId, pn);
+ }
+
+ protected synchronized PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId,
+ IDatasetPartitionWriter dpw) {
+ PartitionNode pn = null;
+
+ if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
+ if (dpw != null) {
+ pn = new PartitionNode(resultSetPartitionId, dpw);
+ insertPartitionNode(resultSetPartitionId, pn);
+ }
+ return pn;
+ }
+ pn = resultPartitionNodesMap.get(resultSetPartitionId);
+ leastRecentlyUsedList.remove(pn);
+ insertPartitionNode(resultSetPartitionId, pn);
+
+ return pn;
+ }
+
+ protected synchronized Page evictPage() throws HyracksDataException {
+ PartitionNode pn = leastRecentlyUsedList.getFirst();
+ IDatasetPartitionWriter dpw = pn.getDatasetPartitionWriter();
+ Page page = dpw.returnPage();
+
+ /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
+ * away all the pages allocated to it and add to the available pages set.
+ */
+ if (page == null) {
+ availPages.addAll(pn);
+ pn.clear();
+ resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+ leastRecentlyUsedList.remove(pn);
+
+ /* Based on the assumption that if the dataset partition writer returned a null page, it should be lying about
+ * the number of pages it holds in which case we just evict all the pages it holds and should thus be able to
+ * add all those pages to available set and we have at least one page to allocate back.
+ */
+ page = getAvailablePage();
+ } else {
+ pn.remove(page);
+
+ // If the partition no more holds any pages, remove it from the linked list and the hash map.
+ if (pn.isEmpty()) {
+ resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+ leastRecentlyUsedList.remove(pn);
+ }
+ }
+
+ return page;
+ }
+
+ protected synchronized Page getAvailablePage() {
+ Iterator<Page> iter = availPages.iterator();
+ Page page = iter.next();
+ iter.remove();
+ return page;
+ }
+
+ private class LeastRecentlyUsedList {
+ private PartitionNode head;
+
+ private PartitionNode tail;
+
+ public LeastRecentlyUsedList() {
+ head = null;
+ tail = null;
+ }
+
+ public void add(PartitionNode node) {
+ if (head == null) {
+ head = tail = node;
+ return;
+ }
+ tail.setNext(node);
+ node.setPrev(tail);
+ tail = node;
+ }
+
+ public void remove(PartitionNode node) {
+ if ((node == head) && (node == tail)) {
+ head = tail = null;
+ return;
+ } else if (node == head) {
+ head = head.getNext();
+ head.setPrev(null);
+ return;
+ } else if (node == tail) {
+ tail = tail.getPrev();
+ tail.setNext(null);
+ return;
+ } else {
+ PartitionNode prev = node.getPrev();
+ PartitionNode next = node.getNext();
+ prev.setNext(next);
+ next.setPrev(prev);
+ }
+ }
+
+ public PartitionNode getFirst() {
+ return head;
+ }
+ }
+
+ private class PartitionNode extends HashSet<Page> {
+ private static final long serialVersionUID = 1L;
+
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final IDatasetPartitionWriter datasetPartitionWriter;
+
+ private PartitionNode prev;
+
+ private PartitionNode next;
+
+ public PartitionNode(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter datasetPartitionWriter) {
+ this.resultSetPartitionId = resultSetPartitionId;
+ this.datasetPartitionWriter = datasetPartitionWriter;
+ prev = null;
+ next = null;
+ }
+
+ public ResultSetPartitionId getResultSetPartitionId() {
+ return resultSetPartitionId;
+ }
+
+ public IDatasetPartitionWriter getDatasetPartitionWriter() {
+ return datasetPartitionWriter;
+ }
+
+ public void setPrev(PartitionNode node) {
+ prev = node;
+ }
+
+ public PartitionNode getPrev() {
+ return prev;
+ }
+
+ public void setNext(PartitionNode node) {
+ next = node;
+ }
+
+ public PartitionNode getNext() {
+ return next;
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
new file mode 100644
index 0000000..1cad54b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -0,0 +1,124 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class DatasetPartitionManager implements IDatasetPartitionManager {
+ private final NodeControllerService ncs;
+
+ private final Executor executor;
+
+ private final Map<JobId, ResultState[]> partitionResultStateMap;
+
+ private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+ private final IWorkspaceFileFactory fileFactory;
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory) {
+ this.ncs = ncs;
+ this.executor = executor;
+ partitionResultStateMap = new HashMap<JobId, ResultState[]>();
+ deallocatableRegistry = new DefaultDeallocatableRegistry();
+ fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+ datasetMemoryManager = new DatasetMemoryManager(availableMemory);
+ }
+
+ @Override
+ public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions) throws HyracksException {
+ DatasetPartitionWriter dpw = null;
+ JobId jobId = ctx.getJobletContext().getJobId();
+ try {
+ ncs.getClusterController().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+ nPartitions, ncs.getDatasetNetworkManager().getNetworkAddress());
+ dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager);
+
+ ResultState[] resultStates = partitionResultStateMap.get(jobId);
+ if (resultStates == null) {
+ resultStates = new ResultState[nPartitions];
+ partitionResultStateMap.put(jobId, resultStates);
+ }
+ resultStates[partition] = dpw.getResultState();
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+
+ return dpw;
+ }
+
+ @Override
+ public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+ try {
+ ncs.getClusterController().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ @Override
+ public void reportPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+ try {
+ ncs.getClusterController().reportResultPartitionFailure(jobId, rsId, partition);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ @Override
+ public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter writer)
+ throws HyracksException {
+ ResultState[] resultStates = partitionResultStateMap.get(jobId);
+
+ if (resultStates == null) {
+ throw new HyracksException("Unknown JobId " + jobId);
+ }
+
+ ResultState resultState = resultStates[partition];
+ if (resultState == null) {
+ throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
+ }
+
+ IDatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
+ dpr.writeTo(writer);
+ }
+
+ @Override
+ public IWorkspaceFileFactory getFileFactory() {
+ return fileFactory;
+ }
+
+ @Override
+ public void close() {
+ deallocatableRegistry.close();
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
new file mode 100644
index 0000000..296c502
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
@@ -0,0 +1,122 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+
+public class DatasetPartitionReader implements IDatasetPartitionReader {
+ private static final Logger LOGGER = Logger.getLogger(DatasetPartitionReader.class.getName());
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ private final Executor executor;
+
+ private final ResultState resultState;
+
+ private IFileHandle fileHandle;
+
+ public DatasetPartitionReader(DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
+ this.datasetMemoryManager = datasetMemoryManager;
+ this.executor = executor;
+ this.resultState = resultState;
+ }
+
+ private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+ long readSize = 0;
+ synchronized (resultState) {
+ while (offset >= resultState.getSize() && !resultState.getEOS()) {
+ try {
+ resultState.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ if (offset >= resultState.getSize() && resultState.getEOS()) {
+ return readSize;
+ }
+
+ if (offset < resultState.getPersistentSize()) {
+ readSize = resultState.getIOManager().syncRead(fileHandle, offset, buffer);
+ }
+
+ if (readSize < buffer.capacity()) {
+ long localPageOffset = offset - resultState.getPersistentSize();
+ int localPageIndex = (int) (localPageOffset / datasetMemoryManager.getPageSize());
+ int pageOffset = (int) (localPageOffset % datasetMemoryManager.getPageSize());
+ Page page = resultState.getPage(localPageIndex);
+ readSize += buffer.remaining();
+ buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+ }
+
+ datasetMemoryManager.pageReferenced(resultState.getResultSetPartitionId());
+ return readSize;
+ }
+
+ @Override
+ public void writeTo(final IFrameWriter writer) {
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ NetworkOutputChannel channel = (NetworkOutputChannel) writer;
+ channel.setFrameSize(resultState.getFrameSize());
+ try {
+ fileHandle = resultState.getIOManager().open(resultState.getValidFileReference(),
+ IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ channel.open();
+ try {
+ long offset = 0;
+ ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
+ while (true) {
+ buffer.clear();
+ long size = read(offset, buffer);
+ if (size <= 0) {
+ break;
+ } else if (size < buffer.limit()) {
+ throw new HyracksDataException("Premature end of file - readSize: " + size
+ + " buffer limit: " + buffer.limit());
+ }
+ offset += size;
+ buffer.flip();
+ channel.nextFrame(buffer);
+ }
+ } finally {
+ channel.close();
+ resultState.getIOManager().close(fileHandle);
+ }
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (HyracksDataException e) {
+ throw new RuntimeException(e);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("result reading successful(" + resultState.getResultSetPartitionId() + ")");
+ }
+ }
+ });
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
new file mode 100644
index 0000000..f6ae540
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetPartitionWriter implements IDatasetPartitionWriter {
+ private static final Logger LOGGER = Logger.getLogger(DatasetPartitionWriter.class.getName());
+
+ private static final String FILE_PREFIX = "result_";
+
+ private final IDatasetPartitionManager manager;
+
+ private final JobId jobId;
+
+ private final ResultSetId resultSetId;
+
+ private final int partition;
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final ResultState resultState;
+
+ private IFileHandle fileHandle;
+
+ public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
+ ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager) {
+ this.manager = manager;
+ this.jobId = jobId;
+ this.resultSetId = rsId;
+ this.partition = partition;
+ this.datasetMemoryManager = datasetMemoryManager;
+
+ resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
+ resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), ctx.getFrameSize());
+ }
+
+ public ResultState getResultState() {
+ return resultState;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("open(" + partition + ")");
+ }
+ String fName = FILE_PREFIX + String.valueOf(partition);
+ FileReference fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(fName);
+ fileHandle = resultState.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ resultState.init(fRef);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ int srcOffset = 0;
+ Page destPage = resultState.getLastPage();
+
+ while (srcOffset < buffer.limit()) {
+ if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+ destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
+ resultState.addPage(destPage);
+ }
+ int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+ destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+ srcOffset += srcLength;
+ resultState.incrementSize(srcLength);
+ }
+
+ synchronized (resultState) {
+ resultState.notifyAll();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ try {
+ manager.reportPartitionFailure(jobId, resultSetId, partition);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("close(" + partition + ")");
+ }
+
+ try {
+ synchronized (resultState) {
+ resultState.setEOS(true);
+ resultState.notifyAll();
+ }
+ manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public Page returnPage() throws HyracksDataException {
+ Page page = resultState.removePage(0);
+
+ IIOManager ioManager = resultState.getIOManager();
+
+ // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
+ if (page == null) {
+ ioManager.close(fileHandle);
+ return null;
+ }
+
+ page.getBuffer().flip();
+
+ long delta = ioManager.syncWrite(fileHandle, resultState.getPersistentSize(), page.getBuffer());
+ resultState.incrementPersistentSize(delta);
+ return page;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
new file mode 100644
index 0000000..3db3fd9
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -0,0 +1,173 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.dataset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class ResultState implements IStateObject {
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final int frameSize;
+
+ private final IIOManager ioManager;
+
+ private final AtomicBoolean eos;
+
+ private final AtomicBoolean readEOS;
+
+ private final List<Page> localPageList;
+
+ private FileReference fileRef;
+
+ private long size;
+
+ private long persistentSize;
+
+ ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, int frameSize) {
+ this.resultSetPartitionId = resultSetPartitionId;
+ this.ioManager = ioManager;
+ this.frameSize = frameSize;
+ eos = new AtomicBoolean(false);
+ readEOS = new AtomicBoolean(false);
+ localPageList = new ArrayList<Page>();
+ }
+
+ public synchronized void init(FileReference fileRef) {
+ this.fileRef = fileRef;
+
+ size = 0;
+ persistentSize = 0;
+ notifyAll();
+ }
+
+ public ResultSetPartitionId getResultSetPartitionId() {
+ return resultSetPartitionId;
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ public IIOManager getIOManager() {
+ return ioManager;
+ }
+
+ public synchronized void incrementSize(long delta) {
+ size += delta;
+ }
+
+ public synchronized long getSize() {
+ return size;
+ }
+
+ public synchronized void incrementPersistentSize(long delta) {
+ persistentSize += delta;
+ }
+
+ public synchronized long getPersistentSize() {
+ return persistentSize;
+ }
+
+ public void setEOS(boolean eos) {
+ this.eos.set(eos);
+ }
+
+ public boolean getEOS() {
+ return eos.get();
+ }
+
+ public boolean getReadEOS() {
+ return readEOS.get();
+ }
+
+ public synchronized void addPage(Page page) {
+ localPageList.add(page);
+ }
+
+ public synchronized Page removePage(int index) {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.remove(index);
+ }
+ return page;
+ }
+
+ public synchronized Page getPage(int index) {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(index);
+ }
+ return page;
+ }
+
+ public synchronized Page getLastPage() {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(localPageList.size() - 1);
+ }
+ return page;
+ }
+
+ public synchronized Page getFirstPage() {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(0);
+ }
+ return page;
+ }
+
+ public synchronized FileReference getValidFileReference() throws InterruptedException {
+ while (fileRef == null)
+ wait();
+ return fileRef;
+ }
+
+ @Override
+ public JobId getJobId() {
+ return resultSetPartitionId.getJobId();
+ }
+
+ @Override
+ public Object getId() {
+ return resultSetPartitionId;
+ }
+
+ @Override
+ public long getMemoryOccupancy() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void toBytes(DataOutput out) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void fromBytes(DataInput in) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
new file mode 100644
index 0000000..5b8b333
--- /dev/null
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class DatasetNetworkManager implements IChannelConnectionFactory {
+ private static final Logger LOGGER = Logger.getLogger(DatasetNetworkManager.class.getName());
+
+ private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IDatasetPartitionManager partitionManager;
+
+ private final MuxDemux md;
+
+ private NetworkAddress networkAddress;
+
+ public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads)
+ throws IOException {
+ this.partitionManager = partitionManager;
+ md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
+ MAX_CONNECTION_ATTEMPTS);
+ }
+
+ public void start() throws IOException {
+ md.start();
+ InetSocketAddress sockAddr = md.getLocalAddress();
+ networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+
+ public void stop() {
+
+ }
+
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+ MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+ return mConn.openChannel();
+ }
+
+ private class ChannelOpenListener implements IChannelOpenListener {
+ @Override
+ public void channelOpened(ChannelControlBlock channel) {
+ channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+ channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+ }
+ }
+
+ private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+ private final ChannelControlBlock ccb;
+
+ private NetworkOutputChannel noc;
+
+ public InitialBufferAcceptor(ChannelControlBlock ccb) {
+ this.ccb = ccb;
+ }
+
+ @Override
+ public void accept(ByteBuffer buffer) {
+ JobId jobId = new JobId(buffer.getLong());
+ int partition = buffer.getInt();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
+ + partition + " on channel: " + ccb);
+ }
+ noc = new NetworkOutputChannel(ccb, 1);
+ try {
+ partitionManager.initializeDatasetPartitionReader(jobId, partition, noc);
+ } catch (HyracksException e) {
+ noc.abort();
+ }
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void error(int ecode) {
+ if (noc != null) {
+ noc.abort();
+ }
+ }
+ }
+
+ public MuxDemuxPerformanceCounters getPerformanceCounters() {
+ return md.getPerformanceCounters();
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
index b805595..c8e4e94 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -27,6 +27,8 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
import edu.uci.ics.hyracks.net.exceptions.NetException;
@@ -36,7 +38,7 @@
import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
-public class NetworkManager {
+public class NetworkManager implements IChannelConnectionFactory {
private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
private static final int MAX_CONNECTION_ATTEMPTS = 5;
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 16e31f7..ba6e6c3 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.partitions.IPartition;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
@@ -82,7 +82,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
for (int i = 0; i < nBuffers; ++i) {
emptyQueue.add(ctx.allocateFrame());
}
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index 45c091a..ea88a75 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -28,12 +28,12 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.IPartition;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
import edu.uci.ics.hyracks.control.common.job.PartitionState;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.net.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
public class PartitionManager {
@@ -98,7 +98,7 @@
List<IPartition> pList = partitionMap.get(partitionId);
if (pList != null && !pList.isEmpty()) {
IPartition partition = pList.get(0);
- writer.setTaskContext(partition.getTaskContext());
+ writer.setFrameSize(partition.getTaskContext().getFrameSize());
partition.writeTo(writer);
if (!partition.isReusable()) {
partitionMap.remove(partitionId);
diff --git a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index bb9669d..7ed9d11 100644
--- a/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -22,10 +22,10 @@
import edu.uci.ics.hyracks.api.comm.PartitionChannel;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkInputChannel;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
import edu.uci.ics.hyracks.control.nc.Joblet;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
public class ReportPartitionAvailabilityWork extends AbstractWork {
private final NodeControllerService ncs;
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml b/fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
index 3f051f9..8f5f04e 100644
--- a/fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
@@ -16,8 +16,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
new file mode 100644
index 0000000..c90ce5a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/RawUTF8StringPointable.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.data.std.primitive;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.data.std.api.AbstractPointable;
+import edu.uci.ics.hyracks.data.std.api.IComparable;
+import edu.uci.ics.hyracks.data.std.api.IHashable;
+import edu.uci.ics.hyracks.data.std.api.IPointable;
+import edu.uci.ics.hyracks.data.std.api.IPointableFactory;
+
+/**
+ * This class provides the raw bytes-based comparison and hash function for UTF8 strings.
+ * Note that the comparison may not deliver the correct ordering for certain languages that include 2 or 3 bytes characters.
+ * But it works for single-byte character languages.
+ */
+public final class RawUTF8StringPointable extends AbstractPointable implements IHashable, IComparable {
+ public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean isFixedLength() {
+ return false;
+ }
+
+ @Override
+ public int getFixedLength() {
+ return 0;
+ }
+ };
+
+ public static final IPointableFactory FACTORY = new IPointableFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IPointable createPointable() {
+ return new RawUTF8StringPointable();
+ }
+
+ @Override
+ public ITypeTraits getTypeTraits() {
+ return TYPE_TRAITS;
+ }
+ };
+
+ @Override
+ public int compareTo(IPointable pointer) {
+ return compareTo(pointer.getByteArray(), pointer.getStartOffset(), pointer.getLength());
+ }
+
+ @Override
+ public int compareTo(byte[] bytes, int start, int length) {
+ int utflen1 = UTF8StringPointable.getUTFLength(this.bytes, this.start);
+ int utflen2 = UTF8StringPointable.getUTFLength(bytes, start);
+
+ int c1 = 0;
+ int c2 = 0;
+
+ int s1Start = this.start + 2;
+ int s2Start = start + 2;
+
+ while (c1 < utflen1 && c2 < utflen2) {
+ char ch1 = (char) this.bytes[s1Start + c1];
+ char ch2 = (char) bytes[s2Start + c2];
+
+ if (ch1 != ch2) {
+ return ch1 - ch2;
+ }
+ c1++;
+ c2++;
+ }
+ return utflen1 - utflen2;
+ }
+
+ @Override
+ public int hash() {
+ int h = 0;
+ int utflen = UTF8StringPointable.getUTFLength(bytes, start);
+ int sStart = start + 2;
+ int c = 0;
+
+ while (c < utflen) {
+ char ch = (char) bytes[sStart + c];
+ h = 31 * h + ch;
+ c++;
+ }
+ return h;
+ }
+
+ public void toString(StringBuilder buffer) {
+ UTF8StringPointable.toString(buffer, bytes, start);
+ }
+}
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
index f6d6093..866ebb0 100644
--- a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -216,4 +216,4 @@
public void toString(StringBuilder buffer) {
toString(buffer, bytes, start);
}
-}
\ No newline at end of file
+}
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
index e8dc9b4..7d7d2c1 100644
--- a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ArrayBackedValueStorage.java
@@ -1,34 +1,28 @@
package edu.uci.ics.hyracks.data.std.util;
import java.io.DataOutput;
-import java.io.DataOutputStream;
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.api.IMutableValueStorage;
import edu.uci.ics.hyracks.data.std.api.IValueReference;
public class ArrayBackedValueStorage implements IMutableValueStorage {
- private final ByteArrayAccessibleOutputStream baaos;
- private final DataOutputStream dos;
-
- public ArrayBackedValueStorage() {
- baaos = new ByteArrayAccessibleOutputStream();
- dos = new DataOutputStream(baaos);
- }
+
+ private final GrowableArray data = new GrowableArray();
@Override
public void reset() {
- baaos.reset();
+ data.reset();
}
@Override
public DataOutput getDataOutput() {
- return dos;
+ return data.getDataOutput();
}
@Override
public byte[] getByteArray() {
- return baaos.getByteArray();
+ return data.getByteArray();
}
@Override
@@ -38,12 +32,12 @@
@Override
public int getLength() {
- return baaos.size();
+ return data.getLength();
}
public void append(IValueReference value) {
try {
- dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+ data.append(value);
} catch (IOException e) {
e.printStackTrace();
}
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
index 8508287..c0ba163 100644
--- a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
@@ -21,6 +21,14 @@
public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
+ public ByteArrayAccessibleOutputStream() {
+ super();
+ }
+
+ public ByteArrayAccessibleOutputStream(int size) {
+ super(size);
+ }
+
public byte[] getByteArray() {
return buf;
}
diff --git a/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
new file mode 100644
index 0000000..c174d4e
--- /dev/null
+++ b/fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/GrowableArray.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.data.std.util;
+
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.api.IValueReference;
+
+public class GrowableArray implements IDataOutputProvider {
+ private final ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
+ private final DataOutputStream dos = new DataOutputStream(baaos);
+
+ @Override
+ public DataOutput getDataOutput() {
+ return dos;
+ }
+
+ public void reset() {
+ baaos.reset();
+ }
+
+ public byte[] getByteArray() {
+ return baaos.getByteArray();
+ }
+
+ public int getLength() {
+ return baaos.size();
+ }
+
+ public void append(IValueReference value) throws IOException {
+ dos.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+ }
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/pom.xml b/fullstack/hyracks/hyracks-dataflow-common/pom.xml
index a0ffb66..1a2950b 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/pom.xml
+++ b/fullstack/hyracks/hyracks-dataflow-common/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index 989bc6b..8c865c4 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -15,14 +15,13 @@
package edu.uci.ics.hyracks.dataflow.common.comm.io;
import java.io.DataOutput;
-import java.io.DataOutputStream;
import java.io.IOException;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
/**
* Array backed tuple builder.
@@ -30,25 +29,21 @@
* @author vinayakb
*/
public class ArrayTupleBuilder implements IDataOutputProvider {
- private final ByteArrayAccessibleOutputStream baaos;
- private final DataOutputStream dos;
+ private final GrowableArray fieldData = new GrowableArray();
private final int[] fEndOffsets;
private int nextField;
public ArrayTupleBuilder(int nFields) {
- baaos = new ByteArrayAccessibleOutputStream();
- dos = new DataOutputStream(baaos);
fEndOffsets = new int[nFields];
}
/**
* Resets the builder.
- *
* reset() must be called before attempting to create a new tuple.
*/
public void reset() {
nextField = 0;
- baaos.reset();
+ fieldData.reset();
}
/**
@@ -66,7 +61,7 @@
* @return Data byte array.
*/
public byte[] getByteArray() {
- return baaos.getByteArray();
+ return fieldData.getByteArray();
}
/**
@@ -75,7 +70,7 @@
* @return data area size.
*/
public int getSize() {
- return baaos.size();
+ return fieldData.getLength();
}
/**
@@ -96,14 +91,15 @@
int fStartOffset = accessor.getFieldStartOffset(tIndex, fIndex);
int fLen = accessor.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
try {
- dos.write(accessor.getBuffer().array(), startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
+ fieldData.getDataOutput().write(accessor.getBuffer().array(),
+ startOffset + accessor.getFieldSlotsLength() + fStartOffset, fLen);
if (FrameConstants.DEBUG_FRAME_IO) {
- dos.writeInt(FrameConstants.FRAME_FIELD_MAGIC);
+ fieldData.getDataOutput().writeInt(FrameConstants.FRAME_FIELD_MAGIC);
}
} catch (IOException e) {
throw new HyracksDataException(e);
}
- fEndOffsets[nextField++] = baaos.size();
+ fEndOffsets[nextField++] = fieldData.getLength();
}
/**
@@ -117,8 +113,8 @@
* @throws HyracksDataException
*/
public <T> void addField(ISerializerDeserializer<T> serDeser, T instance) throws HyracksDataException {
- serDeser.serialize(instance, dos);
- fEndOffsets[nextField++] = baaos.size();
+ serDeser.serialize(instance, fieldData.getDataOutput());
+ fEndOffsets[nextField++] = fieldData.getLength();
}
/**
@@ -134,11 +130,11 @@
*/
public void addField(byte[] bytes, int start, int length) throws HyracksDataException {
try {
- dos.write(bytes, start, length);
+ fieldData.getDataOutput().write(bytes, start, length);
} catch (IOException e) {
throw new HyracksDataException(e);
}
- fEndOffsets[nextField++] = baaos.size();
+ fEndOffsets[nextField++] = fieldData.getLength();
}
/**
@@ -146,7 +142,14 @@
*/
@Override
public DataOutput getDataOutput() {
- return dos;
+ return fieldData.getDataOutput();
+ }
+
+ /**
+ * Get the growable array storing the field data.
+ */
+ public GrowableArray getFieldData() {
+ return fieldData;
}
/**
@@ -156,6 +159,6 @@
* data.
*/
public void addFieldEndOffset() {
- fEndOffsets[nextField++] = baaos.size();
+ fEndOffsets[nextField++] = fieldData.getLength();
}
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
new file mode 100644
index 0000000..07f6ba2
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
+ private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
+
+ private final FrameTupleAppender frameTupleAppender;
+
+ public FrameOutputStream(int frameSize) {
+ super(frameSize);
+ this.frameTupleAppender = new FrameTupleAppender(frameSize);
+ }
+
+ public void reset(ByteBuffer buffer, boolean clear) {
+ frameTupleAppender.reset(buffer, clear);
+ }
+
+ public int getTupleCount() {
+ int tupleCount = frameTupleAppender.getTupleCount();
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("appendTuple(): tuple count: " + tupleCount);
+ }
+ return tupleCount;
+ }
+
+ public boolean appendTuple() {
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("appendTuple(): tuple size: " + count);
+ }
+ boolean appended = frameTupleAppender.append(buf, 0, count);
+ count = 0;
+ return appended;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
new file mode 100644
index 0000000..915a436
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+
+ private final int frameSize;
+ private ByteBuffer buffer;
+
+ public ResultFrameTupleAccessor(int frameSize) {
+ this.frameSize = frameSize;
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public int getTupleCount() {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return getFieldCount() * 4;
+ }
+
+ public void prettyPrint() {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ int tc = getTupleCount();
+ System.err.println("TC: " + tc);
+ for (int i = 0; i < tc; ++i) {
+ System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
+
+ bbis.setByteBuffer(buffer, getTupleStartOffset(i));
+ System.err.print(dis);
+
+ System.err.println("]");
+ }
+ }
+
+ @Override
+ public int getFieldCount() {
+ return 1;
+ }
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..e95e9c2
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/DoubleNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class DoubleNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+ if (prefix >= 0) {
+ return prefix ^ Integer.MIN_VALUE;
+ } else {
+ return (int) ((long) 0xffffffff - (long) prefix);
+ }
+ }
+
+ };
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..d58afc1
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/FloatNormalizedKeyComputerFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+public class FloatNormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ int prefix = IntegerSerializerDeserializer.getInt(bytes, start);
+ if (prefix >= 0) {
+ return prefix ^ Integer.MIN_VALUE;
+ } else {
+ return (int) ((long) 0xffffffff - (long) prefix);
+ }
+ }
+
+ };
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
new file mode 100644
index 0000000..4589909
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/Integer64NormalizedKeyComputerFactory.java
@@ -0,0 +1,55 @@
+package edu.uci.ics.hyracks.dataflow.common.data.normalizers;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+
+public class Integer64NormalizedKeyComputerFactory implements INormalizedKeyComputerFactory {
+
+ private static final long serialVersionUID = 8735044913496854551L;
+
+ @Override
+ public INormalizedKeyComputer createNormalizedKeyComputer() {
+ return new INormalizedKeyComputer() {
+ private static final int POSTIVE_LONG_MASK = (3 << 30);
+ private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
+ private static final int NEGATIVE_LONG_MASK = (0 << 30);
+
+ @Override
+ public int normalize(byte[] bytes, int start, int length) {
+ long value = Integer64SerializerDeserializer.getLong(bytes, start);
+ int highValue = (int) (value >> 32);
+ if (highValue > 0) {
+ /**
+ * larger than Integer.MAX
+ */
+ int highNmk = getKey(highValue);
+ highNmk >>= 2;
+ highNmk |= POSTIVE_LONG_MASK;
+ return highNmk;
+ } else if (highValue == 0) {
+ /**
+ * smaller than Integer.MAX but >=0
+ */
+ int lowNmk = (int) value;
+ lowNmk >>= 2;
+ lowNmk |= NON_NEGATIVE_INT_MASK;
+ return lowNmk;
+ } else {
+ /**
+ * less than 0: have not optimized for that
+ */
+ int highNmk = getKey(highValue);
+ highNmk >>= 2;
+ highNmk |= NEGATIVE_LONG_MASK;
+ return highNmk;
+ }
+ }
+
+ private int getKey(int value) {
+ return value ^ Integer.MIN_VALUE;
+ }
+
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
index 2f7a778..6a01842 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/normalizers/IntegerNormalizedKeyComputerFactory.java
@@ -27,8 +27,7 @@
@Override
public int normalize(byte[] bytes, int start, int length) {
int value = IntegerSerializerDeserializer.getInt(bytes, start);
- long unsignedValue = (long) value;
- return (int) ((unsignedValue - ((long) Integer.MIN_VALUE)) & 0xffffffffL);
+ return value ^Integer.MIN_VALUE;
}
};
}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
index 6a9dab7..d9191c7 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/parsers/LongParserFactory.java
@@ -32,7 +32,7 @@
return new IValueParser() {
@Override
public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
- long n = 0;
+ int n = 0;
int sign = 1;
int i = 0;
boolean pre = true;
@@ -102,7 +102,7 @@
throw new HyracksDataException("Encountered " + ch);
}
}
-
+
try {
out.writeLong(n * sign);
} catch (IOException e) {
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
index 51645c4..ec3c2be 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
@@ -52,10 +52,10 @@
h += fh;
}
if (h < 0) {
- h = -h;
+ h = -(h+1);
}
return h % nParts;
}
};
}
-}
\ No newline at end of file
+}
diff --git a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
index a29209c..8a30a71 100644
--- a/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
+++ b/fullstack/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/data/util/StringUtils.java
@@ -18,17 +18,19 @@
import java.io.IOException;
public class StringUtils {
- public static void writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
-
+ public static int writeCharAsModifiedUTF8(char c, DataOutput dos) throws IOException {
if (c >= 0x0000 && c <= 0x007F) {
dos.writeByte(c);
+ return 1;
} else if (c <= 0x07FF) {
dos.writeByte((byte) (0xC0 | ((c >> 6) & 0x3F)));
dos.writeByte((byte) (0x80 | (c & 0x3F)));
+ return 2;
} else {
dos.writeByte((byte) (0xE0 | ((c >> 12) & 0x0F)));
dos.writeByte((byte) (0x80 | ((c >> 6) & 0x3F)));
dos.writeByte((byte) (0x80 | (c & 0x3F)));
+ return 3;
}
}
diff --git a/fullstack/hyracks/hyracks-dataflow-hadoop/pom.xml b/fullstack/hyracks/hyracks-dataflow-hadoop/pom.xml
index f9d5153..f5135f8 100644
--- a/fullstack/hyracks/hyracks-dataflow-hadoop/pom.xml
+++ b/fullstack/hyracks/hyracks-dataflow-hadoop/pom.xml
@@ -1,8 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-dataflow-hadoop</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hyracks-dataflow-hadoop</name>
<parent>
@@ -18,8 +16,8 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-dataflow-std/pom.xml b/fullstack/hyracks/hyracks-dataflow-std/pom.xml
index bf27d20..2cf0fdc 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -1,8 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-dataflow-std</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hyracks-dataflow-std</name>
<parent>
@@ -18,8 +16,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index f2b56fa..f86d9fb 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -256,8 +256,6 @@
outputAppender.reset(outputFrame, true);
- writer.open();
-
if (tPointers == null) {
// Not sorted
for (int i = 0; i < tableSize; ++i) {
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 3e5e30f..d86f1d5 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -17,7 +17,6 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
@@ -47,11 +46,18 @@
private final ISerializableTable table;
private final int tableSize;
private final TuplePointer storedTuplePointer;
+ private final boolean reverseOutputOrder; //Should we reverse the order of tuples, we are writing in output
public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table)
throws HyracksDataException {
+ this(ctx, tableSize, accessor0, tpc0, accessor1, tpc1, comparator, isLeftOuter, nullWriters1, table, false);
+ }
+
+ public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
+ ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
+ FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1, ISerializableTable table, boolean reverse) throws HyracksDataException {
this.tableSize = tableSize;
this.table = table;
storedTuplePointer = new TuplePointer();
@@ -76,6 +82,7 @@
} else {
nullTupleBuild = null;
}
+ reverseOutputOrder = reverse;
}
public void build(ByteBuffer buffer) throws HyracksDataException {
@@ -108,18 +115,13 @@
int c = tpComparator.compare(accessorProbe, i, accessorBuild, tIndex);
if (c == 0) {
matchFound = true;
- if (!appender.appendConcat(accessorProbe, i, accessorBuild, tIndex)) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessorProbe, i, accessorBuild, tIndex)) {
- throw new IllegalStateException();
- }
- }
+ appendToResult(i, tIndex, writer);
}
} while (true);
if (!matchFound && isLeftOuter) {
- if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
+
+ if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
flushFrame(outBuffer, writer);
appender.reset(outBuffer, true);
@@ -128,6 +130,7 @@
throw new IllegalStateException();
}
}
+
}
}
}
@@ -145,23 +148,25 @@
buffer.position(0);
buffer.limit(buffer.capacity());
}
-
- private static class Link {
- private static final int INIT_POINTERS_SIZE = 8;
-
- long[] pointers;
- int size;
-
- Link() {
- pointers = new long[INIT_POINTERS_SIZE];
- size = 0;
- }
-
- void add(long pointer) {
- if (size >= pointers.length) {
- pointers = Arrays.copyOf(pointers, pointers.length * 2);
+
+ private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException{
+ if(!reverseOutputOrder){
+ if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
+ flushFrame(outBuffer, writer);
+ appender.reset(outBuffer, true);
+ if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
+ throw new IllegalStateException();
+ }
}
- pointers[size++] = pointer;
- }
+ }
+ else{
+ if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
+ flushFrame(outBuffer, writer);
+ appender.reset(outBuffer, true);
+ if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
+ throw new IllegalStateException();
+ }
+ }
+ }
}
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index 7e84229..6870e71 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -14,15 +14,18 @@
*/
package edu.uci.ics.hyracks.dataflow.std.join;
+import java.io.DataOutput;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -42,9 +45,12 @@
private RunFileReader runFileReader;
private int currentMemSize = 0;
private final RunFileWriter runFileWriter;
+ private final boolean isLeftOuter;
+ private final ArrayTupleBuilder nullTupleBuilder;
public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
- ITuplePairComparator comparators, int memSize) throws HyracksDataException {
+ ITuplePairComparator comparators, int memSize, boolean isLeftOuter, INullWriter[] nullWriters1)
+ throws HyracksDataException {
this.accessorInner = accessor1;
this.accessorOuter = accessor0;
this.appender = new FrameTupleAppender(ctx.getFrameSize());
@@ -56,6 +62,19 @@
this.memSize = memSize;
this.ctx = ctx;
+ this.isLeftOuter = isLeftOuter;
+ if (isLeftOuter) {
+ int innerFieldCount = accessorInner.getFieldCount();
+ nullTupleBuilder = new ArrayTupleBuilder(innerFieldCount);
+ DataOutput out = nullTupleBuilder.getDataOutput();
+ for (int i = 0; i < innerFieldCount; i++) {
+ nullWriters1[i].writeNull(out);
+ nullTupleBuilder.addFieldEndOffset();
+ }
+ } else {
+ nullTupleBuilder = null;
+ }
+
FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
this.getClass().getSimpleName() + this.toString());
runFileWriter = new RunFileWriter(file, ctx.getIOManager());
@@ -108,9 +127,11 @@
int tupleCount1 = accessorInner.getTupleCount();
for (int i = 0; i < tupleCount0; ++i) {
+ boolean matchFound = false;
for (int j = 0; j < tupleCount1; ++j) {
int c = compare(accessorOuter, i, accessorInner, j);
if (c == 0) {
+ matchFound = true;
if (!appender.appendConcat(accessorOuter, i, accessorInner, j)) {
flushFrame(outBuffer, writer);
appender.reset(outBuffer, true);
@@ -120,6 +141,18 @@
}
}
}
+
+ if (!matchFound && isLeftOuter) {
+ if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+ nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+ flushFrame(outBuffer, writer);
+ appender.reset(outBuffer, true);
+ if (!appender.appendConcat(accessorOuter, i, nullTupleBuilder.getFieldEndOffsets(),
+ nullTupleBuilder.getByteArray(), 0, nullTupleBuilder.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
}
}
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index a699703..0be01c1 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -25,6 +25,8 @@
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
@@ -47,13 +49,18 @@
private static final long serialVersionUID = 1L;
private final ITuplePairComparatorFactory comparatorFactory;
private final int memSize;
+ private final boolean isLeftOuter;
+ private final INullWriterFactory[] nullWriterFactories1;
public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
- ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize) {
+ ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
+ boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
super(spec, 2, 1);
this.comparatorFactory = comparatorFactory;
this.recordDescriptors[0] = recordDescriptor;
this.memSize = memSize;
+ this.isLeftOuter = isLeftOuter;
+ this.nullWriterFactories1 = nullWriterFactories1;
}
@Override
@@ -111,6 +118,13 @@
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
+ final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
+ if (isLeftOuter) {
+ for (int i = 0; i < nullWriterFactories1.length; i++) {
+ nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
+ }
+ }
+
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
private JoinCacheTaskState state;
@@ -118,8 +132,11 @@
public void open() throws HyracksDataException {
state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
partition));
+
state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
- new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize);
+ new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, isLeftOuter,
+ nullWriters1);
+
}
@Override
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 2905574..6e2b16a 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -81,6 +81,8 @@
private int[] buildPSizeInFrames; //Used for partition tuning
private int freeFramesCounter; //Used for partition tuning
+
+ private boolean isTableEmpty; //Added for handling the case, where build side is empty (tableSize is 0)
public OptimizedHybridHashJoin(IHyracksTaskContext ctx, int memForJoin, int numOfPartitions, String rel0Name,
String rel1Name, int[] keys0, int[] keys1, IBinaryComparator[] comparators, RecordDescriptor buildRd,
@@ -89,10 +91,10 @@
this.memForJoin = memForJoin;
this.buildRd = buildRd;
this.probeRd = probeRd;
- this.buildHpc = probeHpc;
- this.probeHpc = buildHpc;
- this.buildKeys = keys0;
- this.probeKeys = keys1;
+ this.buildHpc = buildHpc;
+ this.probeHpc = probeHpc;
+ this.buildKeys = keys1;
+ this.probeKeys = keys0;
this.comparators = comparators;
this.rel0Name = rel0Name;
this.rel1Name = rel1Name;
@@ -117,10 +119,10 @@
this.memForJoin = memForJoin;
this.buildRd = buildRd;
this.probeRd = probeRd;
- this.buildHpc = probeHpc;
- this.probeHpc = buildHpc;
- this.buildKeys = keys0;
- this.probeKeys = keys1;
+ this.buildHpc = buildHpc;
+ this.probeHpc = probeHpc;
+ this.buildKeys = keys1;
+ this.probeKeys = keys0;
this.comparators = comparators;
this.rel0Name = rel0Name;
this.rel1Name = rel1Name;
@@ -171,6 +173,12 @@
public void build(ByteBuffer buffer) throws HyracksDataException {
accessorBuild.reset(buffer);
int tupleCount = accessorBuild.getTupleCount();
+
+ boolean print = false;
+ if(print){
+ accessorBuild.prettyPrint();
+ }
+
for (int i = 0; i < tupleCount; ++i) {
int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
processTuple(i, pid);
@@ -338,6 +346,7 @@
createInMemoryJoiner(inMemTupCount);
cacheInMemJoin();
+ this.isTableEmpty = (inMemTupCount == 0);
}
private void partitionTune() throws HyracksDataException {
@@ -457,10 +466,14 @@
}
public void probe(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
-
accessorProbe.reset(buffer);
int tupleCount = accessorProbe.getTupleCount();
+ boolean print = false;
+ if(print){
+ accessorProbe.prettyPrint();
+ }
+
if (numOfSpilledParts == 0) {
inMemJoiner.join(buffer, writer);
return;
@@ -604,4 +617,8 @@
+ freeFramesCounter;
return s;
}
+
+ public boolean isTableEmpty(){
+ return this.isTableEmpty;
+ }
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 3a7ee2c..cf39416 100644
--- a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -31,6 +31,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -135,6 +136,7 @@
recordDescriptors[0] = recordDescriptor;
this.isLeftOuter = isLeftOuter;
this.nullWriterFactories1 = nullWriterFactories1;
+
}
@@ -167,10 +169,10 @@
ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(probeAid, buildAid);
builder.addActivity(this, phase1);
- builder.addSourceEdge(0, phase1, 0);
+ builder.addSourceEdge(1, phase1, 0);
builder.addActivity(this, phase2);
- builder.addSourceEdge(1, phase2, 0);
+ builder.addSourceEdge(0, phase2, 0);
builder.addBlockingEdge(phase1, phase2);
@@ -253,14 +255,8 @@
for (int i = 0; i < comparatorFactories.length; i++) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
-
- final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
- if (isLeftOuter) {
- for (int i = 0; i < nullWriterFactories1.length; i++) {
- nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
- }
- }
-
+
+
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
.getJobId(), new TaskId(getActivityId(), partition));
@@ -278,9 +274,17 @@
state.memForJoin = memsize - 2;
state.numOfPartitions = getNumberOfPartitions(state.memForJoin, inputsize0, fudgeFactor,
nPartitions);
- state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
- PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
- buildHpc);
+ if(!isLeftOuter){
+ state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+ PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+ buildHpc);
+ }
+ else{
+ state.hybridHJ = new OptimizedHybridHashJoin(ctx, state.memForJoin, state.numOfPartitions,
+ PROBE_REL, BUILD_REL, probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc,
+ buildHpc, isLeftOuter, nullWriterFactories1);
+ }
+
state.hybridHJ.initBuild();
}
@@ -368,7 +372,9 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- state.hybridHJ.probe(buffer, writer);
+ if(!state.hybridHJ.isTableEmpty()){
+ state.hybridHJ.probe(buffer, writer);
+ }
}
@Override
@@ -418,34 +424,40 @@
//Apply in-Mem HJ if possible
if ((buildPartSize < state.memForJoin) || (probePartSize < state.memForJoin)) {
int tabSize = -1;
- if (buildPartSize < probePartSize) {
+
+ if (isLeftOuter || buildPartSize < probePartSize) {
tabSize = ohhj.getBuildPartitionSizeInTup(pid);
+
if (tabSize == 0) {
throw new HyracksDataException(
"Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
}
- //Build Side is smaller
- applyInMemHashJoin(probeKeys, buildKeys, tabSize, probeRd, buildRd, hpcRep1, hpcRep0,
- buildSideReader, probeSideReader);
+ //Build Side is smaller
+ applyInMemHashJoin(buildKeys, probeKeys, tabSize, probeRd, buildRd, hpcRep0, hpcRep1,
+ buildSideReader, probeSideReader, false, pid);
- } else { //Role Reversal
+ }
+
+ else { //Role Reversal
tabSize = ohhj.getProbePartitionSizeInTup(pid);
if (tabSize == 0) {
throw new HyracksDataException(
"Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
}
//Probe Side is smaller
- applyInMemHashJoin(buildKeys, probeKeys, tabSize, buildRd, probeRd, hpcRep0, hpcRep1,
- probeSideReader, buildSideReader);
+
+ applyInMemHashJoin(probeKeys, buildKeys, tabSize, buildRd, probeRd, hpcRep1, hpcRep0,
+ probeSideReader, buildSideReader, true, pid);
}
}
//Apply (Recursive) HHJ
else {
OptimizedHybridHashJoin rHHj;
- if (buildPartSize < probePartSize) { //Build Side is smaller
+ if (isLeftOuter || buildPartSize < probePartSize) { //Build Side is smaller
int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
nPartitions);
+
rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, PROBE_REL, BUILD_REL,
probeKeys, buildKeys, comparators, probeRd, buildRd, probeHpc, buildHpc);
@@ -488,14 +500,14 @@
for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
+
if (rbrfw == null || rprfw == null) {
continue;
}
int buildSideInTups = rHHj.getBuildPartitionSizeInTup(rPid);
int probeSideInTups = rHHj.getProbePartitionSizeInTup(rPid);
- if (buildSideInTups < probeSideInTups) {
+ if (isLeftOuter || buildSideInTups < probeSideInTups) {
applyNestedLoopJoin(probeRd, buildRd, state.memForJoin, rbrfw, rprfw,
nljComparator0);
} else {
@@ -507,6 +519,7 @@
} else { //Role Reversal (Probe Side is smaller)
int n = getNumberOfPartitions(state.memForJoin, (int) probePartSize, fudgeFactor,
nPartitions);
+
rHHj = new OptimizedHybridHashJoin(ctx, state.memForJoin, n, BUILD_REL, PROBE_REL,
buildKeys, probeKeys, comparators, buildRd, probeRd, buildHpc, probeHpc);
@@ -545,7 +558,7 @@
for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
-
+
if (rbrfw == null || rprfw == null) {
continue;
}
@@ -569,14 +582,14 @@
private void applyInMemHashJoin(int[] bKeys, int[] pKeys, int tabSize, RecordDescriptor buildRDesc,
RecordDescriptor probeRDesc, ITuplePartitionComputer hpcRepLarger,
- ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader)
+ ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader, boolean reverse, int pid)
throws HyracksDataException {
ISerializableTable table = new SerializableHashTable(tabSize, ctx);
InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(
ctx.getFrameSize(), probeRDesc), hpcRepLarger, new FrameTupleAccessor(ctx.getFrameSize(),
buildRDesc), hpcRepSmaller, new FrameTuplePairComparator(pKeys, bKeys, comparators),
- isLeftOuter, nullWriters1, table);
+ isLeftOuter, nullWriters1, table, reverse);
bReader.open();
rPartbuff.clear();
@@ -604,7 +617,7 @@
throws HyracksDataException {
NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
- new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize);
+ new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize, false, null);
ByteBuffer cacheBuff = ctx.allocateFrame();
innerReader.open();
diff --git a/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
new file mode 100644
index 0000000..edca60a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.result;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ResultSetId rsId;
+
+ private final boolean ordered;
+
+ private final IResultSerializerFactory resultSerializerFactory;
+
+ public ResultWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, ResultSetId rsId, boolean ordered,
+ IResultSerializerFactory resultSerializerFactory) throws IOException {
+ super(spec, 1, 0);
+ this.rsId = rsId;
+ this.ordered = ordered;
+ this.resultSerializerFactory = resultSerializerFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
+
+ final ByteBuffer outputBuffer = ctx.allocateFrame();
+
+ final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
+ frameOutputStream.reset(outputBuffer, true);
+ PrintStream printStream = new PrintStream(frameOutputStream);
+
+ final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
+ printStream);
+
+ final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+ IFrameWriter datasetPartitionWriter;
+
+ @Override
+ public void open() throws HyracksDataException {
+ try {
+ datasetPartitionWriter = dpm.createDatasetPartitionWriter(ctx, rsId, ordered, partition,
+ nPartitions);
+ datasetPartitionWriter.open();
+ resultSerializer.init();
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ frameTupleAccessor.reset(buffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+ if (!frameOutputStream.appendTuple()) {
+ datasetPartitionWriter.nextFrame(outputBuffer);
+ frameOutputStream.reset(outputBuffer, true);
+
+ /* TODO(madhusudancs): This works under the assumption that no single serialized record is
+ * longer than the buffer size.
+ */
+ resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+ frameOutputStream.appendTuple();
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ datasetPartitionWriter.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (frameOutputStream.getTupleCount() > 0) {
+ datasetPartitionWriter.nextFrame(outputBuffer);
+ frameOutputStream.reset(outputBuffer, true);
+ }
+ datasetPartitionWriter.close();
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-dist/pom.xml b/fullstack/hyracks/hyracks-dist/pom.xml
new file mode 100755
index 0000000..58a4b1c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/pom.xml
@@ -0,0 +1,74 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>hyracks</artifactId>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>hyracks-dist</artifactId>
+ <name>hyracks-dist</name>
+
+ <properties>
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+ </properties>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-resources-plugin</artifactId>
+ <version>2.5</version>
+ <executions>
+ <execution>
+ <id>copy-scripts</id>
+ <!-- here the phase you need -->
+ <phase>package</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>target/appassembler/</outputDirectory>
+ <resources>
+ <resource>
+ <directory>src/main/resources</directory>
+ </resource>
+ </resources>
+ <directoryMode>0755</directoryMode>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <version>1.6</version>
+ <executions>
+ <execution>
+ <id>process-test-classes</id>
+ <phase>package</phase>
+ <configuration>
+ <target>
+ <chmod file="target/appassembler/bin/*)" perm="755" />
+ </target>
+ </configuration>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/getip.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
new file mode 100755
index 0000000..a691c0f
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/getip.sh
@@ -0,0 +1,25 @@
+#get the OS
+OS_NAME=`uname -a|awk '{print $1}'`
+LINUX_OS='Linux'
+
+if [ $OS_NAME = $LINUX_OS ];
+then
+ #Get IP Address
+ IPADDR=`/sbin/ifconfig eth0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ if [ "$IPADDR" = "" ]
+ then
+ IPADDR=`/sbin/ifconfig em1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ fi
+ if [ "$IPADDR" = "" ]
+ then
+ IPADDR=`/sbin/ifconfig lo | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ fi
+else
+ IPADDR=`/sbin/ifconfig en1 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ if [ "$IPADDR" = "" ]
+ then
+ IPADDR=`/sbin/ifconfig lo0 | grep "inet " | awk '{print $2}' | cut -f 2 -d ':'`
+ fi
+
+fi
+echo $IPADDR
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
new file mode 100755
index 0000000..629bd90
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+ ssh $i "cd ${PREGELIX_PATH}; bin/startnc.sh"
+done
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
new file mode 100755
index 0000000..a0c2063
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startCluster.sh
@@ -0,0 +1,3 @@
+bin/startcc.sh
+sleep 5
+bin/startAllNCs.sh
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
new file mode 100755
index 0000000..fe6cf27
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startDebugNc.sh
@@ -0,0 +1,50 @@
+hostname
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+. conf/debugnc.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR2
+mkdir $NCTMP_DIR2
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR2
+mkdir $NCLOGS_DIR2
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS2 | tr "," "\n")
+for io_dir in $io_dirs
+do
+ rm -rf $io_dir
+ mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+#Get OS
+IPADDR=`bin/getip.sh`
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+NODEID=${NODEID}2
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS2
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR2
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR -data-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS2}" &> $NCLOGS_DIR2/$NODEID.log &
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
new file mode 100755
index 0000000..484ecac
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startcc.sh
@@ -0,0 +1,31 @@
+#!/bin/bash
+hostname
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CCHOST=`bin/getip.sh`
+
+#Remove the temp dir
+rm -rf $CCTMP_DIR
+mkdir $CCTMP_DIR
+
+#Remove the logs dir
+rm -rf $CCLOGS_DIR
+mkdir $CCLOGS_DIR
+
+#Export JAVA_HOME and JAVA_OPTS
+export JAVA_HOME=$JAVA_HOME
+export JAVA_OPTS=$CCJAVA_OPTS
+
+#Launch hyracks cc script
+chmod -R 755 $HYRACKS_HOME
+if [ -f "conf/topology.xml" ]; then
+#Launch hyracks cc script with topology
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+else
+#Launch hyracks cc script without toplogy
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyrackscc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+fi
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
new file mode 100755
index 0000000..23a4c36
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/startnc.sh
@@ -0,0 +1,49 @@
+hostname
+
+MY_NAME=`hostname`
+#Get the IP address of the cc
+CCHOST_NAME=`cat conf/master`
+CURRENT_PATH=`pwd`
+CCHOST=`ssh ${CCHOST_NAME} "cd ${CURRENT_PATH}; bin/getip.sh"`
+
+#Import cluster properties
+. conf/cluster.properties
+
+#Clean up temp dir
+
+rm -rf $NCTMP_DIR
+mkdir $NCTMP_DIR
+
+#Clean up log dir
+rm -rf $NCLOGS_DIR
+mkdir $NCLOGS_DIR
+
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+ rm -rf $io_dir
+ mkdir $io_dir
+done
+
+#Set JAVA_HOME
+export JAVA_HOME=$JAVA_HOME
+
+IPADDR=`bin/getip.sh`
+#echo $IPADDR
+
+#Get node ID
+NODEID=`hostname | cut -d '.' -f 1`
+
+#Set JAVA_OPTS
+export JAVA_OPTS=$NCJAVA_OPTS
+
+cd $HYRACKS_HOME
+HYRACKS_HOME=`pwd`
+
+#Enter the temp dir
+cd $NCTMP_DIR
+
+#Launch hyracks nc
+$HYRACKS_HOME/hyracks-server/target/appassembler/bin/hyracksnc -cc-host $CCHOST -cc-port $CC_CLUSTERPORT -cluster-net-ip-address $IPADDR -data-ip-address $IPADDR -result-ip-address $IPADDR -node-id $NODEID -iodevices "${IO_DIRS}" &> $NCLOGS_DIR/$NODEID.log &
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
new file mode 100755
index 0000000..12367c1
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopAllNCs.sh
@@ -0,0 +1,6 @@
+PREGELIX_PATH=`pwd`
+
+for i in `cat conf/slaves`
+do
+ ssh $i "cd ${PREGELIX_PATH}; bin/stopnc.sh"
+done
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
new file mode 100755
index 0000000..4889934
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopCluster.sh
@@ -0,0 +1,3 @@
+bin/stopAllNCs.sh
+sleep 2
+bin/stopcc.sh
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
new file mode 100755
index 0000000..c2f525a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopcc.sh
@@ -0,0 +1,10 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep hyracks|awk '{print $2}'`
+echo $PID
+kill -9 $PID
+
+#Clean up CC temp dir
+rm -rf $CCTMP_DIR/*
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
new file mode 100755
index 0000000..35c4794
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/bin/stopnc.sh
@@ -0,0 +1,27 @@
+hostname
+. conf/cluster.properties
+
+#Kill process
+PID=`ps -ef|grep ${USER}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+
+if [ "$PID" == "" ]; then
+ PID=`ps -ef|grep ${USER}|grep java|grep 'hyracks'|awk '{print $2}'`
+fi
+
+if [ "$PID" == "" ]; then
+ USERID=`id | sed 's/^uid=//;s/(.*$//'`
+ PID=`ps -ef|grep ${USERID}|grep java|grep 'Dapp.name=hyracksnc'|awk '{print $2}'`
+fi
+
+echo $PID
+kill -9 $PID
+
+#Clean up I/O working dir
+io_dirs=$(echo $IO_DIRS | tr "," "\n")
+for io_dir in $io_dirs
+do
+ rm -rf $io_dir/*
+done
+
+#Clean up NC temp dir
+rm -rf $NCTMP_DIR/*
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
new file mode 100755
index 0000000..3b382f7
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/cluster.properties
@@ -0,0 +1,37 @@
+#The CC port for Hyracks clients
+CC_CLIENTPORT=3099
+
+#The CC port for Hyracks cluster management
+CC_CLUSTERPORT=1099
+
+#The directory of hyracks binaries
+HYRACKS_HOME=../../../
+
+#The tmp directory for cc to install jars
+CCTMP_DIR=/tmp/t1
+
+#The tmp directory for nc to install jars
+NCTMP_DIR=/tmp/t2
+
+#The directory to put cc logs
+CCLOGS_DIR=$CCTMP_DIR/logs
+
+#The directory to put nc logs
+NCLOGS_DIR=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS="/tmp/t3,/tmp/t4"
+
+#The JAVA_HOME
+JAVA_HOME=$JAVA_HOME
+
+#The frame size of the internal dataflow engine
+FRAME_SIZE=65536
+
+#CC JAVA_OPTS
+CCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7001,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+# Yourkit option: -agentpath:/grid/0/dev/vborkar/tools/yjp-10.0.4/bin/linux-x86-64/libyjpagent.so=port=20001"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=7002,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
new file mode 100755
index 0000000..27afa26
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/debugnc.properties
@@ -0,0 +1,12 @@
+#The tmp directory for nc to install jars
+NCTMP_DIR2=/tmp/t-1
+
+#The directory to put nc logs
+NCLOGS_DIR2=$NCTMP_DIR/logs
+
+#Comma separated I/O directories for the spilling of external sort
+IO_DIRS2="/tmp/t-2,/tmp/t-3"
+
+#NC JAVA_OPTS
+NCJAVA_OPTS2="-Xdebug -Xrunjdwp:transport=dt_socket,address=7003,server=y,suspend=n -Xmx1g -Djava.util.logging.config.file=logging.properties"
+
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/conf/master b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/master
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/master
@@ -0,0 +1 @@
+localhost
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/conf/slaves b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/slaves
new file mode 100755
index 0000000..2fbb50c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/slaves
@@ -0,0 +1 @@
+localhost
diff --git a/fullstack/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml
new file mode 100755
index 0000000..4710706
--- /dev/null
+++ b/fullstack/hyracks/hyracks-dist/src/main/resources/conf/topology-template.xml
@@ -0,0 +1,7 @@
+<cluster-topology>
+ <network-switch name="Global">
+ <network-switch name="local">
+ <terminal name="127.0.0.1"/>
+ </network-switch>
+ </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-documentation/pom.xml b/fullstack/hyracks/hyracks-documentation/pom.xml
index ed24adb..7aedd57 100644
--- a/fullstack/hyracks/hyracks-documentation/pom.xml
+++ b/fullstack/hyracks/hyracks-documentation/pom.xml
@@ -1,8 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-documentation</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hyracks-documentation</name>
<parent>
diff --git a/fullstack/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml b/fullstack/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
index 6350054..792cedd 100644
--- a/fullstack/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/btree-example/btreeapp/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.btree</groupId>
<artifactId>btreeapp</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>btreeapp</name>
<parent>
diff --git a/fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml b/fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 82919ca..f941a5b 100644
--- a/fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -37,13 +37,15 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml b/fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
index 8e90606..eb651ce 100644
--- a/fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -41,8 +41,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
index 70c663d..4cf9745 100644
--- a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatapp/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.compat</groupId>
<artifactId>hadoopcompatapp</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hadoopcompatapp</name>
<parent>
@@ -147,8 +146,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
diff --git a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
index 2f0b00f..f52536c 100644
--- a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompatclient/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.compat</groupId>
<artifactId>hadoopcompatclient</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hadoopcompatclient</name>
<parent>
@@ -33,13 +32,14 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
index 1e029be..c397a72 100644
--- a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/hadoopcompathelper/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.compat</groupId>
<artifactId>hadoopcompathelper</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>hadoopcompathelper</name>
<parent>
@@ -32,8 +31,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/pom.xml b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
index a2cf5ae..16787ca 100644
--- a/fullstack/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/hadoop-compat-example/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>hadoop-compat-example</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hadoop-compat-example</name>
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index e31af75..5e7b5c9 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -16,8 +16,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
@@ -31,6 +32,13 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-dataflow-std</artifactId>
<version>0.2.3-SNAPSHOT</version>
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 8482083..33ddca2 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexScanOperatorTest extends AbstractIntegrationTest {
static {
@@ -114,7 +116,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -143,8 +145,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -179,13 +182,16 @@
int[] highKeyFields = null; // + infinity
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index 82fecbe..acd3027 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -114,7 +116,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -143,8 +145,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -184,13 +187,16 @@
int[] highKeyFields = { 1 };
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index e63ce11..ca03b16 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -42,7 +43,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -57,6 +58,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
static {
@@ -82,7 +84,6 @@
private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
@@ -107,7 +108,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -136,8 +137,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -153,12 +155,15 @@
JobSpecification spec = new JobSpecification();
TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, primaryBtreeSplitProvider,
- primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 3c87ae3..34a1cd3 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,9 +45,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -63,6 +64,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreeSecondaryIndexInsertOperatorTest extends AbstractIntegrationTest {
static {
@@ -339,9 +341,11 @@
primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 1304f12..c9ee118 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -137,7 +139,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -166,8 +168,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -180,14 +183,14 @@
public void createSecondaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
- TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
spec.addRoot(secondaryCreateOp);
runTest(spec);
}
-
+
public void loadSecondaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -212,8 +215,9 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
@@ -225,8 +229,9 @@
// load secondary index
int[] fieldPermutation = { 3, 0 };
TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
@@ -268,8 +273,8 @@
// search secondary index
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true,
+ true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
int[] primaryLowKeyFields = { 1 }; // second field from the tuples
@@ -279,13 +284,16 @@
// search primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, primaryLowKeyFields,
- primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 023bdd9..8893567 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -14,29 +14,39 @@
*/
package edu.uci.ics.hyracks.tests.integration;
+import java.io.BufferedReader;
import java.io.File;
+import java.io.FileReader;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.commons.io.FileUtils;
import org.junit.AfterClass;
+import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
public abstract class AbstractIntegrationTest {
private static final Logger LOGGER = Logger.getLogger(AbstractIntegrationTest.class.getName());
@@ -80,6 +90,7 @@
ncConfig1.ccPort = 39001;
ncConfig1.clusterNetIPAddress = "127.0.0.1";
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -89,6 +100,7 @@
ncConfig2.ccPort = 39001;
ncConfig2.clusterNetIPAddress = "127.0.0.1";
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
@@ -107,7 +119,7 @@
cc.stop();
}
- protected void runTest(JobSpecification spec) throws Exception {
+ protected JobId executeTest(JobSpecification spec) throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
@@ -115,25 +127,72 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
- hcc.waitForCompletion(jobId);
- dumpOutputFiles();
+ return jobId;
}
- private void dumpOutputFiles() {
- if (LOGGER.isLoggable(Level.INFO)) {
- for (File f : outputFiles) {
- if (f.exists() && f.isFile()) {
- try {
- LOGGER.info("Reading file: " + f.getAbsolutePath() + " in test: " + getClass().getName());
- String data = FileUtils.readFileToString(f);
- LOGGER.info(data);
- } catch (IOException e) {
- LOGGER.info("Error reading file: " + f.getAbsolutePath());
- LOGGER.info(e.getMessage());
- }
+ protected void runTest(JobSpecification spec) throws Exception {
+ JobId jobId = executeTest(spec);
+ hcc.waitForCompletion(jobId);
+ }
+
+ protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
+ int nReaders = 1;
+ ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+ resultBuffer.clear();
+
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+ IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+ IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
+
+ List<String> resultRecords = new ArrayList<String>();
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+ int readSize = reader.read(resultBuffer);
+
+ while (readSize > 0) {
+
+ try {
+ frameTupleAccessor.reset(resultBuffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+ int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+ bbis.setByteBuffer(resultBuffer, start);
+ byte[] recordBytes = new byte[length];
+ bbis.read(recordBytes, 0, length);
+ resultRecords.add(new String(recordBytes, 0, length));
}
+ } finally {
+ bbis.close();
}
+
+ resultBuffer.clear();
+ readSize = reader.read(resultBuffer);
}
+ return resultRecords;
+ }
+
+ protected boolean runTestAndCompareResults(JobSpecification spec, String[] expectedFileNames) throws Exception {
+ JobId jobId = executeTest(spec);
+
+ List<String> results;
+ for (int i = 0; i < expectedFileNames.length; i++) {
+ results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+ BufferedReader expectedFile = new BufferedReader(new FileReader(expectedFileNames[i]));
+
+ String expectedLine, actualLine;
+ int j = 0;
+ while ((expectedLine = expectedFile.readLine()) != null) {
+ actualLine = results.get(j).trim();
+ Assert.assertEquals(expectedLine, actualLine);
+ j++;
+ }
+ Assert.assertEquals(j, results.size());
+ expectedFile.close();
+ }
+
+ hcc.waitForCompletion(jobId);
+ return true;
}
protected File createTempFile() throws IOException {
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index e0b8c73..24d0ef4 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,6 +16,7 @@
import java.io.File;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
@@ -23,6 +24,7 @@
import java.util.logging.Logger;
import org.apache.commons.io.FileUtils;
+import org.json.JSONArray;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -30,13 +32,20 @@
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
public abstract class AbstractMultiNCIntegrationTest {
@@ -84,6 +93,7 @@
ncConfig.ccPort = 39001;
ncConfig.clusterNetIPAddress = "127.0.0.1";
ncConfig.dataIPAddress = "127.0.0.1";
+ ncConfig.datasetIPAddress = "127.0.0.1";
ncConfig.nodeId = ASTERIX_IDS[i];
asterixNCs[i] = new NodeControllerService(ncConfig);
asterixNCs[i].start();
@@ -112,6 +122,46 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
+
+ int nReaders = 1;
+
+ ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+ resultBuffer.clear();
+
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+ IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+ IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
+
+ JSONArray resultRecords = new JSONArray();
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+ int readSize = reader.read(resultBuffer);
+
+ while (readSize > 0) {
+
+ try {
+ frameTupleAccessor.reset(resultBuffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+ int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+ bbis.setByteBuffer(resultBuffer, start);
+ byte[] recordBytes = new byte[length];
+ bbis.read(recordBytes, 0, length);
+ resultRecords.put(new String(recordBytes, 0, length));
+ }
+ } finally {
+ try {
+ bbis.close();
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ resultBuffer.clear();
+ readSize = reader.read(resultBuffer);
+ }
+
hcc.waitForCompletion(jobId);
dumpOutputFiles();
}
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 93e1e9b..06751e3 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -25,8 +25,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -49,7 +49,6 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
@@ -62,60 +61,42 @@
import edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
/**
*
*/
public class AggregationTest extends AbstractIntegrationTest {
- final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
- "data/tpch0.001/lineitem.tbl"))) });
+ final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+ new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
- final RecordDescriptor desc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE, }, '|');
+ final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+ UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+ FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, }, '|');
- private AbstractSingleActivityOperatorDescriptor getPrinter(
- IOperatorDescriptorRegistry spec, String prefix) throws IOException {
+ private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+ throws IOException {
- AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
- spec, new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC1_ID, createTempFile()
- .getAbsolutePath()),
- new FileSplit(NC2_ID, createTempFile()
- .getAbsolutePath()) }), "\t");
+ ResultSetId rsId = new ResultSetId(1);
+ AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
return printer;
}
@@ -124,54 +105,38 @@
public void singleKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true),
- new FloatSumFieldAggregatorFactory(5, true) }),
- outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+ new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -184,49 +149,34 @@
public void singleKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE});
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true),
- new FloatSumFieldAggregatorFactory(5, true)}),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+ new FloatSumFieldAggregatorFactory(5, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -239,64 +189,43 @@
public void singleKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE});
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(3, false),
- new FloatSumFieldAggregatorFactory(5, false)}),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ new FloatSumFieldAggregatorFactory(5, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(2, false),
- new FloatSumFieldAggregatorFactory(3, false)}),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ new FloatSumFieldAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -309,54 +238,38 @@
public void singleKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -369,49 +282,34 @@
public void singleKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -424,64 +322,43 @@
public void singleKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(2, false),
- new AvgFieldMergeAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -494,52 +371,38 @@
public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -552,47 +415,34 @@
public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -605,63 +455,42 @@
public void singleKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(2, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -674,58 +503,39 @@
public void multiKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -738,51 +548,35 @@
public void multiKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -795,69 +589,43 @@
public void multiKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+ new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -870,60 +638,40 @@
public void multiKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -936,53 +684,36 @@
public void multiKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -995,72 +726,46 @@
public void multiKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
new IntSumFieldAggregatorFactory(3, false),
- new AvgFieldMergeAggregatorFactory(4, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1073,58 +778,39 @@
public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1137,51 +823,35 @@
public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringPreClusterGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1194,71 +864,44 @@
public void multiKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new MinMaxStringFieldAggregatorFactory(3, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(2, false),
+ new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index aea6126..5008991 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -25,6 +25,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,13 +45,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class CountOfCountsTest extends AbstractIntegrationTest {
@Test
@@ -76,12 +78,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -91,13 +91,15 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -148,12 +150,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -163,13 +163,16 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -220,12 +223,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -235,13 +236,16 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 0d5a627..93ed2c7 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -26,8 +26,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -52,12 +52,13 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class LocalityAwareConnectorTest extends AbstractMultiNCIntegrationTest {
@@ -200,13 +201,13 @@
runTest(spec);
}
- private AbstractSingleActivityOperatorDescriptor getPrinter(IOperatorDescriptorRegistry spec, String prefix)
+ private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
throws IOException {
- AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
- new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit("asterix-005", createTempFile().getAbsolutePath()),
- new FileSplit("asterix-006", createTempFile().getAbsolutePath()) }), "\t");
+ ResultSetId rsId = new ResultSetId(1);
+ AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
return printer;
}
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 1ee4400..ec9be32 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class OptimizedSortMergeTest extends AbstractIntegrationTest {
@@ -75,9 +77,11 @@
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -128,9 +132,11 @@
LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, filter, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 9355110..961f780 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -42,7 +43,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class ScanPrintTest extends AbstractIntegrationTest {
@Test
@@ -63,10 +65,11 @@
desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
- new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
@@ -98,9 +101,11 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -135,9 +140,11 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 2c3fddf..0da93f2 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class SortMergeTest extends AbstractIntegrationTest {
@Test
@@ -73,9 +75,11 @@
ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -118,9 +122,11 @@
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
index 2b32142..6040748 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
@@ -26,6 +26,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -35,8 +36,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class SplitOperatorTest extends AbstractIntegrationTest {
@@ -50,6 +52,8 @@
Assert.assertEquals(lineA, lineB);
}
Assert.assertNull(fileB.readLine());
+ fileA.close();
+ fileB.close();
}
@Test
@@ -83,8 +87,11 @@
IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
for (int i = 0; i < outputArity; i++) {
- outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { new FileSplit(NC1_ID,
- outputFile[i].getAbsolutePath()) });
+ ResultSetId rsId = new ResultSetId(i);
+ spec.addResultSetId(rsId);
+
+ outputOp[i] = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i], locations);
}
@@ -96,10 +103,10 @@
for (int i = 0; i < outputArity; i++) {
spec.addRoot(outputOp[i]);
}
- runTest(spec);
-
+ String[] expectedResultsFileNames = new String[outputArity];
for (int i = 0; i < outputArity; i++) {
- compareFiles(inputFileName, outputFile[i].getAbsolutePath());
+ expectedResultsFileNames[i] = inputFileName;
}
+ runTestAndCompareResults(spec, expectedResultsFileNames);
}
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 61d4696..b5eb850 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -14,9 +14,7 @@
*/
package edu.uci.ics.hyracks.tests.integration;
-import java.io.DataOutput;
import java.io.File;
-import java.io.IOException;
import org.junit.Test;
@@ -25,11 +23,10 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,35 +44,15 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
- private static class NoopNullWriterFactory implements INullWriterFactory {
-
- private static final long serialVersionUID = 1L;
- public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
-
- private NoopNullWriterFactory() {
- }
-
- @Override
- public INullWriter createNullWriter() {
- return new INullWriter() {
- @Override
- public void writeNull(DataOutput out) throws HyracksDataException {
- try {
- out.writeShort(0);
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
- };
- }
- }
/*
* TPCH Customer table: CREATE TABLE CUSTOMER ( C_CUSTKEY INTEGER NOT NULL,
@@ -148,9 +125,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -229,9 +208,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -310,9 +291,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -392,9 +375,11 @@
custOrderJoinDesc, true, nullWriterFactories, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -478,9 +463,11 @@
custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -564,9 +551,11 @@
custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -643,9 +632,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -732,9 +723,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -821,9 +814,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -906,9 +901,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -997,9 +994,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 1e60372..99f2d18 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -25,10 +25,12 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -43,8 +45,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class TPCHCustomerOrderNestedLoopJoinTest extends AbstractIntegrationTest {
private static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -165,12 +169,15 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 4, false,
+ null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -239,12 +246,15 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, false,
+ null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -313,12 +323,97 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 6, false,
+ null);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+ IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
+ spec.connect(ordJoinConn, ordScanner, 0, join, 0);
+
+ IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+ spec.connect(custJoinConn, custScanner, 0, join, 1);
+
+ IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ spec.connect(joinPrinterConn, join, 0, printer, 0);
+
+ spec.addRoot(printer);
+ runTest(spec);
+ }
+
+ @Test
+ public void customerOrderCIDOuterJoinMulti() throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ FileSplit[] custSplits = new FileSplit[] {
+ new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))),
+ new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) };
+ IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits);
+ RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+ FileSplit[] ordersSplits = new FileSplit[] {
+ new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))),
+ new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) };
+ IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
+
+ RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
+
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
+
+ FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID);
+
+ INullWriterFactory[] nullWriterFactories = new INullWriterFactory[ordersDesc.getFieldCount()];
+ for (int j = 0; j < nullWriterFactories.length; j++) {
+ nullWriterFactories[j] = NoopNullWriterFactory.INSTANCE;
+ }
+
+ NestedLoopJoinOperatorDescriptor join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 1, 0), custOrderJoinDesc, 5, true,
+ nullWriterFactories);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
index 37b55b8..5b323e6 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
@@ -22,6 +22,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -33,8 +34,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.union.UnionAllOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class UnionTest extends AbstractIntegrationTest {
@Test
@@ -65,10 +67,11 @@
UnionAllOperatorDescriptor unionAll = new UnionAllOperatorDescriptor(spec, 2, desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, unionAll, NC2_ID, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
- new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), csvScanner01, 0, unionAll, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
index 836e72e..600b54b 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -8,6 +8,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
@@ -58,9 +60,11 @@
tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
index 2206a26..b5a4df8 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
@@ -8,6 +8,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
@@ -58,9 +60,11 @@
tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
index d8fd48e..d1071a3 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -48,8 +49,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -74,6 +75,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class WordInvertedIndexTest extends AbstractIntegrationTest {
static {
@@ -87,9 +89,12 @@
private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
private final static String sep = System.getProperty("file.separator");
private final static String dateString = simpleDateFormat.format(new Date());
- private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
- private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
- private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists" + dateString;
+ private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree"
+ + dateString;
+ private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree"
+ + dateString;
+ private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists"
+ + dateString;
private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
@@ -155,7 +160,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
@Test
public void testConjunctiveSearcher() throws Exception {
IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
@@ -180,8 +185,9 @@
private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
int[] fieldPermutation = { 0, 1 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, btreeDataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
return primaryBtreeBulkLoad;
}
@@ -206,8 +212,9 @@
int[] lowKeyFields = null; // - infinity
int[] highKeyFields = null; // + infinity
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
return primaryBtreeSearchOp;
}
@@ -227,9 +234,12 @@
JobSpecification spec = new JobSpecification();
IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
spec.addRoot(printer);
@@ -265,15 +275,14 @@
public void createInvertedIndex() throws Exception {
JobSpecification spec = new JobSpecification();
InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
- storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
- indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- invListsComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory,
- NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
+ tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+ tokenizerFactory, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexCreateOp, NC1_ID);
spec.addRoot(invIndexCreateOp);
runTest(spec);
}
-
+
public void loadInvertedIndex() throws Exception {
JobSpecification spec = new JobSpecification();
IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
@@ -325,20 +334,23 @@
JobSpecification spec = new JobSpecification();
IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
spec.addRoot(printer);
runTest(spec);
}
-
+
@AfterClass
public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- File btree = new File(btreeFileName);
- File invLists = new File(invListsFileName);
+ File primary = new File(primaryFileName);
+ File btree = new File(btreeFileName);
+ File invLists = new File(invListsFileName);
primary.deleteOnExit();
btree.deleteOnExit();
invLists.deleteOnExit();
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 6625148..92b6e14 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -64,6 +65,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -190,9 +192,11 @@
dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryRTreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index ef2950e..2a00394 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -45,6 +46,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -64,6 +66,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
static {
@@ -175,9 +178,11 @@
primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 030afcf..1d86037 100644
--- a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -67,6 +68,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -297,9 +299,11 @@
secondaryTypeTraits, secondaryComparatorFactories, keyFields, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryRTreeSearchOp, 0);
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
new file mode 100644
index 0000000..d119509
--- /dev/null
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/NoopNullWriterFactory.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
+import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoopNullWriterFactory implements INullWriterFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final NoopNullWriterFactory INSTANCE = new NoopNullWriterFactory();
+
+ private NoopNullWriterFactory() {
+ }
+
+ @Override
+ public INullWriter createNullWriter() {
+ return new INullWriter() {
+ @Override
+ public void writeNull(DataOutput out) throws HyracksDataException {
+ try {
+ out.writeShort(0);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..19c4475
--- /dev/null
+++ b/fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.tests.util;
+
+import java.io.DataInputStream;
+import java.io.PrintStream;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultSerializerFactoryProvider implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+ private ResultSerializerFactoryProvider() {
+ }
+
+ public IResultSerializerFactory getResultSerializerFactoryProvider() {
+ return new IResultSerializerFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IResultSerializer createResultSerializer(final RecordDescriptor recordDesc,
+ final PrintStream printStream) {
+ return new IResultSerializer() {
+ private static final long serialVersionUID = 1L;
+
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+
+ @Override
+ public void init() throws HyracksDataException {
+
+ }
+
+ @Override
+ public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+ int start = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength();
+
+ bbis.setByteBuffer(tAccess.getBuffer(), start);
+
+ Object[] record = new Object[recordDesc.getFieldCount()];
+ for (int i = 0; i < record.length; ++i) {
+ Object instance = recordDesc.getFields()[i].deserialize(di);
+ if (i == 0) {
+ printStream.print(String.valueOf(instance));
+ } else {
+ printStream.print(", " + String.valueOf(instance));
+ }
+ }
+ printStream.println();
+ return true;
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-examples/pom.xml b/fullstack/hyracks/hyracks-examples/pom.xml
index 8ce8108..551e2be 100644
--- a/fullstack/hyracks/hyracks-examples/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/pom.xml
@@ -1,8 +1,6 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hyracks-examples</name>
diff --git a/fullstack/hyracks/hyracks-examples/text-example/pom.xml b/fullstack/hyracks/hyracks-examples/text-example/pom.xml
index ba8649e..0476bb3 100644
--- a/fullstack/hyracks/hyracks-examples/text-example/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/text-example/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>text-example</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<packaging>pom</packaging>
<name>text-example</name>
diff --git a/fullstack/hyracks/hyracks-examples/text-example/textapp/pom.xml b/fullstack/hyracks/hyracks-examples/text-example/textapp/pom.xml
index 8ac69d8..945df0b 100644
--- a/fullstack/hyracks/hyracks-examples/text-example/textapp/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/text-example/textapp/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.text</groupId>
<artifactId>textapp</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>textapp</name>
<parent>
@@ -142,8 +141,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
diff --git a/fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml b/fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
index d593ef9..4aace73 100644
--- a/fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples.text</groupId>
<artifactId>textclient</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<name>textclient</name>
<parent>
@@ -33,13 +32,15 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<id>textclient</id>
diff --git a/fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml b/fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
index 8e32c8c..bcb280c 100644
--- a/fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
@@ -36,8 +36,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-examples/tpch-example/pom.xml b/fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
index b237c9b..9951792 100644
--- a/fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -2,7 +2,6 @@
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks.examples</groupId>
<artifactId>tpch-example</artifactId>
- <version>0.2.3-SNAPSHOT</version>
<packaging>pom</packaging>
<name>tpch-example</name>
diff --git a/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 3170306..4e0d9f0 100644
--- a/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -29,13 +29,15 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index 01ccdef..0ad0ff0 100644
--- a/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -199,7 +199,7 @@
if ("nestedloop".equalsIgnoreCase(algo)) {
join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize);
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc, memSize, false, null);
} else if ("gracehash".equalsIgnoreCase(algo)) {
join = new GraceHashJoinOperatorDescriptor(
diff --git a/fullstack/hyracks/hyracks-hadoop-compat/pom.xml b/fullstack/hyracks/hyracks-hadoop-compat/pom.xml
index 849efe3..87aaaa7 100644
--- a/fullstack/hyracks/hyracks-hadoop-compat/pom.xml
+++ b/fullstack/hyracks/hyracks-hadoop-compat/pom.xml
@@ -18,13 +18,15 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml
new file mode 100644
index 0000000..9092655
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-hdfs-0.20.2</artifactId>
+ <name>hyracks-hdfs-0.20.2</name>
+ <parent>
+ <artifactId>hyracks-hdfs</artifactId>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.7.2</version>
+ <configuration>
+ <forkMode>pertest</forkMode>
+ <includes>
+ <include>**/*TestSuite.java</include>
+ <include>**/*Test.java</include>
+ </includes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+
+ <profiles>
+ <profile>
+ <activation>
+ <activeByDefault>true</activeByDefault>
+ </activation>
+ <id>hadoop-0.20.2</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-core</artifactId>
+ <version>0.20.2</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-test</artifactId>
+ <version>0.20.2</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>1.0.4</value>
+ </property>
+ </activation>
+ <id>hadoop-1.0.4</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-core</artifactId>
+ <version>1.0.4</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>1.0.4</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-test</artifactId>
+ <version>1.0.4</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ </profiles>
+
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
new file mode 100644
index 0000000..16ce76b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
@@ -0,0 +1,39 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The wrapper to generate TaskTattemptContext
+ */
+public class ContextFactory {
+
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ public TaskAttemptContext createContext(Configuration conf, TaskAttemptID tid) throws HyracksDataException {
+ try {
+ return new Mapper().new Context(conf, tid, null, null, null, null, null);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public TaskAttemptContext createContext(Configuration conf, int partition) throws HyracksDataException {
+ try {
+ TaskAttemptID tid = new TaskAttemptID("", 0, true, partition, 0);
+ return new TaskAttemptContext(conf, tid);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public JobContext createJobContext(Configuration conf) {
+ return new JobContext(conf, new JobID("0", 0));
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
new file mode 100644
index 0000000..9133d35
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.20.2/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class MiniDFSClusterFactory {
+
+ public MiniDFSCluster getMiniDFSCluster(Configuration conf, int numberOfNC) throws HyracksDataException {
+ try {
+ return new MiniDFSCluster(conf, numberOfNC, true, null);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml
new file mode 100644
index 0000000..8b7ecf0
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/pom.xml
@@ -0,0 +1,219 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-hdfs-0.23.1</artifactId>
+ <name>hyracks-hdfs-0.23.1</name>
+ <parent>
+ <artifactId>hyracks-hdfs</artifactId>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.7.2</version>
+ <configuration>
+ <forkMode>pertest</forkMode>
+ <includes>
+ <include>**/*TestSuite.java</include>
+ <include>**/*Test.java</include>
+ </includes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+
+ <profiles>
+ <profile>
+ <activation>
+ <activeByDefault>true</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>0.23.1</value>
+ </property>
+ </activation>
+ <id>hadoop-0.23.1</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>0.23.1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <version>0.23.1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <version>0.23.1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>0.23.1</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <id>hadoop-0.23.6</id>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>0.23.6</value>
+ </property>
+ </activation>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>0.23.6</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <version>0.23.6</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <version>0.23.6</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>0.23.6</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>cdh-4.2</value>
+ </property>
+ </activation>
+ <id>cdh-4.2</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>2.0.0-cdh4.2.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <version>2.0.0-cdh4.2.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <version>2.0.0-cdh4.2.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>2.0.0-cdh4.2.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>cdh-4.1</value>
+ </property>
+ </activation>
+ <id>cdh-4.1</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>2.0.0-cdh4.1.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <version>2.0.0-cdh4.1.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <version>2.0.0-cdh4.1.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-minicluster</artifactId>
+ <version>2.0.0-cdh4.1.0</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ </profiles>
+
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+
+ <repositories>
+ <repository>
+ <id>cloudera</id>
+ <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+ </repository>
+ </repositories>
+</project>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
new file mode 100644
index 0000000..ddcce64
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/ContextFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.JobContextImpl;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The wrapper to generate TaskTattemptContext
+ */
+public class ContextFactory {
+
+ public TaskAttemptContext createContext(Configuration conf, TaskAttemptID tid) throws HyracksDataException {
+ try {
+ return new TaskAttemptContextImpl(conf, tid);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public TaskAttemptContext createContext(Configuration conf, int partition) throws HyracksDataException {
+ try {
+ TaskAttemptID tid = new TaskAttemptID("", 0, TaskType.REDUCE, partition, 0);
+ return new TaskAttemptContextImpl(conf, tid);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public JobContext createJobContext(Configuration conf) {
+ return new JobContextImpl(conf, new JobID("0", 0));
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
new file mode 100644
index 0000000..ded75f1
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-0.23.1/src/main/java/edu/uci/ics/hyracks/hdfs/MiniDFSClusterFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.hyracks.hdfs;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class MiniDFSClusterFactory {
+
+ public MiniDFSCluster getMiniDFSCluster(Configuration conf, int numberOfNC) throws HyracksDataException {
+ try {
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(conf);
+ builder.numDataNodes(numberOfNC);
+ MiniDFSCluster dfsCluster = builder.build();
+ return dfsCluster;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
new file mode 100644
index 0000000..a28c698a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/pom.xml
@@ -0,0 +1,217 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <name>hyracks-hdfs-core</name>
+ <parent>
+ <artifactId>hyracks-hdfs</artifactId>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.7.2</version>
+ <configuration>
+ <forkMode>pertest</forkMode>
+ <includes>
+ <include>**/*TestSuite.java</include>
+ <include>**/*Test.java</include>
+ </includes>
+ </configuration>
+ </plugin>
+ <plugin>
+ <artifactId>maven-clean-plugin</artifactId>
+ <version>2.5</version>
+ <configuration>
+ <filesets>
+ <fileset>
+ <directory>.</directory>
+ <includes>
+ <include>edu*</include>
+ <include>actual*</include>
+ <include>build*</include>
+ <include>expect*</include>
+ <include>ClusterController*</include>
+ <include>edu.uci.*</include>
+ </includes>
+ </fileset>
+ </filesets>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+
+ <profiles>
+ <profile>
+ <activation>
+ <activeByDefault>true</activeByDefault>
+ </activation>
+ <id>hadoop-0.20.2</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.20.2</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>1.0.4</value>
+ </property>
+ </activation>
+ <id>hadoop-1.0.4</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.20.2</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>0.23.1</value>
+ </property>
+ </activation>
+ <id>hadoop-0.23.1</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.23.1</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>0.23.6</value>
+ </property>
+ </activation>
+ <id>hadoop-0.23.6</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.23.1</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>cdh-4.1</value>
+ </property>
+ </activation>
+ <id>cdh-4.1</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.23.1</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ <property>
+ <name>hadoop</name>
+ <value>cdh-4.2</value>
+ </property>
+ </activation>
+ <id>cdh-4.2</id>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-0.23.1</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+ </profile>
+ </profiles>
+
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>3.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-std</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-cc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>1.3.2</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
new file mode 100644
index 0000000..5d35ec5
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParser.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ *
+ * @param <K>
+ * the key type
+ * @param <V>
+ * the value type
+ */
+public interface IKeyValueParser<K, V> {
+
+ /**
+ * Initialize the key value parser.
+ *
+ * @param writer
+ * The hyracks writer for outputting data.
+ * @throws HyracksDataException
+ */
+ public void open(IFrameWriter writer) throws HyracksDataException;
+
+ /**
+ * Parse a key-value pair returned by HDFS record reader to a tuple.
+ * when the parsers' internal buffer is full, it can flush the buffer to the writer
+ *
+ * @param key
+ * The key returned from Hadoop's InputReader.
+ * @param value
+ * The value returned from Hadoop's InputReader.
+ * @param writer
+ * The hyracks writer for outputting data.
+ * @throws HyracksDataException
+ */
+ public void parse(K key, V value, IFrameWriter writer) throws HyracksDataException;
+
+ /**
+ * Flush the residual tuples in the internal buffer to the writer.
+ * This method is called in the close() of HDFSReadOperatorDescriptor.
+ *
+ * @param writer
+ * The hyracks writer for outputting data.
+ * @throws HyracksDataException
+ */
+ public void close(IFrameWriter writer) throws HyracksDataException;
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
new file mode 100644
index 0000000..7d6f868
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/IKeyValueParserFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSReadOperatorDescriptor.
+ *
+ * @param <K>
+ * the key type
+ * @param <V>
+ * the value type
+ */
+public interface IKeyValueParserFactory<K, V> extends Serializable {
+
+ /**
+ * This method creates a key-value parser.
+ *
+ * @param ctx
+ * the IHyracksTaskContext
+ * @return a key-value parser instance.
+ */
+ public IKeyValueParser<K, V> createKeyValueParser(IHyracksTaskContext ctx) throws HyracksDataException;
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
new file mode 100644
index 0000000..c51c1dd
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollection.java
@@ -0,0 +1,11 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+@SuppressWarnings("deprecation")
+public interface INcCollection {
+
+ public String findNearestAvailableSlot(InputSplit split);
+
+ public int numAvailableSlots();
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
new file mode 100644
index 0000000..ef3ff23
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/INcCollectionBuilder.java
@@ -0,0 +1,18 @@
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+
+/**
+ * NC collections
+ *
+ * @author yingyib
+ */
+public interface INcCollectionBuilder {
+
+ public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+ Map<String, List<String>> ipToNcMapping, Map<String, Integer> ncNameToIndex, String[] NCs, int[] workloads,
+ int slotLimit);
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
new file mode 100644
index 0000000..8e85627
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriter.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriter {
+
+ /**
+ * Initialize the the tuple writer.
+ *
+ * @param output
+ * The channel for output data.
+ * @throws HyracksDataException
+ */
+ public void open(DataOutput output) throws HyracksDataException;
+
+ /**
+ * Write the tuple to the DataOutput.
+ *
+ * @param output
+ * the DataOutput channel
+ * @param tuple
+ * the tuple to write
+ * @throws HyracksDataException
+ */
+ public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException;
+
+ /**
+ * Close the writer.
+ *
+ * @param output
+ * The channel for output data.
+ * @throws HyracksDataException
+ */
+ public void close(DataOutput output) throws HyracksDataException;
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
new file mode 100644
index 0000000..9a025c2
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/api/ITupleWriterFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.api;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Users need to implement this interface to use the HDFSWriteOperatorDescriptor.
+ */
+public interface ITupleWriterFactory extends Serializable {
+
+ /**
+ * @param ctx
+ * the IHyracksTaskContext
+ * @return a tuple writer instance
+ */
+ public ITupleWriter getTupleWriter(IHyracksTaskContext ctx) throws HyracksDataException;
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
new file mode 100644
index 0000000..4fa0164
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/ConfFactory.java
@@ -0,0 +1,41 @@
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("deprecation")
+public class ConfFactory implements Serializable {
+ private static final long serialVersionUID = 1L;
+ private byte[] confBytes;
+
+ public ConfFactory(JobConf conf) throws HyracksDataException {
+ try {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ conf.write(dos);
+ confBytes = bos.toByteArray();
+ dos.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public JobConf getConf() throws HyracksDataException {
+ try {
+ JobConf conf = new JobConf();
+ DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+ conf.readFields(dis);
+ dis.close();
+ return conf;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..f49688b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.util.Arrays;
+
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop old API.
+ * To use this operator, a user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private final ConfFactory confFactory;
+ private final InputSplitsFactory splitsFactory;
+ private final String[] scheduledLocations;
+ private final IKeyValueParserFactory tupleParserFactory;
+ private final boolean[] executed;
+
+ /**
+ * The constructor of HDFSReadOperatorDescriptor.
+ *
+ * @param spec
+ * the JobSpecification object
+ * @param rd
+ * the output record descriptor
+ * @param conf
+ * the Hadoop JobConf object, which contains the input format and the input paths
+ * @param splits
+ * the array of FileSplits (HDFS chunks).
+ * @param scheduledLocations
+ * the node controller names to scan the FileSplits, which is an one-to-one mapping. The String array
+ * is obtained from the edu.cui.ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints(InputSplits[]).
+ * @param tupleParserFactory
+ * the ITupleParserFactory implementation instance.
+ * @throws HyracksException
+ */
+ public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, JobConf conf, InputSplit[] splits,
+ String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+ super(spec, 0, 1);
+ try {
+ this.splitsFactory = new InputSplitsFactory(splits);
+ this.confFactory = new ConfFactory(conf);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ this.scheduledLocations = scheduledLocations;
+ this.executed = new boolean[scheduledLocations.length];
+ Arrays.fill(executed, false);
+ this.tupleParserFactory = tupleParserFactory;
+ this.recordDescriptors[0] = rd;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
+ final InputSplit[] inputSplits = splitsFactory.getSplits();
+
+ return new AbstractUnaryOutputSourceOperatorNodePushable() {
+ private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void initialize() throws HyracksDataException {
+ ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
+ try {
+ Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+ JobConf conf = confFactory.getConf();
+ IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+ writer.open();
+ parser.open(writer);
+ InputFormat inputFormat = conf.getInputFormat();
+ for (int i = 0; i < inputSplits.length; i++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (scheduledLocations[i].equals(nodeName)) {
+ /**
+ * pick an unread split to read
+ * synchronize among simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[i] == false) {
+ executed[i] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ RecordReader reader = inputFormat.getRecordReader(inputSplits[i], conf, Reporter.NULL);
+ Object key = reader.createKey();
+ Object value = reader.createValue();
+ while (reader.next(key, value) == true) {
+ parser.parse(key, value, writer);
+ }
+ }
+ }
+ parser.close(writer);
+ writer.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+ }
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..3ce6b2a
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop old API.
+ * To use this operator, a user need to provide an ITupleWriterFactory.
+ */
+@SuppressWarnings("deprecation")
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private ConfFactory confFactory;
+ private ITupleWriterFactory tupleWriterFactory;
+
+ /**
+ * The constructor of HDFSWriteOperatorDescriptor.
+ *
+ * @param spec
+ * the JobSpecification object
+ * @param conf
+ * the Hadoop JobConf which contains the output path
+ * @param tupleWriterFactory
+ * the ITupleWriterFactory implementation object
+ * @throws HyracksException
+ */
+ public HDFSWriteOperatorDescriptor(JobSpecification spec, JobConf conf, ITupleWriterFactory tupleWriterFactory)
+ throws HyracksException {
+ super(spec, 1, 0);
+ this.confFactory = new ConfFactory(conf);
+ this.tupleWriterFactory = tupleWriterFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+ private FSDataOutputStream dos;
+ private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+ private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+ private FrameTupleReference tuple = new FrameTupleReference();
+ private ITupleWriter tupleWriter;
+ private ClassLoader ctxCL;
+
+ @Override
+ public void open() throws HyracksDataException {
+ ctxCL = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ JobConf conf = confFactory.getConf();
+ String outputDirPath = FileOutputFormat.getOutputPath(conf).toString();
+ String fileName = outputDirPath + File.separator + "part-" + partition;
+
+ tupleWriter = tupleWriterFactory.getTupleWriter(ctx);
+ try {
+ FileSystem dfs = FileSystem.get(conf);
+ dos = dfs.create(new Path(fileName), true);
+ tupleWriter.open(dos);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ tupleWriter.write(dos, tuple);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ tupleWriter.close(dos);
+ dos.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+ }
+
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
new file mode 100644
index 0000000..147e872
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/InputSplitsFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class InputSplitsFactory implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private byte[] splitBytes;
+ private String splitClassName;
+
+ public InputSplitsFactory(InputSplit[] splits) throws HyracksDataException {
+ splitBytes = splitsToBytes(splits);
+ if (splits.length > 0) {
+ splitClassName = splits[0].getClass().getName();
+ } else {
+ splitClassName = FileSplit.class.getName();
+ }
+ }
+
+ public InputSplit[] getSplits() throws HyracksDataException {
+ return bytesToSplits(splitBytes);
+ }
+
+ /**
+ * Convert splits to bytes.
+ *
+ * @param splits
+ * input splits
+ * @return bytes which serialize the splits
+ * @throws IOException
+ */
+ private byte[] splitsToBytes(InputSplit[] splits) throws HyracksDataException {
+ try {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ dos.writeInt(splits.length);
+ for (int i = 0; i < splits.length; i++) {
+ splits[i].write(dos);
+ }
+ dos.close();
+ return bos.toByteArray();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ /**
+ * Covert bytes to splits.
+ *
+ * @param bytes
+ * @return
+ * @throws HyracksDataException
+ */
+ private InputSplit[] bytesToSplits(byte[] bytes) throws HyracksDataException {
+ try {
+ Class splitClass = Class.forName(splitClassName);
+ Constructor[] constructors = splitClass.getDeclaredConstructors();
+ Constructor defaultConstructor = null;
+ for (Constructor constructor : constructors) {
+ if (constructor.getParameterTypes().length == 0) {
+ constructor.setAccessible(true);
+ defaultConstructor = constructor;
+ }
+ }
+ ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+ DataInputStream dis = new DataInputStream(bis);
+ int size = dis.readInt();
+ InputSplit[] splits = new InputSplit[size];
+ for (int i = 0; i < size; i++) {
+ splits[i] = (InputSplit) defaultConstructor.newInstance();
+ splits[i].readFields(dis);
+ }
+ dis.close();
+ return splits;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..90c5977
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryComparatorFactory.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+ private RawBinaryComparatorFactory() {
+ }
+
+ @Override
+ public IBinaryComparator createBinaryComparator() {
+ return new IBinaryComparator() {
+
+ @Override
+ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+ int commonLength = Math.min(l1, l2);
+ for (int i = 0; i < commonLength; i++) {
+ if (b1[s1 + i] != b2[s2 + i]) {
+ return b1[s1 + i] - b2[s2 + i];
+ }
+ }
+ int difference = l1 - l2;
+ return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+ }
+
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
new file mode 100644
index 0000000..7895fec
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/RawBinaryHashFunctionFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+
+public class RawBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
+ private static final long serialVersionUID = 1L;
+
+ public static IBinaryHashFunctionFactory INSTANCE = new RawBinaryHashFunctionFactory();
+
+ private RawBinaryHashFunctionFactory() {
+ }
+
+ @Override
+ public IBinaryHashFunction createBinaryHashFunction() {
+
+ return new IBinaryHashFunction() {
+ @Override
+ public int hash(byte[] bytes, int offset, int length) {
+ int value = 1;
+ int end = offset + length;
+ for (int i = offset; i < end; i++)
+ value = value * 31 + (int) bytes[i];
+ return value;
+ }
+ };
+ }
+
+}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
new file mode 100644
index 0000000..9574bb4
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+public class TextKeyValueParserFactory implements IKeyValueParserFactory<LongWritable, Text> {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IKeyValueParser<LongWritable, Text> createKeyValueParser(final IHyracksTaskContext ctx) {
+
+ final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+ final ByteBuffer buffer = ctx.allocateFrame();
+ final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender.reset(buffer, true);
+
+ return new IKeyValueParser<LongWritable, Text>() {
+
+ @Override
+ public void open(IFrameWriter writer) {
+
+ }
+
+ @Override
+ public void parse(LongWritable key, Text value, IFrameWriter writer) throws HyracksDataException {
+ tb.reset();
+ tb.addField(value.getBytes(), 0, value.getLength());
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(buffer, writer);
+ appender.reset(buffer, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new HyracksDataException("tuple cannot be appended into the frame");
+ }
+ }
+ }
+
+ @Override
+ public void close(IFrameWriter writer) throws HyracksDataException {
+ FrameUtils.flushFrame(buffer, writer);
+ }
+
+ };
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
new file mode 100644
index 0000000..0da14e5
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextTupleWriterFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.lib;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+public class TextTupleWriterFactory implements ITupleWriterFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public ITupleWriter getTupleWriter(IHyracksTaskContext ctx) {
+ return new ITupleWriter() {
+ private byte newLine = "\n".getBytes()[0];
+
+ @Override
+ public void open(DataOutput output) {
+
+ }
+
+ @Override
+ public void write(DataOutput output, ITupleReference tuple) throws HyracksDataException {
+ byte[] data = tuple.getFieldData(0);
+ int start = tuple.getFieldStart(0);
+ int len = tuple.getFieldLength(0);
+ try {
+ output.write(data, start, len);
+ output.writeByte(newLine);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void close(DataOutput output) {
+
+ }
+
+ };
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
new file mode 100644
index 0000000..320b48b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/IPProximityNcCollectionBuilder.java
@@ -0,0 +1,121 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class IPProximityNcCollectionBuilder implements INcCollectionBuilder {
+
+ @Override
+ public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+ final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+ final int[] workloads, final int slotLimit) {
+ final TreeMap<BytesWritable, IntWritable> availableIpsToSlots = new TreeMap<BytesWritable, IntWritable>();
+ for (int i = 0; i < workloads.length; i++) {
+ if (workloads[i] < slotLimit) {
+ BytesWritable ip = new BytesWritable(ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress());
+ IntWritable availableSlot = availableIpsToSlots.get(ip);
+ if (availableSlot == null) {
+ availableSlot = new IntWritable(slotLimit - workloads[i]);
+ availableIpsToSlots.put(ip, availableSlot);
+ } else {
+ availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+ }
+ }
+ }
+ return new INcCollection() {
+
+ @Override
+ public String findNearestAvailableSlot(InputSplit split) {
+ try {
+ String[] locs = split.getLocations();
+ int minDistance = Integer.MAX_VALUE;
+ BytesWritable currentCandidateIp = null;
+ if (locs == null || locs.length > 0) {
+ for (int j = 0; j < locs.length; j++) {
+ /**
+ * get all the IP addresses from the name
+ */
+ InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+ for (InetAddress ip : allIps) {
+ BytesWritable splitIp = new BytesWritable(ip.getAddress());
+ /**
+ * if the node controller exists
+ */
+ BytesWritable candidateNcIp = availableIpsToSlots.floorKey(splitIp);
+ if (candidateNcIp == null) {
+ candidateNcIp = availableIpsToSlots.ceilingKey(splitIp);
+ }
+ if (candidateNcIp != null) {
+ if (availableIpsToSlots.get(candidateNcIp).get() > 0) {
+ byte[] candidateIP = candidateNcIp.getBytes();
+ byte[] splitIP = splitIp.getBytes();
+ int candidateInt = candidateIP[0] << 24 | (candidateIP[1] & 0xFF) << 16
+ | (candidateIP[2] & 0xFF) << 8 | (candidateIP[3] & 0xFF);
+ int splitInt = splitIP[0] << 24 | (splitIP[1] & 0xFF) << 16
+ | (splitIP[2] & 0xFF) << 8 | (splitIP[3] & 0xFF);
+ int distance = Math.abs(candidateInt - splitInt);
+ if (minDistance > distance) {
+ minDistance = distance;
+ currentCandidateIp = candidateNcIp;
+ }
+ }
+ }
+ }
+ }
+ } else {
+ for (Entry<BytesWritable, IntWritable> entry : availableIpsToSlots.entrySet()) {
+ if (entry.getValue().get() > 0) {
+ currentCandidateIp = entry.getKey();
+ break;
+ }
+ }
+ }
+
+ if (currentCandidateIp != null) {
+ /**
+ * Update the entry of the selected IP
+ */
+ IntWritable availableSlot = availableIpsToSlots.get(currentCandidateIp);
+ availableSlot.set(availableSlot.get() - 1);
+ if (availableSlot.get() == 0) {
+ availableIpsToSlots.remove(currentCandidateIp);
+ }
+ /**
+ * Update the entry of the selected NC
+ */
+ List<String> dataLocations = ipToNcMapping.get(InetAddress.getByAddress(
+ currentCandidateIp.getBytes()).getHostAddress());
+ for (String nc : dataLocations) {
+ int ncIndex = ncNameToIndex.get(nc);
+ if (workloads[ncIndex] < slotLimit) {
+ return nc;
+ }
+ }
+ }
+ /** not scheduled */
+ return null;
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public int numAvailableSlots() {
+ return availableIpsToSlots.size();
+ }
+
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
new file mode 100644
index 0000000..fa5e7ae
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/RackAwareNcCollectionBuilder.java
@@ -0,0 +1,211 @@
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.TreeMap;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+@SuppressWarnings("deprecation")
+public class RackAwareNcCollectionBuilder implements INcCollectionBuilder {
+ private static final Logger LOGGER = Logger.getLogger(RackAwareNcCollectionBuilder.class.getName());
+ private ClusterTopology topology;
+
+ public RackAwareNcCollectionBuilder(ClusterTopology topology) {
+ this.topology = topology;
+ }
+
+ @Override
+ public INcCollection build(Map<String, NodeControllerInfo> ncNameToNcInfos,
+ final Map<String, List<String>> ipToNcMapping, final Map<String, Integer> ncNameToIndex, String[] NCs,
+ final int[] workloads, final int slotLimit) {
+ try {
+ final Map<List<Integer>, List<String>> pathToNCs = new HashMap<List<Integer>, List<String>>();
+ for (int i = 0; i < NCs.length; i++) {
+ List<Integer> path = new ArrayList<Integer>();
+ String ipAddress = InetAddress.getByAddress(
+ ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+ topology.lookupNetworkTerminal(ipAddress, path);
+ if (path.size() <= 0) {
+ // if the hyracks nc is not in the defined cluster
+ path.add(Integer.MIN_VALUE);
+ LOGGER.info(NCs[i] + "'s IP address is not in the cluster toplogy file!");
+ }
+ List<String> ncs = pathToNCs.get(path);
+ if (ncs == null) {
+ ncs = new ArrayList<String>();
+ pathToNCs.put(path, ncs);
+ }
+ ncs.add(NCs[i]);
+ }
+
+ final TreeMap<List<Integer>, IntWritable> availableIpsToSlots = new TreeMap<List<Integer>, IntWritable>(
+ new Comparator<List<Integer>>() {
+
+ @Override
+ public int compare(List<Integer> l1, List<Integer> l2) {
+ int commonLength = Math.min(l1.size(), l2.size());
+ for (int i = 0; i < commonLength; i++) {
+ Integer value1 = l1.get(i);
+ Integer value2 = l2.get(i);
+ int cmp = value1 > value2 ? 1 : (value1 < value2 ? -1 : 0);
+ if (cmp != 0) {
+ return cmp;
+ }
+ }
+ return l1.size() > l2.size() ? 1 : (l1.size() < l2.size() ? -1 : 0);
+ }
+
+ });
+ for (int i = 0; i < workloads.length; i++) {
+ if (workloads[i] < slotLimit) {
+ List<Integer> path = new ArrayList<Integer>();
+ String ipAddress = InetAddress.getByAddress(
+ ncNameToNcInfos.get(NCs[i]).getNetworkAddress().getIpAddress()).getHostAddress();
+ topology.lookupNetworkTerminal(ipAddress, path);
+ if (path.size() <= 0) {
+ // if the hyracks nc is not in the defined cluster
+ path.add(Integer.MIN_VALUE);
+ }
+ IntWritable availableSlot = availableIpsToSlots.get(path);
+ if (availableSlot == null) {
+ availableSlot = new IntWritable(slotLimit - workloads[i]);
+ availableIpsToSlots.put(path, availableSlot);
+ } else {
+ availableSlot.set(slotLimit - workloads[i] + availableSlot.get());
+ }
+ }
+ }
+ return new INcCollection() {
+
+ @Override
+ public String findNearestAvailableSlot(InputSplit split) {
+ try {
+ String[] locs = split.getLocations();
+ int minDistance = Integer.MAX_VALUE;
+ List<Integer> currentCandidatePath = null;
+ if (locs == null || locs.length > 0) {
+ for (int j = 0; j < locs.length; j++) {
+ /**
+ * get all the IP addresses from the name
+ */
+ InetAddress[] allIps = InetAddress.getAllByName(locs[j]);
+ boolean inTopology = false;
+ for (InetAddress ip : allIps) {
+ List<Integer> splitPath = new ArrayList<Integer>();
+ boolean inCluster = topology.lookupNetworkTerminal(ip.getHostAddress(), splitPath);
+ if (!inCluster) {
+ continue;
+ }
+ inTopology = true;
+ /**
+ * if the node controller exists
+ */
+ List<Integer> candidatePath = availableIpsToSlots.floorKey(splitPath);
+ if (candidatePath == null) {
+ candidatePath = availableIpsToSlots.ceilingKey(splitPath);
+ }
+ if (candidatePath != null) {
+ if (availableIpsToSlots.get(candidatePath).get() > 0) {
+ int distance = distance(splitPath, candidatePath);
+ if (minDistance > distance) {
+ minDistance = distance;
+ currentCandidatePath = candidatePath;
+ }
+ }
+
+ }
+ }
+
+ if (!inTopology) {
+ LOGGER.info(locs[j] + "'s IP address is not in the cluster toplogy file!");
+ /**
+ * if the machine is not in the toplogy file
+ */
+ List<Integer> candidatePath = null;
+ for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+ if (entry.getValue().get() > 0) {
+ candidatePath = entry.getKey();
+ break;
+ }
+ }
+ /** the split path is empty */
+ if (candidatePath != null) {
+ if (availableIpsToSlots.get(candidatePath).get() > 0) {
+ currentCandidatePath = candidatePath;
+ }
+ }
+ }
+ }
+ } else {
+ for (Entry<List<Integer>, IntWritable> entry : availableIpsToSlots.entrySet()) {
+ if (entry.getValue().get() > 0) {
+ currentCandidatePath = entry.getKey();
+ break;
+ }
+ }
+ }
+
+ if (currentCandidatePath != null && currentCandidatePath.size() > 0) {
+ /**
+ * Update the entry of the selected IP
+ */
+ IntWritable availableSlot = availableIpsToSlots.get(currentCandidatePath);
+ availableSlot.set(availableSlot.get() - 1);
+ if (availableSlot.get() == 0) {
+ availableIpsToSlots.remove(currentCandidatePath);
+ }
+ /**
+ * Update the entry of the selected NC
+ */
+ List<String> candidateNcs = pathToNCs.get(currentCandidatePath);
+ for (String candidate : candidateNcs) {
+ int ncIndex = ncNameToIndex.get(candidate);
+ if (workloads[ncIndex] < slotLimit) {
+ return candidate;
+ }
+ }
+ }
+ /** not scheduled */
+ return null;
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public int numAvailableSlots() {
+ return availableIpsToSlots.size();
+ }
+
+ private int distance(List<Integer> splitPath, List<Integer> candidatePath) {
+ int commonLength = Math.min(splitPath.size(), candidatePath.size());
+ int distance = 0;
+ for (int i = 0; i < commonLength; i++) {
+ distance = distance * 100 + Math.abs(splitPath.get(i) - candidatePath.get(i));
+ }
+ List<Integer> restElements = splitPath.size() > candidatePath.size() ? splitPath : candidatePath;
+ for (int i = commonLength; i < restElements.size(); i++) {
+ distance = distance * 100 + Math.abs(restElements.get(i));
+ }
+ return distance;
+ }
+ };
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
new file mode 100644
index 0000000..3f7997b
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/scheduler/Scheduler.java
@@ -0,0 +1,404 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Random;
+import java.util.logging.Logger;
+
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.mapred.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.hdfs.api.INcCollection;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS. This
+ * class works for Hadoop old API.
+ */
+@SuppressWarnings("deprecation")
+public class Scheduler {
+ private static final Logger LOGGER = Logger.getLogger(Scheduler.class.getName());
+
+ /** a list of NCs */
+ private String[] NCs;
+
+ /** a map from ip to NCs */
+ private Map<String, List<String>> ipToNcMapping = new HashMap<String, List<String>>();
+
+ /** a map from the NC name to the index */
+ private Map<String, Integer> ncNameToIndex = new HashMap<String, Integer>();
+
+ /** a map from NC name to the NodeControllerInfo */
+ private Map<String, NodeControllerInfo> ncNameToNcInfos;
+
+ /**
+ * the nc collection builder
+ */
+ private INcCollectionBuilder ncCollectionBuilder;
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ public Scheduler(String ipAddress, int port) throws HyracksException {
+ try {
+ IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+ this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+ ClusterTopology topology = hcc.getClusterTopology();
+ this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+ : new RackAwareNcCollectionBuilder(topology);
+ loadIPAddressToNCMap(ncNameToNcInfos);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ public Scheduler(String ipAddress, int port, INcCollectionBuilder ncCollectionBuilder) throws HyracksException {
+ try {
+ IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+ this.ncNameToNcInfos = hcc.getNodeControllerInfos();
+ this.ncCollectionBuilder = ncCollectionBuilder;
+ loadIPAddressToNCMap(ncNameToNcInfos);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * the mapping from nc names to nc infos
+ * @throws HyracksException
+ */
+ public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+ this.ncNameToNcInfos = ncNameToNcInfos;
+ this.ncCollectionBuilder = new IPProximityNcCollectionBuilder();
+ loadIPAddressToNCMap(ncNameToNcInfos);
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * the mapping from nc names to nc infos
+ * @param topology
+ * the hyracks cluster toplogy
+ * @throws HyracksException
+ */
+ public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, ClusterTopology topology) throws HyracksException {
+ this(ncNameToNcInfos);
+ this.ncCollectionBuilder = topology == null ? new IPProximityNcCollectionBuilder()
+ : new RackAwareNcCollectionBuilder(topology);
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * the mapping from nc names to nc infos
+ * @throws HyracksException
+ */
+ public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder ncCollectionBuilder)
+ throws HyracksException {
+ this.ncNameToNcInfos = ncNameToNcInfos;
+ this.ncCollectionBuilder = ncCollectionBuilder;
+ loadIPAddressToNCMap(ncNameToNcInfos);
+ }
+
+ /**
+ * Set location constraints for a file scan operator with a list of file
+ * splits. It guarantees the maximum slots a machine can is at most one more
+ * than the minimum slots a machine can get.
+ *
+ * @throws HyracksDataException
+ */
+ public String[] getLocationConstraints(InputSplit[] splits) throws HyracksException {
+ if (splits == null) {
+ /** deal the case when the splits array is null */
+ return new String[] {};
+ }
+ int[] workloads = new int[NCs.length];
+ Arrays.fill(workloads, 0);
+ String[] locations = new String[splits.length];
+ Map<String, IntWritable> locationToNumOfSplits = new HashMap<String, IntWritable>();
+ /**
+ * upper bound number of slots that a machine can get
+ */
+ int upperBoundSlots = splits.length % workloads.length == 0 ? (splits.length / workloads.length)
+ : (splits.length / workloads.length + 1);
+ /**
+ * lower bound number of slots that a machine can get
+ */
+ int lowerBoundSlots = splits.length % workloads.length == 0 ? upperBoundSlots : upperBoundSlots - 1;
+
+ try {
+ Random random = new Random(System.currentTimeMillis());
+ boolean scheduled[] = new boolean[splits.length];
+ Arrays.fill(scheduled, false);
+ /**
+ * scan the splits and build the popularity map
+ * give the machines with less local splits more scheduling priority
+ */
+ buildPopularityMap(splits, locationToNumOfSplits);
+ /**
+ * push data-local lower-bounds slots to each machine
+ */
+ scheduleLocalSlots(splits, workloads, locations, lowerBoundSlots, random, scheduled, locationToNumOfSplits);
+ /**
+ * push data-local upper-bounds slots to each machine
+ */
+ scheduleLocalSlots(splits, workloads, locations, upperBoundSlots, random, scheduled, locationToNumOfSplits);
+
+ int dataLocalCount = 0;
+ for (int i = 0; i < scheduled.length; i++) {
+ if (scheduled[i] == true) {
+ dataLocalCount++;
+ }
+ }
+ LOGGER.info("Data local rate: "
+ + (scheduled.length == 0 ? 0.0 : ((float) dataLocalCount / (float) (scheduled.length))));
+ /**
+ * push non-data-local lower-bounds slots to each machine
+ */
+ scheduleNonLocalSlots(splits, workloads, locations, lowerBoundSlots, scheduled);
+ /**
+ * push non-data-local upper-bounds slots to each machine
+ */
+ scheduleNonLocalSlots(splits, workloads, locations, upperBoundSlots, scheduled);
+ return locations;
+ } catch (IOException e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ /**
+ * Schedule non-local slots to each machine
+ *
+ * @param splits
+ * The HDFS file splits.
+ * @param workloads
+ * The current capacity of each machine.
+ * @param locations
+ * The result schedule.
+ * @param slotLimit
+ * The maximum slots of each machine.
+ * @param scheduled
+ * Indicate which slot is scheduled.
+ */
+ private void scheduleNonLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slotLimit,
+ boolean[] scheduled) throws IOException, UnknownHostException {
+ /**
+ * build the map from available ips to the number of available slots
+ */
+ INcCollection ncCollection = this.ncCollectionBuilder.build(ncNameToNcInfos, ipToNcMapping, ncNameToIndex, NCs,
+ workloads, slotLimit);
+ if (ncCollection.numAvailableSlots() == 0) {
+ return;
+ }
+ /**
+ * schedule no-local file reads
+ */
+ for (int i = 0; i < splits.length; i++) {
+ /** if there is no data-local NC choice, choose a random one */
+ if (!scheduled[i]) {
+ InputSplit split = splits[i];
+ String selectedNcName = ncCollection.findNearestAvailableSlot(split);
+ if (selectedNcName != null) {
+ int ncIndex = ncNameToIndex.get(selectedNcName);
+ workloads[ncIndex]++;
+ scheduled[i] = true;
+ locations[i] = selectedNcName;
+ }
+ }
+ }
+ }
+
+ /**
+ * Schedule data-local slots to each machine.
+ *
+ * @param splits
+ * The HDFS file splits.
+ * @param workloads
+ * The current capacity of each machine.
+ * @param locations
+ * The result schedule.
+ * @param slots
+ * The maximum slots of each machine.
+ * @param random
+ * The random generator.
+ * @param scheduled
+ * Indicate which slot is scheduled.
+ * @throws IOException
+ * @throws UnknownHostException
+ */
+ private void scheduleLocalSlots(InputSplit[] splits, int[] workloads, String[] locations, int slots, Random random,
+ boolean[] scheduled, final Map<String, IntWritable> locationToNumSplits) throws IOException,
+ UnknownHostException {
+ /** scheduling candidates will be ordered inversely according to their popularity */
+ PriorityQueue<String> scheduleCadndiates = new PriorityQueue<String>(3, new Comparator<String>() {
+
+ @Override
+ public int compare(String s1, String s2) {
+ return locationToNumSplits.get(s1).compareTo(locationToNumSplits.get(s2));
+ }
+
+ });
+ for (int i = 0; i < splits.length; i++) {
+ if (scheduled[i]) {
+ continue;
+ }
+ /**
+ * get the location of all the splits
+ */
+ String[] locs = splits[i].getLocations();
+ if (locs.length > 0) {
+ scheduleCadndiates.clear();
+ for (int j = 0; j < locs.length; j++) {
+ scheduleCadndiates.add(locs[j]);
+ }
+
+ for (String candidate : scheduleCadndiates) {
+ /**
+ * get all the IP addresses from the name
+ */
+ InetAddress[] allIps = InetAddress.getAllByName(candidate);
+ /**
+ * iterate overa all ips
+ */
+ for (InetAddress ip : allIps) {
+ /**
+ * if the node controller exists
+ */
+ if (ipToNcMapping.get(ip.getHostAddress()) != null) {
+ /**
+ * set the ncs
+ */
+ List<String> dataLocations = ipToNcMapping.get(ip.getHostAddress());
+ int arrayPos = random.nextInt(dataLocations.size());
+ String nc = dataLocations.get(arrayPos);
+ int pos = ncNameToIndex.get(nc);
+ /**
+ * check if the node is already full
+ */
+ if (workloads[pos] < slots) {
+ locations[i] = nc;
+ workloads[pos]++;
+ scheduled[i] = true;
+ break;
+ }
+ }
+ }
+ /**
+ * break the loop for data-locations if the schedule has
+ * already been found
+ */
+ if (scheduled[i] == true) {
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Scan the splits once and build a popularity map
+ *
+ * @param splits
+ * the split array
+ * @param locationToNumOfSplits
+ * the map to be built
+ * @throws IOException
+ */
+ private void buildPopularityMap(InputSplit[] splits, Map<String, IntWritable> locationToNumOfSplits)
+ throws IOException {
+ for (InputSplit split : splits) {
+ String[] locations = split.getLocations();
+ for (String loc : locations) {
+ IntWritable locCount = locationToNumOfSplits.get(loc);
+ if (locCount == null) {
+ locCount = new IntWritable(0);
+ locationToNumOfSplits.put(loc, locCount);
+ }
+ locCount.set(locCount.get() + 1);
+ }
+ }
+ }
+
+ /**
+ * Load the IP-address-to-NC map from the NCNameToNCInfoMap
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ private void loadIPAddressToNCMap(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+ try {
+ NCs = new String[ncNameToNcInfos.size()];
+ ipToNcMapping.clear();
+ ncNameToIndex.clear();
+ int i = 0;
+
+ /**
+ * build the IP address to NC map
+ */
+ for (Map.Entry<String, NodeControllerInfo> entry : ncNameToNcInfos.entrySet()) {
+ String ipAddr = InetAddress.getByAddress(entry.getValue().getNetworkAddress().getIpAddress())
+ .getHostAddress();
+ List<String> matchedNCs = ipToNcMapping.get(ipAddr);
+ if (matchedNCs == null) {
+ matchedNCs = new ArrayList<String>();
+ ipToNcMapping.put(ipAddr, matchedNCs);
+ }
+ matchedNCs.add(entry.getKey());
+ NCs[i] = entry.getKey();
+ i++;
+ }
+
+ /**
+ * set up the NC name to index mapping
+ */
+ for (i = 0; i < NCs.length; i++) {
+ ncNameToIndex.put(NCs[i], i);
+ }
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
new file mode 100644
index 0000000..d843d27
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/ConfFactory.java
@@ -0,0 +1,40 @@
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.Serializable;
+
+import org.apache.hadoop.mapreduce.Job;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ConfFactory implements Serializable {
+ private static final long serialVersionUID = 1L;
+ private byte[] confBytes;
+
+ public ConfFactory(Job conf) throws HyracksDataException {
+ try {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ conf.getConfiguration().write(dos);
+ confBytes = bos.toByteArray();
+ dos.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public Job getConf() throws HyracksDataException {
+ try {
+ Job conf = new Job();
+ DataInputStream dis = new DataInputStream(new ByteArrayInputStream(confBytes));
+ conf.getConfiguration().readFields(dis);
+ dis.close();
+ return conf;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
new file mode 100644
index 0000000..14dc70c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/FileSplitsFactory.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Constructor;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class FileSplitsFactory implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+ private byte[] splitBytes;
+ private String splitClassName;
+
+ public FileSplitsFactory(List<FileSplit> splits) throws HyracksDataException {
+ splitBytes = splitsToBytes(splits);
+ if (splits.size() > 0) {
+ splitClassName = splits.get(0).getClass().getName();
+ }
+ }
+
+ public List<FileSplit> getSplits() throws HyracksDataException {
+ return bytesToSplits(splitBytes);
+ }
+
+ /**
+ * Convert splits to bytes.
+ *
+ * @param splits
+ * input splits
+ * @return bytes which serialize the splits
+ * @throws IOException
+ */
+ private byte[] splitsToBytes(List<FileSplit> splits) throws HyracksDataException {
+ try {
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ DataOutputStream dos = new DataOutputStream(bos);
+ dos.writeInt(splits.size());
+ int size = splits.size();
+ for (int i = 0; i < size; i++) {
+ splits.get(i).write(dos);
+ }
+ dos.close();
+ return bos.toByteArray();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ /**
+ * Covert bytes to splits.
+ *
+ * @param bytes
+ * @return
+ * @throws HyracksDataException
+ */
+ private List<FileSplit> bytesToSplits(byte[] bytes) throws HyracksDataException {
+ try {
+ Class splitClass = Class.forName(splitClassName);
+ Constructor[] constructors = splitClass.getDeclaredConstructors();
+ Constructor defaultConstructor = null;
+ for (Constructor constructor : constructors) {
+ if (constructor.getParameterTypes().length == 0) {
+ constructor.setAccessible(true);
+ defaultConstructor = constructor;
+ }
+ }
+ ByteArrayInputStream bis = new ByteArrayInputStream(bytes);
+ DataInputStream dis = new DataInputStream(bis);
+ int size = dis.readInt();
+ List<FileSplit> splits = new ArrayList<FileSplit>();
+ for (int i = 0; i < size; i++) {
+ splits.add((FileSplit) defaultConstructor.newInstance());
+ splits.get(i).readFields(dis);
+ }
+ dis.close();
+ return splits;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
new file mode 100644
index 0000000..9e9abdf
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSReadOperatorDescriptor.java
@@ -0,0 +1,160 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParser;
+import edu.uci.ics.hyracks.hdfs.api.IKeyValueParserFactory;
+
+/**
+ * The HDFS file read operator using the Hadoop new API. To use this operator, a
+ * user need to provide an IKeyValueParserFactory implementation which convert
+ * key-value pairs into tuples.
+ */
+@SuppressWarnings("rawtypes")
+public class HDFSReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private final ConfFactory confFactory;
+ private final FileSplitsFactory splitsFactory;
+ private final String[] scheduledLocations;
+ private final IKeyValueParserFactory tupleParserFactory;
+ private final boolean[] executed;
+
+ /**
+ * The constructor of HDFSReadOperatorDescriptor.
+ *
+ * @param spec
+ * the JobSpecification object
+ * @param rd
+ * the output record descriptor
+ * @param conf
+ * the Hadoop JobConf object, which contains the input format and
+ * the input paths
+ * @param splits
+ * the array of FileSplits (HDFS chunks).
+ * @param scheduledLocations
+ * the node controller names to scan the FileSplits, which is an
+ * one-to-one mapping. The String array is obtained from the
+ * edu.cui
+ * .ics.hyracks.hdfs.scheduler.Scheduler.getLocationConstraints
+ * (InputSplits[]).
+ * @param tupleParserFactory
+ * the ITupleParserFactory implementation instance.
+ * @throws HyracksException
+ */
+ public HDFSReadOperatorDescriptor(JobSpecification spec, RecordDescriptor rd, Job conf, List<InputSplit> splits,
+ String[] scheduledLocations, IKeyValueParserFactory tupleParserFactory) throws HyracksException {
+ super(spec, 0, 1);
+ try {
+ List<FileSplit> fileSplits = new ArrayList<FileSplit>();
+ for (int i = 0; i < splits.size(); i++) {
+ fileSplits.add((FileSplit) splits.get(i));
+ }
+ this.splitsFactory = new FileSplitsFactory(fileSplits);
+ this.confFactory = new ConfFactory(conf);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ this.scheduledLocations = scheduledLocations;
+ this.executed = new boolean[scheduledLocations.length];
+ Arrays.fill(executed, false);
+ this.tupleParserFactory = tupleParserFactory;
+ this.recordDescriptors[0] = rd;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
+ final List<FileSplit> inputSplits = splitsFactory.getSplits();
+
+ return new AbstractUnaryOutputSourceOperatorNodePushable() {
+ private String nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+ private ContextFactory ctxFactory = new ContextFactory();
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void initialize() throws HyracksDataException {
+ ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
+ try {
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ Job job = confFactory.getConf();
+ IKeyValueParser parser = tupleParserFactory.createKeyValueParser(ctx);
+ writer.open();
+ InputFormat inputFormat = ReflectionUtils.newInstance(job.getInputFormatClass(),
+ job.getConfiguration());
+ int size = inputSplits.size();
+ for (int i = 0; i < size; i++) {
+ /**
+ * read all the partitions scheduled to the current node
+ */
+ if (scheduledLocations[i].equals(nodeName)) {
+ /**
+ * pick an unread split to read synchronize among
+ * simultaneous partitions in the same machine
+ */
+ synchronized (executed) {
+ if (executed[i] == false) {
+ executed[i] = true;
+ } else {
+ continue;
+ }
+ }
+
+ /**
+ * read the split
+ */
+ TaskAttemptContext context = ctxFactory.createContext(job.getConfiguration(), i);
+ RecordReader reader = inputFormat.createRecordReader(inputSplits.get(i), context);
+ reader.initialize(inputSplits.get(i), context);
+ while (reader.nextKeyValue() == true) {
+ parser.parse(reader.getCurrentKey(), reader.getCurrentValue(), writer);
+ }
+ }
+ }
+ parser.close(writer);
+ writer.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+ }
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
new file mode 100644
index 0000000..c1c227c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriter;
+import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
+
+/**
+ * The HDFS file write operator using the Hadoop new API. To use this operator,
+ * a user need to provide an ITupleWriterFactory.
+ */
+public class HDFSWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ private ConfFactory confFactory;
+ private ITupleWriterFactory tupleWriterFactory;
+
+ /**
+ * The constructor of HDFSWriteOperatorDescriptor.
+ *
+ * @param spec
+ * the JobSpecification object
+ * @param conf
+ * the Hadoop JobConf which contains the output path
+ * @param tupleWriterFactory
+ * the ITupleWriterFactory implementation object
+ * @throws HyracksException
+ */
+ public HDFSWriteOperatorDescriptor(JobSpecification spec, Job conf, ITupleWriterFactory tupleWriterFactory)
+ throws HyracksException {
+ super(spec, 1, 0);
+ this.confFactory = new ConfFactory(conf);
+ this.tupleWriterFactory = tupleWriterFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
+ throws HyracksDataException {
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+
+ private FSDataOutputStream dos;
+ private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
+ private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+ private FrameTupleReference tuple = new FrameTupleReference();
+ private ITupleWriter tupleWriter;
+ private ClassLoader ctxCL;
+
+ @Override
+ public void open() throws HyracksDataException {
+ ctxCL = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ Job conf = confFactory.getConf();
+ String outputPath = FileOutputFormat.getOutputPath(conf).toString();
+ String fileName = outputPath + File.separator + "part-" + partition;
+
+ tupleWriter = tupleWriterFactory.getTupleWriter(ctx);
+ try {
+ FileSystem dfs = FileSystem.get(conf.getConfiguration());
+ dos = dfs.create(new Path(fileName), true);
+ tupleWriter.open(dos);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ tupleWriter.write(dos, tuple);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ tupleWriter.close(dos);
+ dos.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+ }
+
+ };
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
new file mode 100644
index 0000000..cde187d
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/Scheduler.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.hdfs.api.INcCollectionBuilder;
+
+/**
+ * The scheduler conduct data-local scheduling for data reading on HDFS.
+ * This class works for Hadoop new API.
+ */
+@SuppressWarnings("deprecation")
+public class Scheduler {
+
+ private edu.uci.ics.hyracks.hdfs.scheduler.Scheduler scheduler;
+
+ /**
+ * The constructor of the scheduler
+ *
+ * @param ncNameToNcInfos
+ * @throws HyracksException
+ */
+ public Scheduler(String ipAddress, int port) throws HyracksException {
+ scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ipAddress, port);
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * the mapping from nc names to nc infos
+ * @throws HyracksException
+ */
+ public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos) throws HyracksException {
+ scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos);
+ }
+
+ /**
+ * The constructor of the scheduler.
+ *
+ * @param ncNameToNcInfos
+ * the mapping from nc names to nc infos
+ * @throws HyracksException
+ */
+ public Scheduler(Map<String, NodeControllerInfo> ncNameToNcInfos, INcCollectionBuilder builder) throws HyracksException {
+ scheduler = new edu.uci.ics.hyracks.hdfs.scheduler.Scheduler(ncNameToNcInfos, builder);
+ }
+
+ /**
+ * Set location constraints for a file scan operator with a list of file splits
+ *
+ * @throws HyracksDataException
+ */
+ public String[] getLocationConstraints(List<InputSplit> splits) throws HyracksException {
+ try {
+ org.apache.hadoop.mapred.InputSplit[] inputSplits = new org.apache.hadoop.mapred.InputSplit[splits.size()];
+ for (int i = 0; i < inputSplits.length; i++)
+ inputSplits[i] = new WrappedFileSplit(splits.get(i).getLocations(), splits.get(i).getLength());
+ return scheduler.getLocationConstraints(inputSplits);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java
new file mode 100644
index 0000000..1deb469
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/scheduler/WrappedFileSplit.java
@@ -0,0 +1,51 @@
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.InputSplit;
+
+/**
+ * The wrapped implementation of InputSplit, for the new API scheduler
+ * to reuse the old API scheduler
+ */
+@SuppressWarnings("deprecation")
+public class WrappedFileSplit implements InputSplit {
+
+ private String[] locations;
+ private long length;
+
+ public WrappedFileSplit(String[] locations, long length) {
+ this.locations = locations;
+ this.length = length;
+ }
+
+ @Override
+ public void readFields(DataInput input) throws IOException {
+ int len = input.readInt();
+ locations = new String[len];
+ for (int i = 0; i < len; i++)
+ locations[i] = input.readUTF();
+ length = input.readLong();
+ }
+
+ @Override
+ public void write(DataOutput output) throws IOException {
+ output.write(locations.length);
+ for (int i = 0; i < locations.length; i++)
+ output.writeUTF(locations[i]);
+ output.writeLong(length);
+ }
+
+ @Override
+ public long getLength() throws IOException {
+ return length;
+ }
+
+ @Override
+ public String[] getLocations() throws IOException {
+ return locations;
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
new file mode 100644
index 0000000..2686077
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/dataflow/DataflowTest.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.FileInputFormat;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TextInputFormat;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs.dataflow package,
+ * the operators for the Hadoop old API.
+ */
+@SuppressWarnings({ "deprecation" })
+public class DataflowTest extends TestCase {
+
+ private static final String ACTUAL_RESULT_DIR = "actual";
+ private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+ private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+ private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+ private static final String HDFS_INPUT_PATH = "/customer/";
+ private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+ private static final String HYRACKS_APP_NAME = "DataflowTest";
+ private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+ private MiniDFSCluster dfsCluster;
+
+ private JobConf conf = new JobConf();
+ private int numberOfNC = 2;
+
+ @Override
+ public void setUp() throws Exception {
+ cleanupStores();
+ HyracksUtils.init();
+ HyracksUtils.createApp(HYRACKS_APP_NAME);
+ FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+ FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+ startHDFS();
+ }
+
+ private void cleanupStores() throws IOException {
+ FileUtils.forceMkdir(new File("teststore"));
+ FileUtils.forceMkdir(new File("build"));
+ FileUtils.cleanDirectory(new File("teststore"));
+ FileUtils.cleanDirectory(new File("build"));
+ }
+
+ /**
+ * Start the HDFS cluster and setup the data files
+ *
+ * @throws IOException
+ */
+ private void startHDFS() throws IOException {
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+ conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+ FileSystem lfs = FileSystem.getLocal(new Configuration());
+ lfs.delete(new Path("build"), true);
+ System.setProperty("hadoop.log.dir", "logs");
+ dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+ FileSystem dfs = FileSystem.get(conf);
+ Path src = new Path(DATA_PATH);
+ Path dest = new Path(HDFS_INPUT_PATH);
+ Path result = new Path(HDFS_OUTPUT_PATH);
+ dfs.mkdirs(dest);
+ dfs.mkdirs(result);
+ dfs.copyFromLocalFile(src, dest);
+
+ DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+ conf.writeXml(confOutput);
+ confOutput.flush();
+ confOutput.close();
+ }
+
+ /**
+ * Test a job with only HDFS read and writes.
+ *
+ * @throws Exception
+ */
+ public void testHDFSReadWriteOperators() throws Exception {
+ FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+ FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+ conf.setInputFormat(TextInputFormat.class);
+
+ Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+ InputSplit[] splits = conf.getInputFormat().getSplits(conf, numberOfNC * 4);
+
+ String[] readSchedule = scheduler.getLocationConstraints(splits);
+ JobSpecification jobSpec = new JobSpecification();
+ RecordDescriptor recordDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+ String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+ HyracksUtils.NC2_ID };
+ HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+ readSchedule, new TextKeyValueParserFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+ ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+ new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+ HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+ new TextTupleWriterFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+ jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+ jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+ new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+ new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+ 0, writeOperator, 0);
+ jobSpec.addRoot(writeOperator);
+
+ IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+ HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+ JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+ client.waitForCompletion(jobId);
+
+ Assert.assertEquals(true, checkResults());
+ }
+
+ /**
+ * Check if the results are correct
+ *
+ * @return true if correct
+ * @throws Exception
+ */
+ private boolean checkResults() throws Exception {
+ FileSystem dfs = FileSystem.get(conf);
+ Path result = new Path(HDFS_OUTPUT_PATH);
+ Path actual = new Path(ACTUAL_RESULT_DIR);
+ dfs.copyToLocalFile(result, actual);
+
+ TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+ ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+ return true;
+ }
+
+ /**
+ * cleanup hdfs cluster
+ */
+ private void cleanupHDFS() throws Exception {
+ dfsCluster.shutdown();
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+ HyracksUtils.deinit();
+ cleanupHDFS();
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..1f394a9
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -0,0 +1,341 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.scheduler;
+
+import java.io.FileReader;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputSplit;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.topology.ClusterTopology;
+import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
+
+@SuppressWarnings("deprecation")
+public class SchedulerTest extends TestCase {
+ private static String TOPOLOGY_PATH = "src/test/resources/topology.xml";
+
+ private ClusterTopology parseTopology() throws IOException, SAXException {
+ FileReader fr = new FileReader(TOPOLOGY_PATH);
+ InputSource in = new InputSource(fr);
+ try {
+ return TopologyDefinitionParser.parse(in);
+ } finally {
+ fr.close();
+ }
+ }
+
+ /**
+ * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSimple() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ InputSplit[] fileSplits = new InputSplit[6];
+ fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+ fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+ fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ ClusterTopology topology = parseTopology();
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerLargerHDFS() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc7", new NodeControllerInfo("nc7", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.7").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc12", new NodeControllerInfo("nc12", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.12").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+
+ InputSplit[] fileSplits = new InputSplit[12];
+ fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+ fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+ fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+ fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[8] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.14", "10.0.0.11", "10.0.0.13" });
+ fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" });
+ fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" });
+ fileSplits[11] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" });
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc12",
+ "nc7", "nc7", "nc12" };
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc7", "nc12", "nc7",
+ "nc12" };
+ ClusterTopology topology = parseTopology();
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSmallerHDFS() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ InputSplit[] fileSplits = new InputSplit[12];
+ fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+ fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+ fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+ fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+ fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+ fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
+ "nc5", "nc6" };
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ ClusterTopology topology = parseTopology();
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSmallerHDFSOdd() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ InputSplit[] fileSplits = new InputSplit[13];
+ fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[1] = new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[2] = new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" });
+ fileSplits[3] = new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" });
+ fileSplits[4] = new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[5] = new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+ fileSplits[6] = new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
+ fileSplits[7] = new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[8] = new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" });
+ fileSplits[9] = new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" });
+ fileSplits[10] = new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" });
+ fileSplits[11] = new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" });
+ fileSplits[12] = new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" });
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+ "nc5", "nc2", "nc4" };
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ ClusterTopology topology = parseTopology();
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ }
+
+ /**
+ * Test boundary cases where splits array is empty or null
+ *
+ * @throws Exception
+ */
+ public void testSchedulercBoundary() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ /** test empty file splits */
+ InputSplit[] fileSplits = new InputSplit[0];
+ String[] expectedResults = new String[] {};
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ ClusterTopology topology = parseTopology();
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ fileSplits = null;
+ expectedResults = new String[] {};
+
+ scheduler = new Scheduler(ncNameToNcInfos);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ scheduler = new Scheduler(ncNameToNcInfos, topology);
+ locationConstraints = scheduler.getLocationConstraints(fileSplits);
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
new file mode 100644
index 0000000..8c12518
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
@@ -0,0 +1,106 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.utils;
+
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class HyracksUtils {
+
+ public static final String NC1_ID = "nc1";
+ public static final String NC2_ID = "nc2";
+
+ public static final int DEFAULT_HYRACKS_CC_PORT = 1099;
+ public static final int TEST_HYRACKS_CC_PORT = 1099;
+ public static final int TEST_HYRACKS_CC_CLIENT_PORT = 2099;
+ public static final String CC_HOST = "localhost";
+
+ public static final int FRAME_SIZE = 65536;
+
+ private static ClusterControllerService cc;
+ private static NodeControllerService nc1;
+ private static NodeControllerService nc2;
+ private static IHyracksClientConnection hcc;
+
+ public static void init() throws Exception {
+ CCConfig ccConfig = new CCConfig();
+ ccConfig.clientNetIpAddress = CC_HOST;
+ ccConfig.clusterNetIpAddress = CC_HOST;
+ ccConfig.clusterNetPort = TEST_HYRACKS_CC_PORT;
+ ccConfig.clientNetPort = TEST_HYRACKS_CC_CLIENT_PORT;
+ ccConfig.defaultMaxJobAttempts = 0;
+ ccConfig.jobHistorySize = 0;
+ ccConfig.profileDumpPeriod = -1;
+
+ // cluster controller
+ cc = new ClusterControllerService(ccConfig);
+ cc.start();
+
+ // two node controllers
+ NCConfig ncConfig1 = new NCConfig();
+ ncConfig1.ccHost = "localhost";
+ ncConfig1.clusterNetIPAddress = "localhost";
+ ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
+ ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
+ ncConfig1.nodeId = NC1_ID;
+ nc1 = new NodeControllerService(ncConfig1);
+ nc1.start();
+
+ NCConfig ncConfig2 = new NCConfig();
+ ncConfig2.ccHost = "localhost";
+ ncConfig2.clusterNetIPAddress = "localhost";
+ ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
+ ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
+ ncConfig2.nodeId = NC2_ID;
+ nc2 = new NodeControllerService(ncConfig2);
+ nc2.start();
+
+ // hyracks connection
+ hcc = new HyracksConnection(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
+ }
+
+ public static void destroyApp(String hyracksAppName) throws Exception {
+ hcc.destroyApplication(hyracksAppName);
+ }
+
+ public static void createApp(String hyracksAppName) throws Exception {
+ hcc.createApplication(hyracksAppName, null);
+ }
+
+ public static void deinit() throws Exception {
+ nc2.stop();
+ nc1.stop();
+ cc.stop();
+ }
+
+ public static void runJob(JobSpecification spec, String appName) throws Exception {
+ spec.setFrameSize(FRAME_SIZE);
+ JobId jobId = hcc.startJob(appName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ hcc.waitForCompletion(jobId);
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
new file mode 100644
index 0000000..3826688
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/TestUtils.java
@@ -0,0 +1,93 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs.utils;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+
+public class TestUtils {
+
+ public static void compareWithResult(File expectedFile, File actualFile) throws Exception {
+ BufferedReader readerExpected = new BufferedReader(new FileReader(expectedFile));
+ BufferedReader readerActual = new BufferedReader(new FileReader(actualFile));
+ String lineExpected, lineActual;
+ int num = 1;
+ try {
+ while ((lineExpected = readerExpected.readLine()) != null) {
+ lineActual = readerActual.readLine();
+ // Assert.assertEquals(lineExpected, lineActual);
+ if (lineActual == null) {
+ throw new Exception("Actual result changed at line " + num + ":\n< " + lineExpected + "\n> ");
+ }
+ if (!equalStrings(lineExpected, lineActual)) {
+ throw new Exception("Result for changed at line " + num + ":\n< " + lineExpected + "\n> "
+ + lineActual);
+ }
+ ++num;
+ }
+ lineActual = readerActual.readLine();
+ if (lineActual != null) {
+ throw new Exception("Actual result changed at line " + num + ":\n< \n> " + lineActual);
+ }
+ } finally {
+ readerExpected.close();
+ readerActual.close();
+ }
+ }
+
+ private static boolean equalStrings(String s1, String s2) {
+ String[] rowsOne = s1.split("\n");
+ String[] rowsTwo = s2.split("\n");
+
+ if (rowsOne.length != rowsTwo.length)
+ return false;
+
+ for (int i = 0; i < rowsOne.length; i++) {
+ String row1 = rowsOne[i];
+ String row2 = rowsTwo[i];
+
+ if (row1.equals(row2))
+ continue;
+
+ String[] fields1 = row1.split(",");
+ String[] fields2 = row2.split(",");
+
+ for (int j = 0; j < fields1.length; j++) {
+ if (fields1[j].equals(fields2[j])) {
+ continue;
+ } else if (fields1[j].indexOf('.') < 0) {
+ return false;
+ } else {
+ fields1[j] = fields1[j].split("=")[1];
+ fields2[j] = fields2[j].split("=")[1];
+ Double double1 = Double.parseDouble(fields1[j]);
+ Double double2 = Double.parseDouble(fields2[j]);
+ float float1 = (float) double1.doubleValue();
+ float float2 = (float) double2.doubleValue();
+
+ if (Math.abs(float1 - float2) == 0)
+ continue;
+ else {
+ return false;
+ }
+ }
+ }
+ }
+ return true;
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
new file mode 100644
index 0000000..9f77979
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/dataflow/DataflowTest.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.dataflow;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.hdfs.MiniDFSClusterFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryComparatorFactory;
+import edu.uci.ics.hyracks.hdfs.lib.RawBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextKeyValueParserFactory;
+import edu.uci.ics.hyracks.hdfs.lib.TextTupleWriterFactory;
+import edu.uci.ics.hyracks.hdfs.utils.HyracksUtils;
+import edu.uci.ics.hyracks.hdfs.utils.TestUtils;
+import edu.uci.ics.hyracks.hdfs2.scheduler.Scheduler;
+
+/**
+ * Test the edu.uci.ics.hyracks.hdfs2.dataflow package,
+ * the operators for the Hadoop new API.
+ */
+public class DataflowTest extends TestCase {
+
+ private static final String ACTUAL_RESULT_DIR = "actual";
+ private static final String EXPECTED_RESULT_PATH = "src/test/resources/expected";
+ private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+
+ private static final String DATA_PATH = "src/test/resources/data/customer.tbl";
+ private static final String HDFS_INPUT_PATH = "/customer/";
+ private static final String HDFS_OUTPUT_PATH = "/customer_result/";
+
+ private static final String HYRACKS_APP_NAME = "DataflowTest";
+ private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+ private MiniDFSCluster dfsCluster;
+ private MiniDFSClusterFactory dfsClusterFactory = new MiniDFSClusterFactory();
+
+ private Job conf;
+ private int numberOfNC = 2;
+
+ @Override
+ public void setUp() throws Exception {
+ conf = new Job();
+ cleanupStores();
+ HyracksUtils.init();
+ HyracksUtils.createApp(HYRACKS_APP_NAME);
+ FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+ FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+ startHDFS();
+ }
+
+ private void cleanupStores() throws IOException {
+ FileUtils.forceMkdir(new File("teststore"));
+ FileUtils.forceMkdir(new File("build"));
+ FileUtils.cleanDirectory(new File("teststore"));
+ FileUtils.cleanDirectory(new File("build"));
+ }
+
+ /**
+ * Start the HDFS cluster and setup the data files
+ *
+ * @throws IOException
+ */
+ private void startHDFS() throws IOException {
+ conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+ conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+ conf.getConfiguration().addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+
+ FileSystem lfs = FileSystem.getLocal(new Configuration());
+ lfs.delete(new Path("build"), true);
+ System.setProperty("hadoop.log.dir", "logs");
+ dfsCluster = dfsClusterFactory.getMiniDFSCluster(conf.getConfiguration(), numberOfNC);
+ FileSystem dfs = FileSystem.get(conf.getConfiguration());
+ Path src = new Path(DATA_PATH);
+ Path dest = new Path(HDFS_INPUT_PATH);
+ Path result = new Path(HDFS_OUTPUT_PATH);
+ dfs.mkdirs(dest);
+ dfs.mkdirs(result);
+ dfs.copyFromLocalFile(src, dest);
+
+ DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+ conf.getConfiguration().writeXml(confOutput);
+ confOutput.flush();
+ confOutput.close();
+ }
+
+ /**
+ * Test a job with only HDFS read and writes.
+ *
+ * @throws Exception
+ */
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ public void testHDFSReadWriteOperators() throws Exception {
+ FileInputFormat.setInputPaths(conf, HDFS_INPUT_PATH);
+ FileOutputFormat.setOutputPath(conf, new Path(HDFS_OUTPUT_PATH));
+ conf.setInputFormatClass(TextInputFormat.class);
+
+ Scheduler scheduler = new Scheduler(HyracksUtils.CC_HOST, HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+ InputFormat inputFormat = ReflectionUtils.newInstance(conf.getInputFormatClass(), conf.getConfiguration());
+ List<InputSplit> splits = inputFormat.getSplits(conf);
+
+ String[] readSchedule = scheduler.getLocationConstraints(splits);
+ JobSpecification jobSpec = new JobSpecification();
+ RecordDescriptor recordDesc = new RecordDescriptor(
+ new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
+
+ String[] locations = new String[] { HyracksUtils.NC1_ID, HyracksUtils.NC1_ID, HyracksUtils.NC2_ID,
+ HyracksUtils.NC2_ID };
+ HDFSReadOperatorDescriptor readOperator = new HDFSReadOperatorDescriptor(jobSpec, recordDesc, conf, splits,
+ readSchedule, new TextKeyValueParserFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, readOperator, locations);
+
+ ExternalSortOperatorDescriptor sortOperator = new ExternalSortOperatorDescriptor(jobSpec, 10, new int[] { 0 },
+ new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }, recordDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, sortOperator, locations);
+
+ HDFSWriteOperatorDescriptor writeOperator = new HDFSWriteOperatorDescriptor(jobSpec, conf,
+ new TextTupleWriterFactory());
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(jobSpec, writeOperator, HyracksUtils.NC1_ID);
+
+ jobSpec.connect(new OneToOneConnectorDescriptor(jobSpec), readOperator, 0, sortOperator, 0);
+ jobSpec.connect(new MToNPartitioningMergingConnectorDescriptor(jobSpec, new FieldHashPartitionComputerFactory(
+ new int[] { 0 }, new IBinaryHashFunctionFactory[] { RawBinaryHashFunctionFactory.INSTANCE }),
+ new int[] { 0 }, new IBinaryComparatorFactory[] { RawBinaryComparatorFactory.INSTANCE }), sortOperator,
+ 0, writeOperator, 0);
+ jobSpec.addRoot(writeOperator);
+
+ IHyracksClientConnection client = new HyracksConnection(HyracksUtils.CC_HOST,
+ HyracksUtils.TEST_HYRACKS_CC_CLIENT_PORT);
+ JobId jobId = client.startJob(HYRACKS_APP_NAME, jobSpec);
+ client.waitForCompletion(jobId);
+
+ Assert.assertEquals(true, checkResults());
+ }
+
+ /**
+ * Check if the results are correct
+ *
+ * @return true if correct
+ * @throws Exception
+ */
+ private boolean checkResults() throws Exception {
+ FileSystem dfs = FileSystem.get(conf.getConfiguration());
+ Path result = new Path(HDFS_OUTPUT_PATH);
+ Path actual = new Path(ACTUAL_RESULT_DIR);
+ dfs.copyToLocalFile(result, actual);
+
+ TestUtils.compareWithResult(new File(EXPECTED_RESULT_PATH + File.separator + "part-0"), new File(
+ ACTUAL_RESULT_DIR + File.separator + "customer_result" + File.separator + "part-0"));
+ return true;
+ }
+
+ /**
+ * cleanup hdfs cluster
+ */
+ private void cleanupHDFS() throws Exception {
+ dfsCluster.shutdown();
+ }
+
+ @Override
+ public void tearDown() throws Exception {
+ HyracksUtils.destroyApp(HYRACKS_APP_NAME);
+ HyracksUtils.deinit();
+ cleanupHDFS();
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
new file mode 100644
index 0000000..442aeae0
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
@@ -0,0 +1,238 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.hdfs2.scheduler;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.Assert;
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.client.NodeStatus;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+
+/**
+ * Test case for the new HDFS API scheduler
+ */
+public class SchedulerTest extends TestCase {
+
+ /**
+ * Test the scheduler for the case when the Hyracks cluster is the HDFS cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSimple() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+ fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc2", "nc3", "nc5" };
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerLargerHDFS() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+ fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.6" }));
+ fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.7" }));
+ fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc6", "nc1", "nc4", "nc2", "nc2", "nc3", "nc6", "nc5",
+ "nc3", "nc5" };
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSmallerHDFS() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+ fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+ fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+ fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc6",
+ "nc5", "nc6" };
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+ /**
+ * Test the case where the HDFS cluster is a larger than the Hyracks cluster
+ *
+ * @throws Exception
+ */
+ public void testSchedulerSmallerHDFSOdd() throws Exception {
+ Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
+ ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
+ ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
+
+ List<InputSplit> fileSplits = new ArrayList<InputSplit>();
+ fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-2"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-3"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-4"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-5"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-6"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-7"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
+ fileSplits.add(new FileSplit(new Path("part-8"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-9"), 0, 0, new String[] { "10.0.0.4", "10.0.0.5", "10.0.0.1" }));
+ fileSplits.add(new FileSplit(new Path("part-10"), 0, 0, new String[] { "10.0.0.2", "10.0.0.1", "10.0.0.2" }));
+ fileSplits.add(new FileSplit(new Path("part-11"), 0, 0, new String[] { "10.0.0.3", "10.0.0.4", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-12"), 0, 0, new String[] { "10.0.0.2", "10.0.0.3", "10.0.0.5" }));
+ fileSplits.add(new FileSplit(new Path("part-13"), 0, 0, new String[] { "10.0.0.2", "10.0.0.4", "10.0.0.5" }));
+
+ Scheduler scheduler = new Scheduler(ncNameToNcInfos);
+ String[] locationConstraints = scheduler.getLocationConstraints(fileSplits);
+
+ String[] expectedResults = new String[] { "nc1", "nc4", "nc4", "nc1", "nc3", "nc2", "nc2", "nc3", "nc5", "nc1",
+ "nc5", "nc2", "nc4" };
+
+ for (int i = 0; i < locationConstraints.length; i++) {
+ Assert.assertEquals(locationConstraints[i], expectedResults[i]);
+ }
+ }
+
+}
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl
new file mode 100644
index 0000000..5d39c80
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/data/customer.tbl
@@ -0,0 +1,150 @@
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0 b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0
new file mode 100755
index 0000000..ce3b00c
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/expected/part-0
@@ -0,0 +1,150 @@
+100|Customer#000000100|fptUABXcmkC5Wx|20|30-749-445-4907|9889.89|FURNITURE|was furiously fluffily quiet deposits. silent, pending requests boost against |
+101|Customer#000000101|sMmL2rNeHDltovSm Y|2|12-514-298-3699|7470.96|MACHINERY| sleep. pending packages detect slyly ironic pack|
+102|Customer#000000102|UAtflJ06 fn9zBfKjInkQZlWtqaA|19|29-324-978-8538|8462.17|BUILDING|ously regular dependencies nag among the furiously express dinos. blithely final|
+103|Customer#000000103|8KIsQX4LJ7QMsj6DrtFtXu0nUEdV,8a|9|19-216-107-2107|2757.45|BUILDING|furiously pending notornis boost slyly around the blithely ironic ideas? final, even instructions cajole fl|
+104|Customer#000000104|9mcCK L7rt0SwiYtrbO88DiZS7U d7M|10|20-966-284-8065|-588.38|FURNITURE|rate carefully slyly special pla|
+105|Customer#000000105|4iSJe4L SPjg7kJj98Yz3z0B|10|20-793-553-6417|9091.82|MACHINERY|l pains cajole even accounts. quietly final instructi|
+106|Customer#000000106|xGCOEAUjUNG|1|11-751-989-4627|3288.42|MACHINERY|lose slyly. ironic accounts along the evenly regular theodolites wake about the special, final gifts. |
+107|Customer#000000107|Zwg64UZ,q7GRqo3zm7P1tZIRshBDz|15|25-336-529-9919|2514.15|AUTOMOBILE|counts cajole slyly. regular requests wake. furiously regular deposits about the blithely final fo|
+108|Customer#000000108|GPoeEvpKo1|5|15-908-619-7526|2259.38|BUILDING|refully ironic deposits sleep. regular, unusual requests wake slyly|
+109|Customer#000000109|OOOkYBgCMzgMQXUmkocoLb56rfrdWp2NE2c|16|26-992-422-8153|-716.10|BUILDING|es. fluffily final dependencies sleep along the blithely even pinto beans. final deposits haggle furiously furiou|
+10|Customer#000000010|6LrEaV6KR6PLVcgl2ArL Q3rqzLzcT1 v2|5|15-741-346-9870|2753.54|HOUSEHOLD|es regular deposits haggle. fur|
+110|Customer#000000110|mymPfgphaYXNYtk|10|20-893-536-2069|7462.99|AUTOMOBILE|nto beans cajole around the even, final deposits. quickly bold packages according to the furiously regular dept|
+111|Customer#000000111|CBSbPyOWRorloj2TBvrK9qp9tHBs|22|32-582-283-7528|6505.26|MACHINERY|ly unusual instructions detect fluffily special deposits-- theodolites nag carefully during the ironic dependencies|
+112|Customer#000000112|RcfgG3bO7QeCnfjqJT1|19|29-233-262-8382|2953.35|FURNITURE|rmanently unusual multipliers. blithely ruthless deposits are furiously along the|
+113|Customer#000000113|eaOl5UBXIvdY57rglaIzqvfPD,MYfK|12|22-302-930-4756|2912.00|BUILDING|usly regular theodolites boost furiously doggedly pending instructio|
+114|Customer#000000114|xAt 5f5AlFIU|14|24-805-212-7646|1027.46|FURNITURE|der the carefully express theodolites are after the packages. packages are. bli|
+115|Customer#000000115|0WFt1IXENmUT2BgbsB0ShVKJZt0HCBCbFl0aHc|8|18-971-699-1843|7508.92|HOUSEHOLD|sits haggle above the carefully ironic theodolite|
+116|Customer#000000116|yCuVxIgsZ3,qyK2rloThy3u|16|26-632-309-5792|8403.99|BUILDING|as. quickly final sauternes haggle slyly carefully even packages. brave, ironic pinto beans are above the furious|
+117|Customer#000000117|uNhM,PzsRA3S,5Y Ge5Npuhi|24|34-403-631-3505|3950.83|FURNITURE|affix. instructions are furiously sl|
+118|Customer#000000118|OVnFuHygK9wx3xpg8|18|28-639-943-7051|3582.37|AUTOMOBILE|uick packages alongside of the furiously final deposits haggle above the fluffily even foxes. blithely dogged dep|
+119|Customer#000000119|M1ETOIecuvH8DtM0Y0nryXfW|7|17-697-919-8406|3930.35|FURNITURE|express ideas. blithely ironic foxes thrash. special acco|
+11|Customer#000000011|PkWS 3HlXqwTuzrKg633BEi|23|33-464-151-3439|-272.60|BUILDING|ckages. requests sleep slyly. quickly even pinto beans promise above the slyly regular pinto beans. |
+120|Customer#000000120|zBNna00AEInqyO1|12|22-291-534-1571|363.75|MACHINERY| quickly. slyly ironic requests cajole blithely furiously final dependen|
+121|Customer#000000121|tv nCR2YKupGN73mQudO|17|27-411-990-2959|6428.32|BUILDING|uriously stealthy ideas. carefully final courts use carefully|
+122|Customer#000000122|yp5slqoNd26lAENZW3a67wSfXA6hTF|3|13-702-694-4520|7865.46|HOUSEHOLD| the special packages hinder blithely around the permanent requests. bold depos|
+123|Customer#000000123|YsOnaaER8MkvK5cpf4VSlq|5|15-817-151-1168|5897.83|BUILDING|ependencies. regular, ironic requests are fluffily regu|
+124|Customer#000000124|aTbyVAW5tCd,v09O|18|28-183-750-7809|1842.49|AUTOMOBILE|le fluffily even dependencies. quietly s|
+125|Customer#000000125|,wSZXdVR xxIIfm9s8ITyLl3kgjT6UC07GY0Y|19|29-261-996-3120|-234.12|FURNITURE|x-ray finally after the packages? regular requests c|
+126|Customer#000000126|ha4EHmbx3kg DYCsP6DFeUOmavtQlHhcfaqr|22|32-755-914-7592|1001.39|HOUSEHOLD|s about the even instructions boost carefully furiously ironic pearls. ruthless, |
+127|Customer#000000127|Xyge4DX2rXKxXyye1Z47LeLVEYMLf4Bfcj|21|31-101-672-2951|9280.71|MACHINERY|ic, unusual theodolites nod silently after the final, ironic instructions: pending r|
+128|Customer#000000128|AmKUMlJf2NRHcKGmKjLS|4|14-280-874-8044|-986.96|HOUSEHOLD|ing packages integrate across the slyly unusual dugouts. blithely silent ideas sublate carefully. blithely expr|
+129|Customer#000000129|q7m7rbMM0BpaCdmxloCgBDRCleXsXkdD8kf|7|17-415-148-7416|9127.27|HOUSEHOLD| unusual deposits boost carefully furiously silent ideas. pending accounts cajole slyly across|
+12|Customer#000000012|9PWKuhzT4Zr1Q|13|23-791-276-1263|3396.49|HOUSEHOLD| to the carefully final braids. blithely regular requests nag. ironic theodolites boost quickly along|
+130|Customer#000000130|RKPx2OfZy0Vn 8wGWZ7F2EAvmMORl1k8iH|9|19-190-993-9281|5073.58|HOUSEHOLD|ix slowly. express packages along the furiously ironic requests integrate daringly deposits. fur|
+131|Customer#000000131|jyN6lAjb1FtH10rMC,XzlWyCBrg75|11|21-840-210-3572|8595.53|HOUSEHOLD|jole special packages. furiously final dependencies about the furiously speci|
+132|Customer#000000132|QM5YabAsTLp9|4|14-692-150-9717|162.57|HOUSEHOLD|uickly carefully special theodolites. carefully regular requests against the blithely unusual instructions |
+133|Customer#000000133|IMCuXdpIvdkYO92kgDGuyHgojcUs88p|17|27-408-997-8430|2314.67|AUTOMOBILE|t packages. express pinto beans are blithely along the unusual, even theodolites. silent packages use fu|
+134|Customer#000000134|sUiZ78QCkTQPICKpA9OBzkUp2FM|11|21-200-159-5932|4608.90|BUILDING|yly fluffy foxes boost final ideas. b|
+135|Customer#000000135|oZK,oC0 fdEpqUML|19|29-399-293-6241|8732.91|FURNITURE| the slyly final accounts. deposits cajole carefully. carefully sly packag|
+136|Customer#000000136|QoLsJ0v5C1IQbh,DS1|7|17-501-210-4726|-842.39|FURNITURE|ackages sleep ironic, final courts. even requests above the blithely bold requests g|
+137|Customer#000000137|cdW91p92rlAEHgJafqYyxf1Q|16|26-777-409-5654|7838.30|HOUSEHOLD|carefully regular theodolites use. silent dolphins cajo|
+138|Customer#000000138|5uyLAeY7HIGZqtu66Yn08f|5|15-394-860-4589|430.59|MACHINERY|ts doze on the busy ideas. regular|
+139|Customer#000000139|3ElvBwudHKL02732YexGVFVt |9|19-140-352-1403|7897.78|MACHINERY|nstructions. quickly ironic ideas are carefully. bold, |
+13|Customer#000000013|nsXQu0oVjD7PM659uC3SRSp|3|13-761-547-5974|3857.34|BUILDING|ounts sleep carefully after the close frays. carefully bold notornis use ironic requests. blithely|
+140|Customer#000000140|XRqEPiKgcETII,iOLDZp5jA|4|14-273-885-6505|9963.15|MACHINERY|ies detect slyly ironic accounts. slyly ironic theodolites hag|
+141|Customer#000000141|5IW,WROVnikc3l7DwiUDGQNGsLBGOL6Dc0|1|11-936-295-6204|6706.14|FURNITURE|packages nag furiously. carefully unusual accounts snooze according to the fluffily regular pinto beans. slyly spec|
+142|Customer#000000142|AnJ5lxtLjioClr2khl9pb8NLxG2,|9|19-407-425-2584|2209.81|AUTOMOBILE|. even, express theodolites upo|
+143|Customer#000000143|681r22uL452zqk 8By7I9o9enQfx0|16|26-314-406-7725|2186.50|MACHINERY|across the blithely unusual requests haggle theodo|
+144|Customer#000000144|VxYZ3ebhgbltnetaGjNC8qCccjYU05 fePLOno8y|1|11-717-379-4478|6417.31|MACHINERY|ges. slyly regular accounts are slyly. bold, idle reque|
+145|Customer#000000145|kQjHmt2kcec cy3hfMh969u|13|23-562-444-8454|9748.93|HOUSEHOLD|ests? express, express instructions use. blithely fina|
+146|Customer#000000146|GdxkdXG9u7iyI1,,y5tq4ZyrcEy|3|13-835-723-3223|3328.68|FURNITURE|ffily regular dinos are slyly unusual requests. slyly specia|
+147|Customer#000000147|6VvIwbVdmcsMzuu,C84GtBWPaipGfi7DV|18|28-803-187-4335|8071.40|AUTOMOBILE|ress packages above the blithely regular packages sleep fluffily blithely ironic accounts. |
+148|Customer#000000148|BhSPlEWGvIJyT9swk vCWE|11|21-562-498-6636|2135.60|HOUSEHOLD|ing to the carefully ironic requests. carefully regular dependencies about the theodolites wake furious|
+149|Customer#000000149|3byTHCp2mNLPigUrrq|19|29-797-439-6760|8959.65|AUTOMOBILE|al instructions haggle against the slyly bold w|
+14|Customer#000000014|KXkletMlL2JQEA |1|11-845-129-3851|5266.30|FURNITURE|, ironic packages across the unus|
+150|Customer#000000150|zeoGShTjCwGPplOWFkLURrh41O0AZ8dwNEEN4 |18|28-328-564-7630|3849.48|MACHINERY|ole blithely among the furiously pending packages. furiously bold ideas wake fluffily ironic idea|
+15|Customer#000000015|YtWggXoOLdwdo7b0y,BZaGUQMLJMX1Y,EC,6Dn|23|33-687-542-7601|2788.52|HOUSEHOLD| platelets. regular deposits detect asymptotes. blithely unusual packages nag slyly at the fluf|
+16|Customer#000000016|cYiaeMLZSMAOQ2 d0W,|10|20-781-609-3107|4681.03|FURNITURE|kly silent courts. thinly regular theodolites sleep fluffily after |
+17|Customer#000000017|izrh 6jdqtp2eqdtbkswDD8SG4SzXruMfIXyR7|2|12-970-682-3487|6.34|AUTOMOBILE|packages wake! blithely even pint|
+18|Customer#000000018|3txGO AiuFux3zT0Z9NYaFRnZt|6|16-155-215-1315|5494.43|BUILDING|s sleep. carefully even instructions nag furiously alongside of t|
+19|Customer#000000019|uc,3bHIx84H,wdrmLOjVsiqXCq2tr|18|28-396-526-5053|8914.71|HOUSEHOLD| nag. furiously careful packages are slyly at the accounts. furiously regular in|
+1|Customer#000000001|IVhzIApeRb ot,c,E|15|25-989-741-2988|711.56|BUILDING|to the even, regular platelets. regular, ironic epitaphs nag e|
+20|Customer#000000020|JrPk8Pqplj4Ne|22|32-957-234-8742|7603.40|FURNITURE|g alongside of the special excuses-- fluffily enticing packages wake |
+21|Customer#000000021|XYmVpr9yAHDEn|8|18-902-614-8344|1428.25|MACHINERY| quickly final accounts integrate blithely furiously u|
+22|Customer#000000022|QI6p41,FNs5k7RZoCCVPUTkUdYpB|3|13-806-545-9701|591.98|MACHINERY|s nod furiously above the furiously ironic ideas. |
+23|Customer#000000023|OdY W13N7Be3OC5MpgfmcYss0Wn6TKT|3|13-312-472-8245|3332.02|HOUSEHOLD|deposits. special deposits cajole slyly. fluffily special deposits about the furiously |
+24|Customer#000000024|HXAFgIAyjxtdqwimt13Y3OZO 4xeLe7U8PqG|13|23-127-851-8031|9255.67|MACHINERY|into beans. fluffily final ideas haggle fluffily|
+25|Customer#000000025|Hp8GyFQgGHFYSilH5tBfe|12|22-603-468-3533|7133.70|FURNITURE|y. accounts sleep ruthlessly according to the regular theodolites. unusual instructions sleep. ironic, final|
+26|Customer#000000026|8ljrc5ZeMl7UciP|22|32-363-455-4837|5182.05|AUTOMOBILE|c requests use furiously ironic requests. slyly ironic dependencies us|
+27|Customer#000000027|IS8GIyxpBrLpMT0u7|3|13-137-193-2709|5679.84|BUILDING| about the carefully ironic pinto beans. accoun|
+28|Customer#000000028|iVyg0daQ,Tha8x2WPWA9m2529m|8|18-774-241-1462|1007.18|FURNITURE| along the regular deposits. furiously final pac|
+29|Customer#000000029|sJ5adtfyAkCK63df2,vF25zyQMVYE34uh|0|10-773-203-7342|7618.27|FURNITURE|its after the carefully final platelets x-ray against |
+2|Customer#000000002|XSTf4,NCwDVaWNe6tEgvwfmRchLXak|13|23-768-687-3665|121.65|AUTOMOBILE|l accounts. blithely ironic theodolites integrate boldly: caref|
+30|Customer#000000030|nJDsELGAavU63Jl0c5NKsKfL8rIJQQkQnYL2QJY|1|11-764-165-5076|9321.01|BUILDING|lithely final requests. furiously unusual account|
+31|Customer#000000031|LUACbO0viaAv6eXOAebryDB xjVst|23|33-197-837-7094|5236.89|HOUSEHOLD|s use among the blithely pending depo|
+32|Customer#000000032|jD2xZzi UmId,DCtNBLXKj9q0Tlp2iQ6ZcO3J|15|25-430-914-2194|3471.53|BUILDING|cial ideas. final, furious requests across the e|
+33|Customer#000000033|qFSlMuLucBmx9xnn5ib2csWUweg D|17|27-375-391-1280|-78.56|AUTOMOBILE|s. slyly regular accounts are furiously. carefully pending requests|
+34|Customer#000000034|Q6G9wZ6dnczmtOx509xgE,M2KV|15|25-344-968-5422|8589.70|HOUSEHOLD|nder against the even, pending accounts. even|
+35|Customer#000000035|TEjWGE4nBzJL2|17|27-566-888-7431|1228.24|HOUSEHOLD|requests. special, express requests nag slyly furiousl|
+36|Customer#000000036|3TvCzjuPzpJ0,DdJ8kW5U|21|31-704-669-5769|4987.27|BUILDING|haggle. enticing, quiet platelets grow quickly bold sheaves. carefully regular acc|
+37|Customer#000000037|7EV4Pwh,3SboctTWt|8|18-385-235-7162|-917.75|FURNITURE|ilent packages are carefully among the deposits. furiousl|
+38|Customer#000000038|a5Ee5e9568R8RLP 2ap7|12|22-306-880-7212|6345.11|HOUSEHOLD|lar excuses. closely even asymptotes cajole blithely excuses. carefully silent pinto beans sleep carefully fin|
+39|Customer#000000039|nnbRg,Pvy33dfkorYE FdeZ60|2|12-387-467-6509|6264.31|AUTOMOBILE|tions. slyly silent excuses slee|
+3|Customer#000000003|MG9kdTD2WBHm|1|11-719-748-3364|7498.12|AUTOMOBILE| deposits eat slyly ironic, even instructions. express foxes detect slyly. blithely even accounts abov|
+40|Customer#000000040|gOnGWAyhSV1ofv|3|13-652-915-8939|1335.30|BUILDING|rges impress after the slyly ironic courts. foxes are. blithely |
+41|Customer#000000041|IM9mzmyoxeBmvNw8lA7G3Ydska2nkZF|10|20-917-711-4011|270.95|HOUSEHOLD|ly regular accounts hang bold, silent packages. unusual foxes haggle slyly above the special, final depo|
+42|Customer#000000042|ziSrvyyBke|5|15-416-330-4175|8727.01|BUILDING|ssly according to the pinto beans: carefully special requests across the even, pending accounts wake special|
+43|Customer#000000043|ouSbjHk8lh5fKX3zGso3ZSIj9Aa3PoaFd|19|29-316-665-2897|9904.28|MACHINERY|ial requests: carefully pending foxes detect quickly. carefully final courts cajole quickly. carefully|
+44|Customer#000000044|Oi,dOSPwDu4jo4x,,P85E0dmhZGvNtBwi|16|26-190-260-5375|7315.94|AUTOMOBILE|r requests around the unusual, bold a|
+45|Customer#000000045|4v3OcpFgoOmMG,CbnF,4mdC|9|19-715-298-9917|9983.38|AUTOMOBILE|nto beans haggle slyly alongside of t|
+46|Customer#000000046|eaTXWWm10L9|6|16-357-681-2007|5744.59|AUTOMOBILE|ctions. accounts sleep furiously even requests. regular, regular accounts cajole blithely around the final pa|
+47|Customer#000000047|b0UgocSqEW5 gdVbhNT|2|12-427-271-9466|274.58|BUILDING|ions. express, ironic instructions sleep furiously ironic ideas. furi|
+48|Customer#000000048|0UU iPhBupFvemNB|0|10-508-348-5882|3792.50|BUILDING|re fluffily pending foxes. pending, bold platelets sleep slyly. even platelets cajo|
+49|Customer#000000049|cNgAeX7Fqrdf7HQN9EwjUa4nxT,68L FKAxzl|10|20-908-631-4424|4573.94|FURNITURE|nusual foxes! fluffily pending packages maintain to the regular |
+4|Customer#000000004|XxVSJsLAGtn|4|14-128-190-5944|2866.83|MACHINERY| requests. final, regular ideas sleep final accou|
+50|Customer#000000050|9SzDYlkzxByyJ1QeTI o|6|16-658-112-3221|4266.13|MACHINERY|ts. furiously ironic accounts cajole furiously slyly ironic dinos.|
+51|Customer#000000051|uR,wEaiTvo4|12|22-344-885-4251|855.87|FURNITURE|eposits. furiously regular requests integrate carefully packages. furious|
+52|Customer#000000052|7 QOqGqqSy9jfV51BC71jcHJSD0|11|21-186-284-5998|5630.28|HOUSEHOLD|ic platelets use evenly even accounts. stealthy theodolites cajole furiou|
+53|Customer#000000053|HnaxHzTfFTZs8MuCpJyTbZ47Cm4wFOOgib|15|25-168-852-5363|4113.64|HOUSEHOLD|ar accounts are. even foxes are blithely. fluffily pending deposits boost|
+54|Customer#000000054|,k4vf 5vECGWFy,hosTE,|4|14-776-370-4745|868.90|AUTOMOBILE|sual, silent accounts. furiously express accounts cajole special deposits. final, final accounts use furi|
+55|Customer#000000055|zIRBR4KNEl HzaiV3a i9n6elrxzDEh8r8pDom|10|20-180-440-8525|4572.11|MACHINERY|ully unusual packages wake bravely bold packages. unusual requests boost deposits! blithely ironic packages ab|
+56|Customer#000000056|BJYZYJQk4yD5B|10|20-895-685-6920|6530.86|FURNITURE|. notornis wake carefully. carefully fluffy requests are furiously even accounts. slyly expre|
+57|Customer#000000057|97XYbsuOPRXPWU|21|31-835-306-1650|4151.93|AUTOMOBILE|ove the carefully special packages. even, unusual deposits sleep slyly pend|
+58|Customer#000000058|g9ap7Dk1Sv9fcXEWjpMYpBZIRUohi T|13|23-244-493-2508|6478.46|HOUSEHOLD|ideas. ironic ideas affix furiously express, final instructions. regular excuses use quickly e|
+59|Customer#000000059|zLOCP0wh92OtBihgspOGl4|1|11-355-584-3112|3458.60|MACHINERY|ously final packages haggle blithely after the express deposits. furiou|
+5|Customer#000000005|KvpyuHCplrB84WgAiGV6sYpZq7Tj|3|13-750-942-6364|794.47|HOUSEHOLD|n accounts will have to unwind. foxes cajole accor|
+60|Customer#000000060|FyodhjwMChsZmUz7Jz0H|12|22-480-575-5866|2741.87|MACHINERY|latelets. blithely unusual courts boost furiously about the packages. blithely final instruct|
+61|Customer#000000061|9kndve4EAJxhg3veF BfXr7AqOsT39o gtqjaYE|17|27-626-559-8599|1536.24|FURNITURE|egular packages shall have to impress along the |
+62|Customer#000000062|upJK2Dnw13,|7|17-361-978-7059|595.61|MACHINERY|kly special dolphins. pinto beans are slyly. quickly regular accounts are furiously a|
+63|Customer#000000063|IXRSpVWWZraKII|21|31-952-552-9584|9331.13|AUTOMOBILE|ithely even accounts detect slyly above the fluffily ir|
+64|Customer#000000064|MbCeGY20kaKK3oalJD,OT|3|13-558-731-7204|-646.64|BUILDING|structions after the quietly ironic theodolites cajole be|
+65|Customer#000000065|RGT yzQ0y4l0H90P783LG4U95bXQFDRXbWa1sl,X|23|33-733-623-5267|8795.16|AUTOMOBILE|y final foxes serve carefully. theodolites are carefully. pending i|
+66|Customer#000000066|XbsEqXH1ETbJYYtA1A|22|32-213-373-5094|242.77|HOUSEHOLD|le slyly accounts. carefully silent packages benea|
+67|Customer#000000067|rfG0cOgtr5W8 xILkwp9fpCS8|9|19-403-114-4356|8166.59|MACHINERY|indle furiously final, even theodo|
+68|Customer#000000068|o8AibcCRkXvQFh8hF,7o|12|22-918-832-2411|6853.37|HOUSEHOLD| pending pinto beans impress realms. final dependencies |
+69|Customer#000000069|Ltx17nO9Wwhtdbe9QZVxNgP98V7xW97uvSH1prEw|9|19-225-978-5670|1709.28|HOUSEHOLD|thely final ideas around the quickly final dependencies affix carefully quickly final theodolites. final accounts c|
+6|Customer#000000006|sKZz0CsnMD7mp4Xd0YrBvx,LREYKUWAh yVn|20|30-114-968-4951|7638.57|AUTOMOBILE|tions. even deposits boost according to the slyly bold packages. final accounts cajole requests. furious|
+70|Customer#000000070|mFowIuhnHjp2GjCiYYavkW kUwOjIaTCQ|22|32-828-107-2832|4867.52|FURNITURE|fter the special asymptotes. ideas after the unusual frets cajole quickly regular pinto be|
+71|Customer#000000071|TlGalgdXWBmMV,6agLyWYDyIz9MKzcY8gl,w6t1B|7|17-710-812-5403|-611.19|HOUSEHOLD|g courts across the regular, final pinto beans are blithely pending ac|
+72|Customer#000000072|putjlmskxE,zs,HqeIA9Wqu7dhgH5BVCwDwHHcf|2|12-759-144-9689|-362.86|FURNITURE|ithely final foxes sleep always quickly bold accounts. final wat|
+73|Customer#000000073|8IhIxreu4Ug6tt5mog4|0|10-473-439-3214|4288.50|BUILDING|usual, unusual packages sleep busily along the furiou|
+74|Customer#000000074|IkJHCA3ZThF7qL7VKcrU nRLl,kylf |4|14-199-862-7209|2764.43|MACHINERY|onic accounts. blithely slow packages would haggle carefully. qui|
+75|Customer#000000075|Dh 6jZ,cwxWLKQfRKkiGrzv6pm|18|28-247-803-9025|6684.10|AUTOMOBILE| instructions cajole even, even deposits. finally bold deposits use above the even pains. slyl|
+76|Customer#000000076|m3sbCvjMOHyaOofH,e UkGPtqc4|0|10-349-718-3044|5745.33|FURNITURE|pecial deposits. ironic ideas boost blithely according to the closely ironic theodolites! furiously final deposits n|
+77|Customer#000000077|4tAE5KdMFGD4byHtXF92vx|17|27-269-357-4674|1738.87|BUILDING|uffily silent requests. carefully ironic asymptotes among the ironic hockey players are carefully bli|
+78|Customer#000000078|HBOta,ZNqpg3U2cSL0kbrftkPwzX|9|19-960-700-9191|7136.97|FURNITURE|ests. blithely bold pinto beans h|
+79|Customer#000000079|n5hH2ftkVRwW8idtD,BmM2|15|25-147-850-4166|5121.28|MACHINERY|es. packages haggle furiously. regular, special requests poach after the quickly express ideas. blithely pending re|
+7|Customer#000000007|TcGe5gaZNgVePxU5kRrvXBfkasDTea|18|28-190-982-9759|9561.95|AUTOMOBILE|ainst the ironic, express theodolites. express, even pinto beans among the exp|
+80|Customer#000000080|K,vtXp8qYB |0|10-267-172-7101|7383.53|FURNITURE|tect among the dependencies. bold accounts engage closely even pinto beans. ca|
+81|Customer#000000081|SH6lPA7JiiNC6dNTrR|20|30-165-277-3269|2023.71|BUILDING|r packages. fluffily ironic requests cajole fluffily. ironically regular theodolit|
+82|Customer#000000082|zhG3EZbap4c992Gj3bK,3Ne,Xn|18|28-159-442-5305|9468.34|AUTOMOBILE|s wake. bravely regular accounts are furiously. regula|
+83|Customer#000000083|HnhTNB5xpnSF20JBH4Ycs6psVnkC3RDf|22|32-817-154-4122|6463.51|BUILDING|ccording to the quickly bold warhorses. final, regular foxes integrate carefully. bold packages nag blithely ev|
+84|Customer#000000084|lpXz6Fwr9945rnbtMc8PlueilS1WmASr CB|11|21-546-818-3802|5174.71|FURNITURE|ly blithe foxes. special asymptotes haggle blithely against the furiously regular depo|
+85|Customer#000000085|siRerlDwiolhYR 8FgksoezycLj|5|15-745-585-8219|3386.64|FURNITURE|ronic ideas use above the slowly pendin|
+86|Customer#000000086|US6EGGHXbTTXPL9SBsxQJsuvy|0|10-677-951-2353|3306.32|HOUSEHOLD|quests. pending dugouts are carefully aroun|
+87|Customer#000000087|hgGhHVSWQl 6jZ6Ev|23|33-869-884-7053|6327.54|FURNITURE|hely ironic requests integrate according to the ironic accounts. slyly regular pla|
+88|Customer#000000088|wtkjBN9eyrFuENSMmMFlJ3e7jE5KXcg|16|26-516-273-2566|8031.44|AUTOMOBILE|s are quickly above the quickly ironic instructions; even requests about the carefully final deposi|
+89|Customer#000000089|dtR, y9JQWUO6FoJExyp8whOU|14|24-394-451-5404|1530.76|FURNITURE|counts are slyly beyond the slyly final accounts. quickly final ideas wake. r|
+8|Customer#000000008|I0B10bB0AymmC, 0PrRYBCP1yGJ8xcBPmWhl5|17|27-147-574-9335|6819.74|BUILDING|among the slyly regular theodolites kindle blithely courts. carefully even theodolites haggle slyly along the ide|
+90|Customer#000000090|QxCzH7VxxYUWwfL7|16|26-603-491-1238|7354.23|BUILDING|sly across the furiously even |
+91|Customer#000000091|S8OMYFrpHwoNHaGBeuS6E 6zhHGZiprw1b7 q|8|18-239-400-3677|4643.14|AUTOMOBILE|onic accounts. fluffily silent pinto beans boost blithely according to the fluffily exp|
+92|Customer#000000092|obP PULk2LH LqNF,K9hcbNqnLAkJVsl5xqSrY,|2|12-446-416-8471|1182.91|MACHINERY|. pinto beans hang slyly final deposits. ac|
+93|Customer#000000093|EHXBr2QGdh|7|17-359-388-5266|2182.52|MACHINERY|press deposits. carefully regular platelets r|
+94|Customer#000000094|IfVNIN9KtkScJ9dUjK3Pg5gY1aFeaXewwf|9|19-953-499-8833|5500.11|HOUSEHOLD|latelets across the bold, final requests sleep according to the fluffily bold accounts. unusual deposits amon|
+95|Customer#000000095|EU0xvmWvOmUUn5J,2z85DQyG7QCJ9Xq7|15|25-923-255-2929|5327.38|MACHINERY|ithely. ruthlessly final requests wake slyly alongside of the furiously silent pinto beans. even the|
+96|Customer#000000096|vWLOrmXhRR|8|18-422-845-1202|6323.92|AUTOMOBILE|press requests believe furiously. carefully final instructions snooze carefully. |
+97|Customer#000000097|OApyejbhJG,0Iw3j rd1M|17|27-588-919-5638|2164.48|AUTOMOBILE|haggle slyly. bold, special ideas are blithely above the thinly bold theo|
+98|Customer#000000098|7yiheXNSpuEAwbswDW|12|22-885-845-6889|-551.37|BUILDING|ages. furiously pending accounts are quickly carefully final foxes: busily pe|
+99|Customer#000000099|szsrOiPtCHVS97Lt|15|25-515-237-9232|4088.65|HOUSEHOLD|cajole slyly about the regular theodolites! furiously bold requests nag along the pending, regular packages. somas|
+9|Customer#000000009|xKiAFTjUsCuxfeleNqefumTrjS|8|18-338-906-3675|8324.07|FURNITURE|r theodolites according to the requests wake thinly excuses: pending requests haggle furiousl|
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml
new file mode 100644
index 0000000..47dfac5
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/core-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+ <name>fs.default.name</name>
+ <value>hdfs://127.0.0.1:31888</value>
+</property>
+<property>
+ <name>hadoop.tmp.dir</name>
+ <value>/tmp/hadoop</value>
+</property>
+
+
+</configuration>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml
new file mode 100644
index 0000000..8d29b1d
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/hdfs-site.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+<property>
+ <name>dfs.replication</name>
+ <value>1</value>
+</property>
+
+<property>
+ <name>dfs.block.size</name>
+ <value>65536</value>
+</property>
+
+</configuration>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties
new file mode 100755
index 0000000..d5e6004
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/log4j.properties
@@ -0,0 +1,94 @@
+# Define some default values that can be overridden by system properties
+hadoop.root.logger=FATAL,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshhold=FATAL
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+#
+# TaskLog Appender
+#
+
+#Default values
+hadoop.tasklog.taskid=null
+hadoop.tasklog.noKeepSplits=4
+hadoop.tasklog.totalLogFileSize=100
+hadoop.tasklog.purgeLogSplits=true
+hadoop.tasklog.logsRetainHours=12
+
+log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
+log4j.appender.TLA.taskId=${hadoop.tasklog.taskid}
+log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize}
+
+log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+
+#
+# Rolling File Appender
+#
+
+#log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+#log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Logfile size and and 30-day backups
+#log4j.appender.RFA.MaxFileSize=1MB
+#log4j.appender.RFA.MaxBackupIndex=30
+
+#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+#
+# FSNamesystem Audit logging
+# All audit events are logged at INFO level
+#
+log4j.logger.org.apache.hadoop.fs.FSNamesystem.audit=WARN
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG
+#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+
+# Jets3t library
+log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.metrics.jvm.EventCounter
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml
new file mode 100644
index 0000000..39b6505
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/hadoop/conf/mapred-site.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+
+<!-- Put site-specific property overrides in this file. -->
+
+<configuration>
+
+ <property>
+ <name>mapred.job.tracker</name>
+ <value>localhost:29007</value>
+ </property>
+ <property>
+ <name>mapred.tasktracker.map.tasks.maximum</name>
+ <value>20</value>
+ </property>
+ <property>
+ <name>mapred.tasktracker.reduce.tasks.maximum</name>
+ <value>20</value>
+ </property>
+ <property>
+ <name>mapred.max.split.size</name>
+ <value>2048</value>
+ </property>
+
+</configuration>
diff --git a/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
new file mode 100644
index 0000000..3a0ac7e
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/resources/topology.xml
@@ -0,0 +1,32 @@
+<cluster-topology>
+ <network-switch name="all">
+ <network-switch name="rack1">
+ <terminal name="10.0.0.1" />
+ <terminal name="10.0.0.5" />
+ <terminal name="10.0.0.9" />
+ <terminal name="10.0.0.13" />
+ <terminal name="10.0.0.17" />
+ </network-switch>
+ <network-switch name="rack2">
+ <terminal name="10.0.0.2" />
+ <terminal name="10.0.0.6" />
+ <terminal name="10.0.0.10" />
+ <terminal name="10.0.0.14" />
+ <terminal name="10.0.0.18" />
+ </network-switch>
+ <network-switch name="rack3">
+ <terminal name="10.0.0.3" />
+ <terminal name="10.0.0.7" />
+ <terminal name="10.0.0.11" />
+ <terminal name="10.0.0.15" />
+ <terminal name="10.0.0.19" />
+ </network-switch>
+ <network-switch name="rack4">
+ <terminal name="10.0.0.4" />
+ <terminal name="10.0.0.8" />
+ <terminal name="10.0.0.12" />
+ <terminal name="10.0.0.16" />
+ <terminal name="10.0.0.20" />
+ </network-switch>
+ </network-switch>
+</cluster-topology>
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-hdfs/pom.xml b/fullstack/hyracks/hyracks-hdfs/pom.xml
new file mode 100644
index 0000000..5ed76e9
--- /dev/null
+++ b/fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <artifactId>hyracks-hdfs</artifactId>
+ <packaging>pom</packaging>
+ <name>hyracks-hdfs</name>
+
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <modules>
+ <module>hyracks-hdfs-0.20.2</module>
+ <module>hyracks-hdfs-0.23.1</module>
+ <module>hyracks-hdfs-core</module>
+ </modules>
+</project>
diff --git a/fullstack/hyracks/hyracks-ipc/pom.xml b/fullstack/hyracks/hyracks-ipc/pom.xml
index cb0de08..6f5e09f 100644
--- a/fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/fullstack/hyracks/hyracks-ipc/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml b/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
index 50c05da..a8fc29e 100644
--- a/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
+++ b/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
@@ -17,8 +17,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java b/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
index 47de024..fc06a68 100644
--- a/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
+++ b/fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
@@ -55,6 +55,7 @@
cmdLineBuffer.append(" -data-ip-address ").append(dataIpAddress);
cmdLineBuffer.append(" -node-id ").append(nodeId);
cmdLineBuffer.append(" -cluster-net-ip-address 127.0.0.1");
+ cmdLineBuffer.append(" -result-ip-address 127.0.0.1");
if (ccPort != 0) {
cmdLineBuffer.append(" -cc-port ").append(ccPort);
}
diff --git a/fullstack/hyracks/hyracks-net/pom.xml b/fullstack/hyracks/hyracks-net/pom.xml
index a079306..fb486df 100644
--- a/fullstack/hyracks/hyracks-net/pom.xml
+++ b/fullstack/hyracks/hyracks-net/pom.xml
@@ -15,8 +15,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
index dabc8a4..8fa99be 100644
--- a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
+++ b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/ChannelSet.java
@@ -58,6 +58,7 @@
ChannelControlBlock allocateChannel() throws NetException {
synchronized (mConn) {
+ cleanupClosedChannels();
int idx = allocationBitmap.nextClearBit(0);
if (idx < 0 || idx >= ccbArray.length) {
cleanupClosedChannels();
@@ -231,4 +232,4 @@
ccbArray = Arrays.copyOf(ccbArray, ccbArray.length * 2);
}
}
-}
\ No newline at end of file
+}
diff --git a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
index c719bc4..e4df6b9 100644
--- a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
+++ b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -48,9 +48,9 @@
* Constructor.
*
* @param localAddress
- * - TCP/IP socket address to listen on
+ * - TCP/IP socket address to listen on. Null for non-listening unidirectional sockets
* @param listener
- * - Callback interface to report channel events
+ * - Callback interface to report channel events. Null for non-listening unidirectional sockets
* @param nThreads
* - Number of threads to use for data transfer
* @param maxConnectionAttempts
diff --git a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
index d13a17e..a9061e1 100644
--- a/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/fullstack/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -45,15 +45,23 @@
}
public void start(InetSocketAddress localAddress) throws IOException {
- serverSocketChannel = ServerSocketChannel.open();
- ServerSocket serverSocket = serverSocketChannel.socket();
- serverSocket.bind(localAddress);
- this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+ // Setup a server socket listening channel only if the TCPEndpoint is a listening endpoint.
+ if (localAddress != null) {
+ serverSocketChannel = ServerSocketChannel.open();
+ ServerSocket serverSocket = serverSocketChannel.socket();
+ serverSocket.bind(localAddress);
+ this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+ }
+
ioThreads = new IOThread[nThreads];
for (int i = 0; i < ioThreads.length; ++i) {
ioThreads[i] = new IOThread();
}
- ioThreads[0].registerServerSocket(serverSocketChannel);
+
+ if (localAddress != null) {
+ ioThreads[0].registerServerSocket(serverSocketChannel);
+ }
+
for (int i = 0; i < ioThreads.length; ++i) {
ioThreads[i].start();
}
diff --git a/fullstack/hyracks/hyracks-server/pom.xml b/fullstack/hyracks/hyracks-server/pom.xml
index 2699c27..e0fc40a 100644
--- a/fullstack/hyracks/hyracks-server/pom.xml
+++ b/fullstack/hyracks/hyracks-server/pom.xml
@@ -15,13 +15,15 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>appassembler-maven-plugin</artifactId>
+ <version>1.3</version>
<executions>
<execution>
<configuration>
diff --git a/fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 118d46d..f251d51 100644
--- a/fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-storage-am-common/pom.xml b/fullstack/hyracks/hyracks-storage-am-common/pom.xml
index 0b32733..dbc4f41d 100644
--- a/fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/pom.xml b/fullstack/hyracks/hyracks-storage-am-invertedindex/pom.xml
index a647e9d..5fe2d96 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/pom.xml
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index d00bea6..6744f70 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -15,13 +15,13 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
-import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -41,7 +41,7 @@
private FrameTupleAccessor accessor;
private ArrayTupleBuilder builder;
- private DataOutput builderDos;
+ private GrowableArray builderFieldData;
private FrameTupleAppender appender;
private ByteBuffer writeBuffer;
@@ -60,7 +60,7 @@
accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
writeBuffer = ctx.allocateFrame();
builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
- builderDos = builder.getDataOutput();
+ builderFieldData = builder.getFieldData();
appender = new FrameTupleAppender(ctx.getFrameSize());
appender.reset(writeBuffer, true);
writer.open();
@@ -87,7 +87,7 @@
builder.reset();
try {
IToken token = tokenizer.getToken();
- token.serializeToken(builderDos);
+ token.serializeToken(builderFieldData);
builder.addFieldEndOffset();
} catch (IOException e) {
throw new HyracksDataException(e.getMessage());
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index 99e76dc..3525fc3 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -202,7 +202,9 @@
public void endBulkLoad(IIndexBulkLoadContext ictx) throws HyracksDataException {
// Create entry in btree for last inverted list.
InvertedIndexBulkLoadContext ctx = (InvertedIndexBulkLoadContext) ictx;
- createAndInsertBTreeTuple(ctx);
+ if (ctx.lastTuple.getFieldData(0) != null) {
+ createAndInsertBTreeTuple(ctx);
+ }
btree.endBulkLoad(ctx.btreeBulkLoadCtx);
ctx.deinit();
}
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index 1f886a2..af5dad3 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.impls;
-import java.io.DataOutput;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -28,6 +27,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
@@ -75,7 +75,7 @@
protected RecordDescriptor queryTokenRecDesc = new RecordDescriptor(
new ISerializerDeserializer[] { UTF8StringSerializerDeserializer.INSTANCE });
protected ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(queryTokenRecDesc.getFieldCount());
- protected DataOutput queryTokenDos = queryTokenBuilder.getDataOutput();
+ protected GrowableArray queryTokenFieldData = queryTokenBuilder.getFieldData();
protected FrameTupleAppender queryTokenAppender;
protected ByteBuffer queryTokenFrame;
@@ -158,7 +158,7 @@
queryTokenBuilder.reset();
try {
IToken token = queryTokenizer.getToken();
- token.serializeToken(queryTokenDos);
+ token.serializeToken(queryTokenFieldData);
queryTokenBuilder.addFieldEndOffset();
// WARNING: assuming one frame is big enough to hold all tokens
queryTokenAppender.append(queryTokenBuilder.getFieldEndOffsets(), queryTokenBuilder.getByteArray(), 0,
@@ -294,7 +294,8 @@
boolean advanceCursor = true;
boolean advancePrevResult = false;
int resultTidx = 0;
-
+ currentNumResults = 0;
+
resultFrameTupleAcc.reset(prevCurrentBuffer);
resultFrameTupleApp.reset(newCurrentBuffer, true);
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
index 65afa65..2f60952 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/AbstractUTF8Token.java
@@ -22,6 +22,7 @@
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
public abstract class AbstractUTF8Token implements IToken {
public static final int GOLDEN_RATIO_32 = 0x09e3779b9;
@@ -97,8 +98,8 @@
}
@Override
- public void serializeTokenCount(DataOutput dos) throws IOException {
- handleCountTypeTag(dos);
- dos.writeInt(tokenCount);
+ public void serializeTokenCount(GrowableArray out) throws IOException {
+ handleCountTypeTag(out.getDataOutput());
+ out.getDataOutput().writeInt(tokenCount);
}
}
\ No newline at end of file
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
index b7bb828..a1a4354 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8NGramToken.java
@@ -19,10 +19,10 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import java.io.DataOutput;
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
public class HashedUTF8NGramToken extends UTF8NGramToken {
public HashedUTF8NGramToken(byte tokenTypeTag, byte countTypeTag) {
@@ -30,8 +30,8 @@
}
@Override
- public void serializeToken(DataOutput dos) throws IOException {
- handleTokenTypeTag(dos);
+ public void serializeToken(GrowableArray out) throws IOException {
+ handleTokenTypeTag(out.getDataOutput());
int hash = GOLDEN_RATIO_32;
@@ -59,6 +59,6 @@
// token count
hash += tokenCount;
- dos.writeInt(hash);
+ out.getDataOutput().writeInt(hash);
}
}
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
index 42ed053..20405c6 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/HashedUTF8WordToken.java
@@ -19,10 +19,10 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import java.io.DataOutput;
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
public class HashedUTF8WordToken extends UTF8WordToken {
@@ -76,12 +76,12 @@
}
@Override
- public void serializeToken(DataOutput dos) throws IOException {
+ public void serializeToken(GrowableArray out) throws IOException {
if (tokenTypeTag > 0) {
- dos.write(tokenTypeTag);
+ out.getDataOutput().write(tokenTypeTag);
}
// serialize hash value
- dos.writeInt(hash);
+ out.getDataOutput().writeInt(hash);
}
}
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
index c1840d7..47467a1 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/IToken.java
@@ -19,9 +19,10 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import java.io.DataOutput;
import java.io.IOException;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
+
public interface IToken {
public byte[] getData();
@@ -34,7 +35,7 @@
public void reset(byte[] data, int start, int length, int tokenLength,
int tokenCount);
- public void serializeToken(DataOutput dos) throws IOException;
+ public void serializeToken(GrowableArray out) throws IOException;
- public void serializeTokenCount(DataOutput dos) throws IOException;
+ public void serializeTokenCount(GrowableArray out) throws IOException;
}
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
index 59cadc8..8cb9818 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8NGramToken.java
@@ -19,10 +19,10 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import java.io.DataOutput;
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
public class UTF8NGramToken extends AbstractUTF8Token implements INGramToken {
@@ -49,34 +49,39 @@
}
@Override
- public void serializeToken(DataOutput dos) throws IOException {
- handleTokenTypeTag(dos);
+ public void serializeToken(GrowableArray out) throws IOException {
+ handleTokenTypeTag(out.getDataOutput());
+ int tokenUTF8LenOff = out.getLength();
// regular chars
int numRegChars = tokenLength - numPreChars - numPostChars;
// assuming pre and post char need 1-byte each in utf8
- int tokenUTF8Len = getLowerCaseUTF8Len(numRegChars) + numPreChars + numPostChars;
+ int tokenUTF8Len = numPreChars + numPostChars;
- // write utf8 length indicator
- StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+ // Write dummy UTF length which will be correctly set later.
+ out.getDataOutput().writeShort(0);
// pre chars
for (int i = 0; i < numPreChars; i++) {
- StringUtils.writeCharAsModifiedUTF8(PRECHAR, dos);
+ StringUtils.writeCharAsModifiedUTF8(PRECHAR, out.getDataOutput());
}
int pos = start;
for (int i = 0; i < numRegChars; i++) {
char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
- StringUtils.writeCharAsModifiedUTF8(c, dos);
+ tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
pos += UTF8StringPointable.charSize(data, pos);
}
// post chars
for (int i = 0; i < numPostChars; i++) {
- StringUtils.writeCharAsModifiedUTF8(POSTCHAR, dos);
+ StringUtils.writeCharAsModifiedUTF8(POSTCHAR, out.getDataOutput());
}
+
+ // Set UTF length of token.
+ out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+ out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
}
public void setNumPrePostChars(int numPreChars, int numPostChars) {
diff --git a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
index 97a1e12..9d7fe7c 100644
--- a/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
+++ b/fullstack/hyracks/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/tokenizers/UTF8WordToken.java
@@ -19,10 +19,10 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers;
-import java.io.DataOutput;
import java.io.IOException;
import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.dataflow.common.data.util.StringUtils;
public class UTF8WordToken extends AbstractUTF8Token {
@@ -32,16 +32,20 @@
}
@Override
- public void serializeToken(DataOutput dos) throws IOException {
- handleTokenTypeTag(dos);
-
- int tokenUTF8Len = getLowerCaseUTF8Len(tokenLength);
- StringUtils.writeUTF8Len(tokenUTF8Len, dos);
+ public void serializeToken(GrowableArray out) throws IOException {
+ handleTokenTypeTag(out.getDataOutput());
+ int tokenUTF8LenOff = out.getLength();
+ int tokenUTF8Len = 0;
+ // Write dummy UTF length which will be correctly set later.
+ out.getDataOutput().writeShort(0);
int pos = start;
for (int i = 0; i < tokenLength; i++) {
char c = Character.toLowerCase(UTF8StringPointable.charAt(data, pos));
- StringUtils.writeCharAsModifiedUTF8(c, dos);
+ tokenUTF8Len += StringUtils.writeCharAsModifiedUTF8(c, out.getDataOutput());
pos += UTF8StringPointable.charSize(data, pos);
}
+ // Set UTF length of token.
+ out.getByteArray()[tokenUTF8LenOff] = (byte) ((tokenUTF8Len >>> 8) & 0xFF);
+ out.getByteArray()[tokenUTF8LenOff + 1] = (byte) ((tokenUTF8Len >>> 0) & 0xFF);
}
}
diff --git a/fullstack/hyracks/hyracks-storage-am-rtree/pom.xml b/fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
index 61620ec..6c2d734 100644
--- a/fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 9818dce..d9b7b97 100644
--- a/fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -40,9 +40,10 @@
IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
- IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput,
+ IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
this.keyFields = keyFields;
}
diff --git a/fullstack/hyracks/hyracks-storage-common/pom.xml b/fullstack/hyracks/hyracks-storage-common/pom.xml
index 289171a..3360097 100644
--- a/fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-test-support/pom.xml b/fullstack/hyracks/hyracks-test-support/pom.xml
index 25a5378..89233c9 100644
--- a/fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/fullstack/hyracks/hyracks-test-support/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index c122b25..0ca93b2 100644
--- a/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/fullstack/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -101,6 +102,11 @@
}
@Override
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return null;
+ }
+
+ @Override
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
// TODO Auto-generated method stub
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index 7b03a71..d0bb883 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
index 2cf6ce2..59c8c46 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/pom.xml
@@ -18,9 +18,10 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
<encoding>UTF-8</encoding>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
index 5f15a91..3fb6407 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/NGramTokenizerTest.java
@@ -33,6 +33,7 @@
import org.junit.Before;
import org.junit.Test;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
@@ -41,207 +42,196 @@
public class NGramTokenizerTest {
- private char PRECHAR = '#';
- private char POSTCHAR = '$';
+ private char PRECHAR = '#';
+ private char POSTCHAR = '$';
- private String str = "Jürgen S. Generic's Car";
- private byte[] inputBuffer;
+ private String str = "Jürgen S. Generic's Car";
+ private byte[] inputBuffer;
- private int gramLength = 3;
+ private int gramLength = 3;
- private void getExpectedGrams(String s, int gramLength,
- ArrayList<String> grams, boolean prePost) {
+ private void getExpectedGrams(String s, int gramLength, ArrayList<String> grams, boolean prePost) {
- String tmp = s.toLowerCase();
- if (prePost) {
- StringBuilder preBuilder = new StringBuilder();
- for (int i = 0; i < gramLength - 1; i++) {
- preBuilder.append(PRECHAR);
- }
- String pre = preBuilder.toString();
+ String tmp = s.toLowerCase();
+ if (prePost) {
+ StringBuilder preBuilder = new StringBuilder();
+ for (int i = 0; i < gramLength - 1; i++) {
+ preBuilder.append(PRECHAR);
+ }
+ String pre = preBuilder.toString();
- StringBuilder postBuilder = new StringBuilder();
- for (int i = 0; i < gramLength - 1; i++) {
- postBuilder.append(POSTCHAR);
- }
- String post = postBuilder.toString();
+ StringBuilder postBuilder = new StringBuilder();
+ for (int i = 0; i < gramLength - 1; i++) {
+ postBuilder.append(POSTCHAR);
+ }
+ String post = postBuilder.toString();
- tmp = pre + s.toLowerCase() + post;
- }
+ tmp = pre + s.toLowerCase() + post;
+ }
- for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
- String gram = tmp.substring(i, i + gramLength);
- grams.add(gram);
- }
- }
+ for (int i = 0; i < tmp.length() - gramLength + 1; i++) {
+ String gram = tmp.substring(i, i + gramLength);
+ grams.add(gram);
+ }
+ }
- @Before
- public void init() throws Exception {
- // serialize string into bytes
- ByteArrayOutputStream baos = new ByteArrayOutputStream();
- DataOutput dos = new DataOutputStream(baos);
- dos.writeUTF(str);
- inputBuffer = baos.toByteArray();
- }
+ @Before
+ public void init() throws Exception {
+ // serialize string into bytes
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutput dos = new DataOutputStream(baos);
+ dos.writeUTF(str);
+ inputBuffer = baos.toByteArray();
+ }
- void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost)
- throws IOException {
- HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
- NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
- gramLength, prePost, false, false, tokenFactory);
- tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+ void runTestNGramTokenizerWithCountedHashedUTF8Tokens(boolean prePost) throws IOException {
+ HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+ NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, false,
+ false, tokenFactory);
+ tokenizer.reset(inputBuffer, 0, inputBuffer.length);
- ArrayList<String> expectedGrams = new ArrayList<String>();
- getExpectedGrams(str, gramLength, expectedGrams, prePost);
- ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
- HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
- for (String s : expectedGrams) {
- Integer count = gramCounts.get(s);
- if (count == null) {
- count = 1;
- gramCounts.put(s, count);
- } else {
- count++;
- }
+ ArrayList<String> expectedGrams = new ArrayList<String>();
+ getExpectedGrams(str, gramLength, expectedGrams, prePost);
+ ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+ HashMap<String, Integer> gramCounts = new HashMap<String, Integer>();
+ for (String s : expectedGrams) {
+ Integer count = gramCounts.get(s);
+ if (count == null) {
+ count = 1;
+ gramCounts.put(s, count);
+ } else {
+ count++;
+ }
- int hash = tokenHash(s, count);
- expectedHashedGrams.add(hash);
- }
+ int hash = tokenHash(s, count);
+ expectedHashedGrams.add(hash);
+ }
- int tokenCount = 0;
+ int tokenCount = 0;
- while (tokenizer.hasNext()) {
- tokenizer.next();
+ while (tokenizer.hasNext()) {
+ tokenizer.next();
- // serialize hashed token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ // serialize hashed token
+ GrowableArray tokenStorage = new GrowableArray();
- IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ IToken token = tokenizer.getToken();
+ token.serializeToken(tokenStorage);
- // deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(
- tokenBaos.toByteArray());
- DataInput in = new DataInputStream(bais);
+ // deserialize token
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+ DataInput in = new DataInputStream(bais);
- Integer hashedGram = in.readInt();
+ Integer hashedGram = in.readInt();
- // System.out.println(hashedGram);
+ // System.out.println(hashedGram);
- Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+ Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
- tokenCount++;
- }
- // System.out.println("---------");
- }
+ tokenCount++;
+ }
+ // System.out.println("---------");
+ }
- void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost)
- throws IOException {
- HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
- NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
- gramLength, prePost, true, false, tokenFactory);
- tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+ void runTestNGramTokenizerWithHashedUTF8Tokens(boolean prePost) throws IOException {
+ HashedUTF8NGramTokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
+ NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+ tokenFactory);
+ tokenizer.reset(inputBuffer, 0, inputBuffer.length);
- ArrayList<String> expectedGrams = new ArrayList<String>();
- getExpectedGrams(str, gramLength, expectedGrams, prePost);
- ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
- for (String s : expectedGrams) {
- int hash = tokenHash(s, 1);
- expectedHashedGrams.add(hash);
- }
+ ArrayList<String> expectedGrams = new ArrayList<String>();
+ getExpectedGrams(str, gramLength, expectedGrams, prePost);
+ ArrayList<Integer> expectedHashedGrams = new ArrayList<Integer>();
+ for (String s : expectedGrams) {
+ int hash = tokenHash(s, 1);
+ expectedHashedGrams.add(hash);
+ }
- int tokenCount = 0;
+ int tokenCount = 0;
- while (tokenizer.hasNext()) {
- tokenizer.next();
+ while (tokenizer.hasNext()) {
+ tokenizer.next();
- // serialize hashed token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ // serialize hashed token
+ GrowableArray tokenStorage = new GrowableArray();
- IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ IToken token = tokenizer.getToken();
+ token.serializeToken(tokenStorage);
- // deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(
- tokenBaos.toByteArray());
- DataInput in = new DataInputStream(bais);
+ // deserialize token
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+ DataInput in = new DataInputStream(bais);
- Integer hashedGram = in.readInt();
+ Integer hashedGram = in.readInt();
- // System.out.println(hashedGram);
+ // System.out.println(hashedGram);
- Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
+ Assert.assertEquals(expectedHashedGrams.get(tokenCount), hashedGram);
- tokenCount++;
- }
- // System.out.println("---------");
- }
+ tokenCount++;
+ }
+ // System.out.println("---------");
+ }
- void runTestNGramTokenizerWithUTF8Tokens(boolean prePost)
- throws IOException {
- UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
- NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(
- gramLength, prePost, true, false, tokenFactory);
- tokenizer.reset(inputBuffer, 0, inputBuffer.length);
+ void runTestNGramTokenizerWithUTF8Tokens(boolean prePost) throws IOException {
+ UTF8NGramTokenFactory tokenFactory = new UTF8NGramTokenFactory();
+ NGramUTF8StringBinaryTokenizer tokenizer = new NGramUTF8StringBinaryTokenizer(gramLength, prePost, true, false,
+ tokenFactory);
+ tokenizer.reset(inputBuffer, 0, inputBuffer.length);
- ArrayList<String> expectedGrams = new ArrayList<String>();
- getExpectedGrams(str, gramLength, expectedGrams, prePost);
+ ArrayList<String> expectedGrams = new ArrayList<String>();
+ getExpectedGrams(str, gramLength, expectedGrams, prePost);
- int tokenCount = 0;
+ int tokenCount = 0;
- while (tokenizer.hasNext()) {
- tokenizer.next();
+ while (tokenizer.hasNext()) {
+ tokenizer.next();
- // serialize hashed token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ // serialize hashed token
+ GrowableArray tokenStorage = new GrowableArray();
- IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ IToken token = tokenizer.getToken();
+ token.serializeToken(tokenStorage);
- // deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(
- tokenBaos.toByteArray());
- DataInput in = new DataInputStream(bais);
+ // deserialize token
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
+ DataInput in = new DataInputStream(bais);
- String strGram = in.readUTF();
+ String strGram = in.readUTF();
- // System.out.println("\"" + strGram + "\"");
+ // System.out.println("\"" + strGram + "\"");
- Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
+ Assert.assertEquals(expectedGrams.get(tokenCount), strGram);
- tokenCount++;
- }
- // System.out.println("---------");
- }
+ tokenCount++;
+ }
+ // System.out.println("---------");
+ }
- @Test
- public void testNGramTokenizerWithCountedHashedUTF8Tokens()
- throws Exception {
- runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
- runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
- }
+ @Test
+ public void testNGramTokenizerWithCountedHashedUTF8Tokens() throws Exception {
+ runTestNGramTokenizerWithCountedHashedUTF8Tokens(false);
+ runTestNGramTokenizerWithCountedHashedUTF8Tokens(true);
+ }
- @Test
- public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
- runTestNGramTokenizerWithHashedUTF8Tokens(false);
- runTestNGramTokenizerWithHashedUTF8Tokens(true);
- }
+ @Test
+ public void testNGramTokenizerWithHashedUTF8Tokens() throws Exception {
+ runTestNGramTokenizerWithHashedUTF8Tokens(false);
+ runTestNGramTokenizerWithHashedUTF8Tokens(true);
+ }
- @Test
- public void testNGramTokenizerWithUTF8Tokens() throws IOException {
- runTestNGramTokenizerWithUTF8Tokens(false);
- runTestNGramTokenizerWithUTF8Tokens(true);
- }
+ @Test
+ public void testNGramTokenizerWithUTF8Tokens() throws IOException {
+ runTestNGramTokenizerWithUTF8Tokens(false);
+ runTestNGramTokenizerWithUTF8Tokens(true);
+ }
- public int tokenHash(String token, int tokenCount) {
- int h = AbstractUTF8Token.GOLDEN_RATIO_32;
- for (int i = 0; i < token.length(); i++) {
- h ^= token.charAt(i);
- h *= AbstractUTF8Token.GOLDEN_RATIO_32;
- }
- return h + tokenCount;
- }
+ public int tokenHash(String token, int tokenCount) {
+ int h = AbstractUTF8Token.GOLDEN_RATIO_32;
+ for (int i = 0; i < token.length(); i++) {
+ h ^= token.charAt(i);
+ h *= AbstractUTF8Token.GOLDEN_RATIO_32;
+ }
+ return h + tokenCount;
+ }
}
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index c3c9b99..47a068b 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -27,6 +27,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -112,20 +113,19 @@
}
private class TokenIdPair implements Comparable<TokenIdPair> {
- public ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
- public DataOutputStream dos = new DataOutputStream(baaos);
+ public final GrowableArray tokenStorage = new GrowableArray();
public int id;
TokenIdPair(IToken token, int id) throws IOException {
- token.serializeToken(dos);
+ token.serializeToken(tokenStorage);
this.id = id;
}
@Override
public int compareTo(TokenIdPair o) {
- int cmp = btreeBinCmps[0].compare(baaos.getByteArray(), 0,
- baaos.getByteArray().length, o.baaos.getByteArray(), 0,
- o.baaos.getByteArray().length);
+ int cmp = btreeBinCmps[0].compare(tokenStorage.getByteArray(), 0,
+ tokenStorage.getByteArray().length, o.tokenStorage.getByteArray(), 0,
+ o.tokenStorage.getByteArray().length);
if (cmp == 0) {
return id - o.id;
} else {
@@ -157,8 +157,8 @@
for (TokenIdPair t : pairs) {
tb.reset();
- tb.addField(t.baaos.getByteArray(), 0,
- t.baaos.getByteArray().length);
+ tb.addField(t.tokenStorage.getByteArray(), 0,
+ t.tokenStorage.getByteArray().length);
IntegerSerializerDeserializer.INSTANCE.serialize(t.id, tb.getDataOutput());
tb.addFieldEndOffset();
tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
index 53fb96d..810c5f5 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/WordTokenizerTest.java
@@ -34,6 +34,7 @@
import org.junit.Before;
import org.junit.Test;
+import edu.uci.ics.hyracks.data.std.util.GrowableArray;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.AbstractUTF8Token;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
@@ -127,14 +128,13 @@
tokenizer.next();
// serialize token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ GrowableArray tokenStorage = new GrowableArray();
IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ token.serializeToken(tokenStorage);
// deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
DataInput in = new DataInputStream(bais);
Integer hashedToken = in.readInt();
@@ -159,14 +159,13 @@
tokenizer.next();
// serialize token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ GrowableArray tokenStorage = new GrowableArray();
IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ token.serializeToken(tokenStorage);
// deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
DataInput in = new DataInputStream(bais);
Integer hashedToken = in.readInt();
@@ -191,14 +190,13 @@
tokenizer.next();
// serialize hashed token
- ByteArrayOutputStream tokenBaos = new ByteArrayOutputStream();
- DataOutput tokenDos = new DataOutputStream(tokenBaos);
+ GrowableArray tokenStorage = new GrowableArray();
IToken token = tokenizer.getToken();
- token.serializeToken(tokenDos);
+ token.serializeToken(tokenStorage);
// deserialize token
- ByteArrayInputStream bais = new ByteArrayInputStream(tokenBaos.toByteArray());
+ ByteArrayInputStream bais = new ByteArrayInputStream(tokenStorage.getByteArray());
DataInput in = new DataInputStream(bais);
String strToken = in.readUTF();
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index ea86042..7b1a3f3 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index bd10e13..8e429f9 100644
--- a/fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -18,8 +18,9 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
+ <fork>true</fork>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
index 9e453a6..d33ddc5 100644
--- a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
+++ b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-am/pom.xml
@@ -14,8 +14,8 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
<plugin>
diff --git a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
index 08935a7..649aa6c 100644
--- a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
+++ b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-client/pom.xml
@@ -14,8 +14,8 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
<plugin>
diff --git a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
index 3aaf4a2..fe210fd 100644
--- a/fullstack/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
+++ b/fullstack/hyracks/hyracks-yarn/hyracks-yarn-common/pom.xml
@@ -14,8 +14,8 @@
<artifactId>maven-compiler-plugin</artifactId>
<version>2.0.2</version>
<configuration>
- <source>1.6</source>
- <target>1.6</target>
+ <source>1.7</source>
+ <target>1.7</target>
</configuration>
</plugin>
</plugins>
diff --git a/fullstack/hyracks/pom.xml b/fullstack/hyracks/pom.xml
index 570421e..b699542 100644
--- a/fullstack/hyracks/pom.xml
+++ b/fullstack/hyracks/pom.xml
@@ -1,4 +1,4 @@
-
+<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks</groupId>
@@ -8,24 +8,9 @@
<name>hyracks</name>
<properties>
- <jvm.extraargs />
+ <jvm.extraargs/>
</properties>
- <profiles>
- <profile>
- <id>macosx</id>
- <activation>
- <os>
- <name>mac os x</name>
- </os>
- <jdk>1.7</jdk>
- </activation>
- <properties>
- <jvm.extraargs>-Djava.nio.channels.spi.SelectorProvider=sun.nio.ch.KQueueSelectorProvider</jvm.extraargs>
- </properties>
- </profile>
- </profiles>
-
<build>
<plugins>
<plugin>
@@ -44,6 +29,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
+ <version>2.13</version>
<configuration>
<forkMode>pertest</forkMode>
<argLine>-enableassertions -Djava.util.logging.config.file=${user.home}/logging.properties -Xdebug -Xrunjdwp:transport=dt_socket,server=y,address=8000,suspend=n ${jvm.extraargs}</argLine>
@@ -57,6 +43,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-changelog-plugin</artifactId>
+ <version>2.2</version>
</plugin>
</plugins>
</reporting>
@@ -96,6 +83,8 @@
<modules>
<module>hyracks-ipc</module>
<module>hyracks-api</module>
+ <module>hyracks-comm</module>
+ <module>hyracks-client</module>
<module>hyracks-dataflow-common</module>
<module>hyracks-dataflow-std</module>
<module>hyracks-dataflow-hadoop</module>
@@ -116,5 +105,7 @@
<module>hyracks-hadoop-compat</module>
<!--module>hyracks-yarn</module-->
<module>hyracks-maven-plugins</module>
+ <module>hyracks-hdfs</module>
+ <module>hyracks-dist</module>
</modules>
</project>