Merged hyracks_scheduling branch into hyracks_dev_next
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_dev_next@543 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
new file mode 100644
index 0000000..e621a09
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
@@ -0,0 +1,36 @@
+/*
+ * 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.channels;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public interface IInputChannel {
+ public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException;
+
+ public void setAttachment(Object attachment);
+
+ public Object getAttachment();
+
+ public ByteBuffer getNextBuffer();
+
+ public void recycleBuffer(ByteBuffer buffer);
+
+ public void open() throws HyracksDataException;
+
+ public void close() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
similarity index 74%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
index c884998..dd91f28 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
@@ -12,9 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.channels;
-public enum Direction {
- INPUT,
- OUTPUT,
+public interface IInputChannelMonitor {
+ public void notifyDataAvailability(IInputChannel channel, int nFrames);
+
+ public void notifyEndOfStream(IInputChannel channel);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
index 779e9ac..1377e37 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/AbstractHyracksConnection.java
@@ -16,7 +16,6 @@
import java.io.File;
import java.util.EnumSet;
-import java.util.UUID;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
@@ -26,6 +25,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
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.job.JobStatus;
import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
@@ -65,27 +65,27 @@
}
@Override
- public UUID createJob(String appName, JobSpecification jobSpec) throws Exception {
+ public JobId createJob(String appName, JobSpecification jobSpec) throws Exception {
return createJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
}
@Override
- public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
return hci.createJob(appName, JavaSerializationUtils.serialize(jobSpec), jobFlags);
}
@Override
- public JobStatus getJobStatus(UUID jobId) throws Exception {
+ public JobStatus getJobStatus(JobId jobId) throws Exception {
return hci.getJobStatus(jobId);
}
@Override
- public void start(UUID jobId) throws Exception {
+ public void start(JobId jobId) throws Exception {
hci.start(jobId);
}
@Override
- public void waitForCompletion(UUID jobId) throws Exception {
+ public void waitForCompletion(JobId jobId) throws Exception {
hci.waitForCompletion(jobId);
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index a164fee..659a181 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -16,9 +16,9 @@
import java.io.File;
import java.util.EnumSet;
-import java.util.UUID;
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.job.JobStatus;
@@ -27,13 +27,13 @@
public void destroyApplication(String appName) throws Exception;
- public UUID createJob(String appName, JobSpecification jobSpec) throws Exception;
+ public JobId createJob(String appName, JobSpecification jobSpec) throws Exception;
- public UUID createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
- public JobStatus getJobStatus(UUID jobId) throws Exception;
+ public JobStatus getJobStatus(JobId jobId) throws Exception;
- public void start(UUID jobId) throws Exception;
+ public void start(JobId jobId) throws Exception;
- public void waitForCompletion(UUID jobId) throws Exception;
+ public void waitForCompletion(JobId jobId) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index c4bf0e6..f57ac97 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -16,9 +16,9 @@
import java.rmi.Remote;
import java.util.EnumSet;
-import java.util.UUID;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
public interface IHyracksClientInterface extends Remote {
@@ -30,11 +30,11 @@
public void destroyApplication(String appName) throws Exception;
- public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
+ public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
- public JobStatus getJobStatus(UUID jobId) throws Exception;
+ public JobStatus getJobStatus(JobId jobId) throws Exception;
- public void start(UUID jobId) throws Exception;
+ public void start(JobId jobId) throws Exception;
- public void waitForCompletion(UUID jobId) throws Exception;
+ public void waitForCompletion(JobId jobId) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
index 9e3cbbe..ec05d0b 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IConnectionEntry.java
@@ -24,8 +24,6 @@
SelectionKey getSelectionKey();
- void setDataReceiveListener(IDataReceiveListener listener);
-
void attach(Object attachment);
Object getAttachment();
@@ -38,11 +36,5 @@
UUID getStageId();
- void setJobId(UUID jobId);
-
- void setStageId(UUID stageId);
-
boolean aborted();
-
- void abort();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.java
new file mode 100644
index 0000000..89e6da3
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionCollector.java
@@ -0,0 +1,42 @@
+/*
+ * 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.comm;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public interface IPartitionCollector {
+ public JobId getJobId();
+
+ public ConnectorDescriptorId getConnectorId();
+
+ public int getReceiverIndex();
+
+ public void open() throws HyracksException;
+
+ public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException;
+
+ public IFrameReader getReader() throws HyracksException;
+
+ public void close() throws HyracksException;
+
+ public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException;
+
+ public void abort();
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
similarity index 77%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
index 463e4e1..e15ec6d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
@@ -14,10 +14,8 @@
*/
package edu.uci.ics.hyracks.api.comm;
-import java.io.IOException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public interface IDataReceiveListener {
- public void dataReceived(IConnectionEntry entry) throws IOException;
-
- public void eos(IConnectionEntry entry);
+public interface IPartitionWriterFactory {
+ public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
similarity index 65%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
index 7e74aee..8ae10c4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiver.java
@@ -14,12 +14,14 @@
*/
package edu.uci.ics.hyracks.api.comm;
-import java.util.UUID;
+import java.util.BitSet;
-public interface IDataReceiveListenerFactory {
- public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex);
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
- public UUID getJobId();
+public interface IReceiver {
+ public void setReceiverEventListener(IReceiverEventListener listener) throws HyracksException;
- public UUID getStageId();
+ public void init(int sender) throws HyracksException;
+
+ public ISenderProxy getReadySender(BitSet senderMask, int senderIndexPreference);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
similarity index 78%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
index 463e4e1..98c083e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IReceiverEventListener.java
@@ -14,10 +14,8 @@
*/
package edu.uci.ics.hyracks.api.comm;
-import java.io.IOException;
+public interface IReceiverEventListener {
+ public void notifySenderAvailability(int sender);
-public interface IDataReceiveListener {
- public void dataReceived(IConnectionEntry entry) throws IOException;
-
- public void eos(IConnectionEntry entry);
+ public void notifySenderCount(int senderCount);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
similarity index 82%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
index c884998..88c8f27 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISender.java
@@ -12,9 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.comm;
-public enum Direction {
- INPUT,
- OUTPUT,
+public interface ISender {
+ public IFrameWriter createSenderWriter(int receiverIndex);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
similarity index 78%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
index c884998..6b40bae 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/ISenderProxy.java
@@ -12,9 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.comm;
-public enum Direction {
- INPUT,
- OUTPUT,
+import java.nio.ByteBuffer;
+
+public interface ISenderProxy {
+ public ByteBuffer getReadBuffer();
+
+ public int getSenderIndex();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.java
new file mode 100644
index 0000000..b75ed07
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/PartitionChannel.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.comm;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionChannel {
+ private PartitionId partitionId;
+
+ private IInputChannel channel;
+
+ public PartitionChannel(PartitionId partitionId, IInputChannel channel) {
+ this.partitionId = partitionId;
+ this.channel = channel;
+ }
+
+ public PartitionId getPartitionId() {
+ return partitionId;
+ }
+
+ public IInputChannel getInputChannel() {
+ return channel;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.java
new file mode 100644
index 0000000..0668e00
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/Constraint.java
@@ -0,0 +1,46 @@
+/*
+ * 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.constraints;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+
+public class Constraint implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final LValueConstraintExpression lValue;
+
+ private final ConstraintExpression rValue;
+
+ public Constraint(LValueConstraintExpression lValue, ConstraintExpression rValue) {
+ this.lValue = lValue;
+ this.rValue = rValue;
+ }
+
+ public LValueConstraintExpression getLValue() {
+ return lValue;
+ }
+
+ public ConstraintExpression getRValue() {
+ return rValue;
+ }
+
+ @Override
+ public String toString() {
+ return lValue + " in " + rValue;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
similarity index 81%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
index c884998..2e41b0a 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
@@ -12,9 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.constraints;
-public enum Direction {
- INPUT,
- OUTPUT,
+public interface IConstraintAcceptor {
+ public void addConstraint(Constraint constraint);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
deleted file mode 100644
index 7e74814..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintExpressionAcceptor.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.constraints;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-
-public interface IConstraintExpressionAcceptor {
- public void addConstraintExpression(ConstraintExpression constraintExpression);
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
index 309fca1..0e4ca69 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/PartitionConstraintHelper.java
@@ -14,45 +14,31 @@
*/
package edu.uci.ics.hyracks.api.constraints;
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.BelongsToExpression;
import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.EnumeratedCollectionExpression;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public class PartitionConstraintHelper {
public static void addPartitionCountConstraint(JobSpecification spec, IOperatorDescriptor op, int count) {
- spec.addUserConstraint(new RelationalExpression(new PartitionCountExpression(op.getOperatorId()),
- new ConstantExpression(count), RelationalExpression.Operator.EQUAL));
+ spec.addUserConstraint(new Constraint(new PartitionCountExpression(op.getOperatorId()), new ConstantExpression(
+ count)));
}
public static void addLocationChoiceConstraint(JobSpecification spec, IOperatorDescriptor op, String[][] choices) {
addPartitionCountConstraint(spec, op, choices.length);
for (int i = 0; i < choices.length; ++i) {
- String[] choice = choices[i];
- List<ConstraintExpression> choiceExprs = new ArrayList<ConstraintExpression>();
- for (String c : choice) {
- choiceExprs.add(new ConstantExpression(c));
- }
- spec.addUserConstraint(new BelongsToExpression(new PartitionLocationExpression(op.getOperatorId(), i),
- new EnumeratedCollectionExpression(choiceExprs)));
+ spec.addUserConstraint(new Constraint(new PartitionLocationExpression(op.getOperatorId(), i),
+ new ConstantExpression(choices[i])));
}
}
public static void addAbsoluteLocationConstraint(JobSpecification spec, IOperatorDescriptor op, String... locations) {
addPartitionCountConstraint(spec, op, locations.length);
for (int i = 0; i < locations.length; ++i) {
- List<ConstraintExpression> choiceExprs = new ArrayList<ConstraintExpression>();
- choiceExprs.add(new ConstantExpression(locations[i]));
- spec.addUserConstraint(new BelongsToExpression(new PartitionLocationExpression(op.getOperatorId(), i),
- new EnumeratedCollectionExpression(choiceExprs)));
+ spec.addUserConstraint(new Constraint(new PartitionLocationExpression(op.getOperatorId(), i),
+ new ConstantExpression(locations[i])));
}
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java
deleted file mode 100644
index c307e3e..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/BelongsToExpression.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.constraints.expressions;
-
-import java.util.Collection;
-
-public final class BelongsToExpression extends ConstraintExpression {
- private static final long serialVersionUID = 1L;
-
- private final ConstraintExpression itemExpression;
-
- private final ConstraintExpression setExpression;
-
- public BelongsToExpression(ConstraintExpression itemExpression, ConstraintExpression setExpression) {
- this.itemExpression = itemExpression;
- this.setExpression = setExpression;
- }
-
- @Override
- public ExpressionTag getTag() {
- return ExpressionTag.BELONGS_TO;
- }
-
- public ConstraintExpression getItemExpression() {
- return itemExpression;
- }
-
- public ConstraintExpression getSetExpression() {
- return setExpression;
- }
-
- @Override
- public void getChildren(Collection<ConstraintExpression> children) {
- children.add(itemExpression);
- children.add(setExpression);
- }
-
- @Override
- protected void toString(StringBuilder buffer) {
- buffer.append(getTag()).append('(');
- itemExpression.toString(buffer);
- buffer.append(", ");
- setExpression.toString(buffer);
- buffer.append(')');
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
index afd5051..811c048 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ConstraintExpression.java
@@ -23,11 +23,7 @@
public enum ExpressionTag {
CONSTANT,
PARTITION_COUNT,
- RESOURCE_SUM,
PARTITION_LOCATION,
- ENUMERATED_SET,
- BELONGS_TO,
- RELATIONAL,
}
public abstract ExpressionTag getTag();
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java
deleted file mode 100644
index d0ce706..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/EnumeratedCollectionExpression.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.constraints.expressions;
-
-import java.util.Collection;
-
-public class EnumeratedCollectionExpression extends ConstraintExpression {
- private static final long serialVersionUID = 1L;
-
- private final Collection<ConstraintExpression> enumeration;
-
- public EnumeratedCollectionExpression(Collection<ConstraintExpression> enumeration) {
- this.enumeration = enumeration;
- }
-
- @Override
- public ExpressionTag getTag() {
- return ExpressionTag.ENUMERATED_SET;
- }
-
- public Collection<ConstraintExpression> getMembers() {
- return enumeration;
- }
-
- @Override
- public void getChildren(Collection<ConstraintExpression> children) {
- children.addAll(enumeration);
- }
-
- @Override
- protected void toString(StringBuilder buffer) {
- buffer.append(getTag()).append('(');
- boolean first = true;
- for (ConstraintExpression ce : enumeration) {
- if (!first) {
- buffer.append(", ");
- }
- first = false;
- ce.toString(buffer);
- }
- buffer.append(')');
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
similarity index 76%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
index c884998..7557460 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
@@ -12,9 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.constraints.expressions;
-public enum Direction {
- INPUT,
- OUTPUT,
+public abstract class LValueConstraintExpression extends ConstraintExpression {
+ private static final long serialVersionUID = 1L;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
index c538ff7..dc73947 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionCountExpression.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-public final class PartitionCountExpression extends ConstraintExpression {
+public final class PartitionCountExpression extends LValueConstraintExpression {
private static final long serialVersionUID = 1L;
private final OperatorDescriptorId opId;
@@ -44,4 +44,29 @@
protected void toString(StringBuilder buffer) {
buffer.append(getTag()).append('(').append(opId.toString()).append(')');
}
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((opId == null) ? 0 : opId.hashCode());
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ PartitionCountExpression other = (PartitionCountExpression) obj;
+ if (opId == null) {
+ if (other.opId != null)
+ return false;
+ } else if (!opId.equals(other.opId))
+ return false;
+ return true;
+ }
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
index 50e4e47..fbfdc1c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/PartitionLocationExpression.java
@@ -18,7 +18,7 @@
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-public final class PartitionLocationExpression extends ConstraintExpression {
+public final class PartitionLocationExpression extends LValueConstraintExpression {
private static final long serialVersionUID = 1L;
private final OperatorDescriptorId opId;
@@ -50,4 +50,32 @@
protected void toString(StringBuilder buffer) {
buffer.append(getTag()).append('(').append(opId.toString()).append(", ").append(partition).append(')');
}
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((opId == null) ? 0 : opId.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;
+ PartitionLocationExpression other = (PartitionLocationExpression) obj;
+ if (opId == null) {
+ if (other.opId != null)
+ return false;
+ } else if (!opId.equals(other.opId))
+ return false;
+ if (partition != other.partition)
+ return false;
+ return true;
+ }
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java
deleted file mode 100644
index d90bce9..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/RelationalExpression.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.constraints.expressions;
-
-import java.util.Collection;
-
-public class RelationalExpression extends ConstraintExpression {
- private static final long serialVersionUID = 1L;
-
- public enum Operator {
- EQUAL,
- NOT_EQUAL,
- LESS,
- LESS_EQUAL,
- GREATER,
- GREATER_EQUAL
- }
-
- private final ConstraintExpression left;
- private final ConstraintExpression right;
- private final Operator op;
-
- public RelationalExpression(ConstraintExpression left, ConstraintExpression right, Operator op) {
- this.left = left;
- this.right = right;
- this.op = op;
- }
-
- @Override
- public ExpressionTag getTag() {
- return ExpressionTag.RELATIONAL;
- }
-
- public ConstraintExpression getLeft() {
- return left;
- }
-
- public ConstraintExpression getRight() {
- return right;
- }
-
- public Operator getOperator() {
- return op;
- }
-
- @Override
- public void getChildren(Collection<ConstraintExpression> children) {
- children.add(left);
- children.add(right);
- }
-
- @Override
- protected void toString(StringBuilder buffer) {
- buffer.append(getTag()).append('(').append(op).append(", ");
- left.toString(buffer);
- buffer.append(", ");
- right.toString(buffer);
- buffer.append(')');
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java
deleted file mode 100644
index 3fbdd08..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/ResourceSumExpression.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.constraints.expressions;
-
-import java.util.Collection;
-
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-
-public final class ResourceSumExpression extends ConstraintExpression {
- private static final long serialVersionUID = 1L;
-
- private final OperatorDescriptorId opId;
-
- private final String resourceName;
-
- public ResourceSumExpression(OperatorDescriptorId opId, String resourceName) {
- this.opId = opId;
- this.resourceName = resourceName;
- }
-
- @Override
- public ExpressionTag getTag() {
- return ExpressionTag.RESOURCE_SUM;
- }
-
- public OperatorDescriptorId getOperatorDescriptorId() {
- return opId;
- }
-
- public String getResourceName() {
- return resourceName;
- }
-
- @Override
- public void getChildren(Collection<ConstraintExpression> children) {
- }
-
- @Override
- protected void toString(StringBuilder buffer) {
- buffer.append(getTag()).append('(').append(opId.toString()).append(", ").append(resourceName).append(')');
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
index 534507e..e3c71ea 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksJobletContext.java
@@ -14,19 +14,16 @@
*/
package edu.uci.ics.hyracks.api.context;
-import java.util.UUID;
-
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
public interface IHyracksJobletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
public INCApplicationContext getApplicationContext();
- public UUID getJobId();
-
- public int getAttempt();
+ public JobId getJobId();
public ICounterContext getCounterContext();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
deleted file mode 100644
index ae171fe..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.context;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
-
-public interface IHyracksStageletContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
- public IHyracksJobletContext getJobletContext();
-
- public UUID getStageId();
-
- public ICounterContext getCounterContext();
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
new file mode 100644
index 0000000..7609951
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.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.context;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
+
+public interface IHyracksTaskContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
+ public IHyracksJobletContext getJobletContext();
+
+ public TaskAttemptId getTaskAttemptId();
+
+ public ICounterContext getCounterContext();
+
+ public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException;
+
+ public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException;
+
+ public void setVariable(String name, Object value) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
deleted file mode 100644
index 9023cee..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/IClusterController.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.control;
-
-import java.rmi.Remote;
-import java.util.List;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
-
-public interface IClusterController extends Remote {
- public NodeParameters registerNode(INodeController nodeController) throws Exception;
-
- public void unregisterNode(INodeController nodeController) throws Exception;
-
- public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
- throws Exception;
-
- public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
-
- public void nodeHeartbeat(String id) throws Exception;
-
- public void reportProfile(String id, List<JobProfile> profiles) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
deleted file mode 100644
index d0269fc..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/INodeController.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.control;
-
-import java.rmi.Remote;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-
-public interface INodeController extends Remote {
- public String getId() throws Exception;
-
- public NCConfig getConfiguration() throws Exception;
-
- public NodeCapability getNodeCapability() throws Exception;
-
- public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt, byte[] plan,
- UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions)
- throws Exception;
-
- public void initializeJobletPhase2(String appName, UUID jobId, byte[] plan, UUID stageId,
- Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
- Map<PortInstanceId, Endpoint> globalPortMap) throws Exception;
-
- public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception;
-
- public void abortJoblet(UUID jobId, int attempt) throws Exception;
-
- public void cleanUpJob(UUID jobId) throws Exception;
-
- public void startStage(UUID jobId, UUID stageId) throws Exception;
-
- public void notifyRegistration(IClusterController ccs) throws Exception;
-
- public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
- throws Exception;
-
- public void destroyApplication(String appName) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
similarity index 73%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
index 12f76fe..c6ca51c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityNodeId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ActivityId.java
@@ -15,14 +15,13 @@
package edu.uci.ics.hyracks.api.dataflow;
import java.io.Serializable;
-import java.util.UUID;
-public final class ActivityNodeId implements Serializable {
+public final class ActivityId implements Serializable {
private static final long serialVersionUID = 1L;
private final OperatorDescriptorId odId;
- private final UUID id;
+ private final long id;
- public ActivityNodeId(OperatorDescriptorId odId, UUID id) {
+ public ActivityId(OperatorDescriptorId odId, long id) {
this.odId = odId;
this.id = id;
}
@@ -31,13 +30,13 @@
return odId;
}
- public UUID getLocalId() {
+ public long getLocalId() {
return id;
}
@Override
public int hashCode() {
- return odId.hashCode() + id.hashCode();
+ return (int) (odId.hashCode() + id);
}
@Override
@@ -45,14 +44,14 @@
if (this == o) {
return true;
}
- if (!(o instanceof ActivityNodeId)) {
+ if (!(o instanceof ActivityId)) {
return false;
}
- ActivityNodeId other = (ActivityNodeId) o;
- return other.odId.equals(odId) && other.id.equals(id);
+ ActivityId other = (ActivityId) o;
+ return other.odId.equals(odId) && other.id == id;
}
public String toString() {
- return "ANID:" + id;
+ return "ANID:[" + odId + "]:" + id;
}
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
index 86d8118..3133a7f 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/ConnectorDescriptorId.java
@@ -15,24 +15,23 @@
package edu.uci.ics.hyracks.api.dataflow;
import java.io.Serializable;
-import java.util.UUID;
public final class ConnectorDescriptorId implements Serializable {
private static final long serialVersionUID = 1L;
- private UUID id;
+ private int id;
- public ConnectorDescriptorId(UUID id) {
+ public ConnectorDescriptorId(int id) {
this.id = id;
}
- public UUID getId() {
+ public int getId() {
return id;
}
@Override
public int hashCode() {
- return id.hashCode();
+ return id;
}
@Override
@@ -44,7 +43,7 @@
return false;
}
ConnectorDescriptorId other = (ConnectorDescriptorId) obj;
- return id.equals(other.id);
+ return id == other.id;
}
@Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
similarity index 76%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
index 39d2eda..6b519d6 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityNode.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivity.java
@@ -16,16 +16,14 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
-public interface IActivityNode extends Serializable {
- public ActivityNodeId getActivityNodeId();
+public interface IActivity extends Serializable {
+ public ActivityId getActivityId();
- public IOperatorDescriptor getOwner();
-
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
index c6bc5e3..56870b2 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
@@ -15,11 +15,11 @@
package edu.uci.ics.hyracks.api.dataflow;
public interface IActivityGraphBuilder {
- public void addTask(IActivityNode task);
+ public void addActivity(IActivity task);
- public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked);
+ public void addBlockingEdge(IActivity blocker, IActivity blocked);
- public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex);
+ public void addSourceEdge(int operatorInputIndex, IActivity task, int taskInputIndex);
- public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex);
+ public void addTargetEdge(int operatorOutputIndex, IActivity task, int taskOutputIndex);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index 3dcf620..ec8ab76 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
@@ -15,19 +15,20 @@
package edu.uci.ics.hyracks.api.dataflow;
import java.io.Serializable;
+import java.util.BitSet;
import org.json.JSONException;
import org.json.JSONObject;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+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.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
/**
* Connector that connects operators in a Job.
@@ -60,8 +61,8 @@
* @return data writer.
* @throws Exception
*/
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
throws HyracksDataException;
/**
@@ -71,20 +72,17 @@
* Context
* @param recordDesc
* Job plan
- * @param demux
- * Connection Demultiplexer
- * @param index
+ * @param receiverIndex
* ordinal index of the data consumer partition
* @param nProducerPartitions
* Number of partitions of the producing operator.
* @param nConsumerPartitions
* Number of partitions of the consuming operator.
- * @return data reader
+ * @return partition collector
* @throws HyracksDataException
*/
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException;
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ int receiverIndex, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException;
/**
* Contribute any scheduling constraints imposed by this connector
@@ -94,10 +92,16 @@
* @param plan
* - Job Plan
*/
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx);
/**
+ * Indicate which consumer partitions may receive data from the given producer partition.
+ */
+ public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+ BitSet targetBitmap);
+
+ /**
* Translate this connector descriptor to JSON.
*
* @return
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
index 9e8dbcb..78847eb 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IOperatorDescriptor.java
@@ -20,9 +20,9 @@
import org.json.JSONObject;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
/**
* Descriptor for operators in Hyracks.
@@ -64,7 +64,7 @@
* @param builder
* - graph builder
*/
- public void contributeTaskGraph(IActivityGraphBuilder builder);
+ public void contributeActivities(IActivityGraphBuilder builder);
/**
* Contributes any scheduling constraints imposed by this operator.
@@ -74,7 +74,7 @@
* @param plan
* - Job Plan
*/
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx);
/**
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
index bbf3cf5..b858736 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/OperatorDescriptorId.java
@@ -15,24 +15,23 @@
package edu.uci.ics.hyracks.api.dataflow;
import java.io.Serializable;
-import java.util.UUID;
public final class OperatorDescriptorId implements Serializable {
private static final long serialVersionUID = 1L;
- private final UUID id;
+ private final int id;
- public OperatorDescriptorId(UUID id) {
+ public OperatorDescriptorId(int id) {
this.id = id;
}
- public UUID getId() {
+ public int getId() {
return id;
}
@Override
public int hashCode() {
- return id.hashCode();
+ return id;
}
@Override
@@ -43,7 +42,7 @@
if (!(o instanceof OperatorDescriptorId)) {
return false;
}
- return ((OperatorDescriptorId) o).id.equals(id);
+ return ((OperatorDescriptorId) o).id == id;
}
@Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
deleted file mode 100644
index 844266d..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/PortInstanceId.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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;
-
-import java.io.Serializable;
-
-public final class PortInstanceId implements Serializable {
- private static final long serialVersionUID = 1L;
-
- private OperatorDescriptorId odId;
- private Direction direction;
- private int portIndex;
- private int partition;
-
- public PortInstanceId(OperatorDescriptorId odId, Direction direction, int portIndex, int partition) {
- this.odId = odId;
- this.direction = direction;
- this.portIndex = portIndex;
- this.partition = partition;
- }
-
- public OperatorDescriptorId getOperatorId() {
- return odId;
- }
-
- public Direction getDirection() {
- return direction;
- }
-
- public int getPortIndex() {
- return portIndex;
- }
-
- public int getPartition() {
- return partition;
- }
-
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + ((direction == null) ? 0 : direction.hashCode());
- result = prime * result + ((odId == null) ? 0 : odId.hashCode());
- result = prime * result + partition;
- result = prime * result + portIndex;
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj)
- return true;
- if (obj == null)
- return false;
- if (getClass() != obj.getClass())
- return false;
- PortInstanceId other = (PortInstanceId) obj;
- if (direction == null) {
- if (other.direction != null)
- return false;
- } else if (!direction.equals(other.direction))
- return false;
- if (odId == null) {
- if (other.odId != null)
- return false;
- } else if (!odId.equals(other.odId))
- return false;
- if (partition != other.partition)
- return false;
- if (portIndex != other.portIndex)
- return false;
- return true;
- }
-
- @Override
- public String toString() {
- return odId + ":" + direction + ":" + partition + ":" + portIndex;
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
new file mode 100644
index 0000000..d4e6972
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskAttemptId.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+public final class TaskAttemptId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final TaskId taskId;
+
+ private final int attempt;
+
+ public TaskAttemptId(TaskId taskId, int attempt) {
+ this.taskId = taskId;
+ this.attempt = attempt;
+ }
+
+ public TaskId getTaskId() {
+ return taskId;
+ }
+
+ public int getAttempt() {
+ return attempt;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof TaskAttemptId)) {
+ return false;
+ }
+ TaskAttemptId oTaskId = (TaskAttemptId) o;
+ return oTaskId.attempt == attempt && oTaskId.taskId.equals(taskId);
+ }
+
+ @Override
+ public int hashCode() {
+ return taskId.hashCode() + attempt;
+ }
+
+ @Override
+ public String toString() {
+ return "TAID:[" + taskId + "]:" + attempt;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
new file mode 100644
index 0000000..ee63355
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/TaskId.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import java.io.Serializable;
+
+public final class TaskId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityId activityId;
+
+ private final int partition;
+
+ public TaskId(ActivityId activityId, int partition) {
+ this.activityId = activityId;
+ this.partition = partition;
+ }
+
+ public ActivityId getActivityId() {
+ return activityId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof TaskId)) {
+ return false;
+ }
+ TaskId oTaskId = (TaskId) o;
+ return oTaskId.partition == partition && oTaskId.activityId.equals(activityId);
+ }
+
+ @Override
+ public int hashCode() {
+ return activityId.hashCode() + partition;
+ }
+
+ @Override
+ public String toString() {
+ return "TID:[" + activityId + "]:" + partition;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
similarity index 67%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
index 88efc1f..d6d6fee 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicy.java
@@ -12,20 +12,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.api.dataflow.connectors;
import java.io.Serializable;
-public class NodeCapability implements Serializable {
- private static final long serialVersionUID = 1L;
+public interface IConnectorPolicy extends Serializable {
+ public boolean requiresProducerConsumerCoscheduling();
- private int cpuCount;
+ public boolean consumerWaitsForProducerToFinish();
- public int getCPUCount() {
- return cpuCount;
- }
+ public boolean materializeOnSendSide();
- public void setCPUCount(int cpuCount) {
- this.cpuCount = cpuCount;
- }
+ public boolean materializeOnReceiveSide();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
similarity index 66%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
index 88efc1f..ca684e4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/IConnectorPolicyAssignmentPolicy.java
@@ -12,20 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.api.dataflow.connectors;
import java.io.Serializable;
-public class NodeCapability implements Serializable {
- private static final long serialVersionUID = 1L;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
- private int cpuCount;
-
- public int getCPUCount() {
- return cpuCount;
- }
-
- public void setCPUCount(int cpuCount) {
- this.cpuCount = cpuCount;
- }
+public interface IConnectorPolicyAssignmentPolicy extends Serializable {
+ public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
+ int[] fanouts);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.java
new file mode 100644
index 0000000..39b3904
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/PipeliningConnectorPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.connectors;
+
+public final class PipeliningConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return true;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnSendSide() {
+ return false;
+ }
+
+ @Override
+ public boolean materializeOnReceiveSide() {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.java
new file mode 100644
index 0000000..a5017de
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedBlockingConnectorPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.connectors;
+
+public final class SendSideMaterializedBlockingConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return false;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnSendSide() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnReceiveSide() {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.java
new file mode 100644
index 0000000..5d522c0
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedPipeliningConnectorPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.connectors;
+
+public final class SendSideMaterializedPipeliningConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return false;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return false;
+ }
+
+ @Override
+ public boolean materializeOnSendSide() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnReceiveSide() {
+ return false;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.java
new file mode 100644
index 0000000..82608c4
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.connectors;
+
+public final class SendSideMaterializedReceiveSideMaterializedBlockingConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return false;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnSendSide() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnReceiveSide() {
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.java
new file mode 100644
index 0000000..1cc0583
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/connectors/SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy.java
@@ -0,0 +1,39 @@
+/*
+ * 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.connectors;
+
+public final class SendSideMaterializedReceiveSideMaterializedPipeliningConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return false;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return false;
+ }
+
+ @Override
+ public boolean materializeOnSendSide() {
+ return true;
+ }
+
+ @Override
+ public boolean materializeOnReceiveSide() {
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
index e9ba2c3..9e27077 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/FileReference.java
@@ -51,4 +51,8 @@
public int hashCode() {
return file.hashCode();
}
+
+ public void delete() {
+ file.delete();
+ }
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
index 1311ec5..7b9a7c4 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/io/IWorkspaceFileFactory.java
@@ -17,5 +17,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IWorkspaceFileFactory {
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException;
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException;
+
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
index 008fdb1..c3a0f7e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobLifecycleListener.java
@@ -14,14 +14,12 @@
*/
package edu.uci.ics.hyracks.api.job;
-import java.util.UUID;
-
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
public interface IJobLifecycleListener {
- public void notifyJobCreation(UUID jobId, JobSpecification jobSpec) throws HyracksException;
+ public void notifyJobCreation(JobId jobId, JobSpecification jobSpec) throws HyracksException;
- public void notifyJobStart(UUID jobId) throws HyracksException;
+ public void notifyJobStart(JobId jobId) throws HyracksException;
- public void notifyJobFinish(UUID jobId) throws HyracksException;
+ public void notifyJobFinish(JobId jobId) throws HyracksException;
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
new file mode 100644
index 0000000..930d299
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -0,0 +1,263 @@
+/*
+ * 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.job;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+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.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.util.Pair;
+
+public class JobActivityGraph implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final String appName;
+
+ private final JobSpecification jobSpec;
+
+ private final EnumSet<JobFlag> jobFlags;
+
+ private final Map<ActivityId, IActivity> activityNodes;
+
+ private final Map<ActivityId, Set<ActivityId>> blocker2blockedMap;
+
+ private final Map<ActivityId, Set<ActivityId>> blocked2blockerMap;
+
+ private final Map<OperatorDescriptorId, Set<ActivityId>> operatorActivityMap;
+
+ private final Map<ActivityId, List<Integer>> activityInputMap;
+
+ private final Map<ActivityId, List<Integer>> activityOutputMap;
+
+ private final Map<OperatorDescriptorId, List<ActivityId>> operatorInputMap;
+
+ private final Map<OperatorDescriptorId, List<ActivityId>> operatorOutputMap;
+
+ public JobActivityGraph(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+ this.appName = appName;
+ this.jobSpec = jobSpec;
+ this.jobFlags = jobFlags;
+ activityNodes = new HashMap<ActivityId, IActivity>();
+ blocker2blockedMap = new HashMap<ActivityId, Set<ActivityId>>();
+ blocked2blockerMap = new HashMap<ActivityId, Set<ActivityId>>();
+ operatorActivityMap = new HashMap<OperatorDescriptorId, Set<ActivityId>>();
+ activityInputMap = new HashMap<ActivityId, List<Integer>>();
+ activityOutputMap = new HashMap<ActivityId, List<Integer>>();
+ operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
+ operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
+ }
+
+ public String getApplicationName() {
+ return appName;
+ }
+
+ public JobSpecification getJobSpecification() {
+ return jobSpec;
+ }
+
+ public EnumSet<JobFlag> getJobFlags() {
+ return jobFlags;
+ }
+
+ public Map<ActivityId, IActivity> getActivityNodeMap() {
+ return activityNodes;
+ }
+
+ public Map<ActivityId, Set<ActivityId>> getBlocker2BlockedMap() {
+ return blocker2blockedMap;
+ }
+
+ public Map<ActivityId, Set<ActivityId>> getBlocked2BlockerMap() {
+ return blocked2blockerMap;
+ }
+
+ public Map<OperatorDescriptorId, Set<ActivityId>> getOperatorActivityMap() {
+ return operatorActivityMap;
+ }
+
+ public Map<ActivityId, List<Integer>> getActivityInputMap() {
+ return activityInputMap;
+ }
+
+ public Map<ActivityId, List<Integer>> getActivityOutputMap() {
+ return activityOutputMap;
+ }
+
+ public Map<OperatorDescriptorId, List<ActivityId>> getOperatorInputMap() {
+ return operatorInputMap;
+ }
+
+ public Map<OperatorDescriptorId, List<ActivityId>> getOperatorOutputMap() {
+ return operatorOutputMap;
+ }
+
+ public List<IConnectorDescriptor> getActivityInputConnectorDescriptors(ActivityId hanId) {
+ List<Integer> inputIndexes = activityInputMap.get(hanId);
+ if (inputIndexes == null) {
+ return null;
+ }
+ OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
+ List<IConnectorDescriptor> inputs = new ArrayList<IConnectorDescriptor>();
+ for (Integer i : inputIndexes) {
+ inputs.add(jobSpec.getInputConnectorDescriptor(ownerId, i));
+ }
+ return inputs;
+ }
+
+ public List<IConnectorDescriptor> getActivityOutputConnectorDescriptors(ActivityId hanId) {
+ List<Integer> outputIndexes = activityOutputMap.get(hanId);
+ if (outputIndexes == null) {
+ return null;
+ }
+ OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
+ List<IConnectorDescriptor> outputs = new ArrayList<IConnectorDescriptor>();
+ for (Integer i : outputIndexes) {
+ outputs.add(jobSpec.getOutputConnectorDescriptor(ownerId, i));
+ }
+ return outputs;
+ }
+
+ public ActivityId getConsumerActivity(ConnectorDescriptorId cdId) {
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
+ .getConnectorOperatorMap().get(cdId);
+
+ OperatorDescriptorId consumerOpId = connEdge.second.first.getOperatorId();
+ int consumerInputIdx = connEdge.second.second;
+
+ for (ActivityId anId : operatorActivityMap.get(consumerOpId)) {
+ List<Integer> anInputs = activityInputMap.get(anId);
+ if (anInputs != null) {
+ for (Integer idx : anInputs) {
+ if (idx.intValue() == consumerInputIdx) {
+ return anId;
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ public ActivityId getProducerActivity(ConnectorDescriptorId cdId) {
+ Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
+ .getConnectorOperatorMap().get(cdId);
+
+ OperatorDescriptorId producerOpId = connEdge.first.first.getOperatorId();
+ int producerInputIdx = connEdge.first.second;
+
+ for (ActivityId anId : operatorActivityMap.get(producerOpId)) {
+ List<Integer> anOutputs = activityOutputMap.get(anId);
+ if (anOutputs != null) {
+ for (Integer idx : anOutputs) {
+ if (idx.intValue() == producerInputIdx) {
+ return anId;
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ public RecordDescriptor getActivityInputRecordDescriptor(ActivityId hanId, int inputIndex) {
+ int opInputIndex = getActivityInputMap().get(hanId).get(inputIndex);
+ return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
+ }
+
+ public RecordDescriptor getActivityOutputRecordDescriptor(ActivityId hanId, int outputIndex) {
+ int opOutputIndex = getActivityOutputMap().get(hanId).get(outputIndex);
+ return jobSpec.getOperatorOutputRecordDescriptor(hanId.getOperatorDescriptorId(), opOutputIndex);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buffer = new StringBuilder();
+ buffer.append("ActivityNodes: " + activityNodes);
+ buffer.append('\n');
+ buffer.append("Blocker->Blocked: " + blocker2blockedMap);
+ buffer.append('\n');
+ buffer.append("Blocked->Blocker: " + blocked2blockerMap);
+ buffer.append('\n');
+ return buffer.toString();
+ }
+
+ public JSONObject toJSON() throws JSONException {
+ JSONObject jplan = new JSONObject();
+
+ jplan.put("type", "plan");
+ jplan.put("flags", jobFlags.toString());
+
+ JSONArray jans = new JSONArray();
+ for (IActivity an : activityNodes.values()) {
+ JSONObject jan = new JSONObject();
+ jan.put("type", "activity");
+ jan.put("id", an.getActivityId().toString());
+ jan.put("java-class", an.getClass().getName());
+ jan.put("operator-id", an.getActivityId().getOperatorDescriptorId().toString());
+
+ List<IConnectorDescriptor> inputs = getActivityInputConnectorDescriptors(an.getActivityId());
+ if (inputs != null) {
+ JSONArray jInputs = new JSONArray();
+ for (int i = 0; i < inputs.size(); ++i) {
+ JSONObject jInput = new JSONObject();
+ jInput.put("type", "activity-input");
+ jInput.put("input-port", i);
+ jInput.put("connector-id", inputs.get(i).getConnectorId().toString());
+ jInputs.put(jInput);
+ }
+ jan.put("inputs", jInputs);
+ }
+
+ List<IConnectorDescriptor> outputs = getActivityOutputConnectorDescriptors(an.getActivityId());
+ if (outputs != null) {
+ JSONArray jOutputs = new JSONArray();
+ for (int i = 0; i < outputs.size(); ++i) {
+ JSONObject jOutput = new JSONObject();
+ jOutput.put("type", "activity-output");
+ jOutput.put("output-port", i);
+ jOutput.put("connector-id", outputs.get(i).getConnectorId().toString());
+ jOutputs.put(jOutput);
+ }
+ jan.put("outputs", jOutputs);
+ }
+
+ Set<ActivityId> blockers = getBlocked2BlockerMap().get(an.getActivityId());
+ if (blockers != null) {
+ JSONArray jDeps = new JSONArray();
+ for (ActivityId blocker : blockers) {
+ jDeps.put(blocker.toString());
+ }
+ jan.put("depends-on", jDeps);
+ }
+ jans.put(jan);
+ }
+ jplan.put("activities", jans);
+
+ return jplan;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.java
new file mode 100644
index 0000000..ffb9bba
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobId.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.api.job;
+
+import java.io.Serializable;
+
+public final class JobId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final long id;
+
+ public JobId(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 JobId)) {
+ return false;
+ }
+ return ((JobId) o).id == id;
+ }
+
+ @Override
+ public String toString() {
+ return "JID:" + id;
+ }
+
+ public static JobId parse(String str) {
+ if (str.startsWith("JID:")) {
+ str = str.substring(4);
+ return new JobId(Long.parseLong(str));
+ }
+ throw new IllegalArgumentException();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
deleted file mode 100644
index 13da402..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobPlan.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * 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.job;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-
-public class JobPlan implements Serializable {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
-
- private final UUID jobId;
-
- private final JobSpecification jobSpec;
-
- private final EnumSet<JobFlag> jobFlags;
-
- private final Map<ActivityNodeId, IActivityNode> activityNodes;
-
- private final Map<ActivityNodeId, Set<ActivityNodeId>> blocker2blockedMap;
-
- private final Map<ActivityNodeId, Set<ActivityNodeId>> blocked2blockerMap;
-
- private final Map<OperatorDescriptorId, Set<ActivityNodeId>> operatorTaskMap;
-
- private final Map<ActivityNodeId, List<Integer>> taskInputMap;
-
- private final Map<ActivityNodeId, List<Integer>> taskOutputMap;
-
- private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorInputMap;
-
- private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorOutputMap;
-
- public JobPlan(String appName, UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
- this.appName = appName;
- this.jobId = jobId;
- this.jobSpec = jobSpec;
- this.jobFlags = jobFlags;
- activityNodes = new HashMap<ActivityNodeId, IActivityNode>();
- blocker2blockedMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
- blocked2blockerMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
- operatorTaskMap = new HashMap<OperatorDescriptorId, Set<ActivityNodeId>>();
- taskInputMap = new HashMap<ActivityNodeId, List<Integer>>();
- taskOutputMap = new HashMap<ActivityNodeId, List<Integer>>();
- operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
- operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
- }
-
- public String getApplicationName() {
- return appName;
- }
-
- public UUID getJobId() {
- return jobId;
- }
-
- public JobSpecification getJobSpecification() {
- return jobSpec;
- }
-
- public EnumSet<JobFlag> getJobFlags() {
- return jobFlags;
- }
-
- public Map<ActivityNodeId, IActivityNode> getActivityNodeMap() {
- return activityNodes;
- }
-
- public Map<ActivityNodeId, Set<ActivityNodeId>> getBlocker2BlockedMap() {
- return blocker2blockedMap;
- }
-
- public Map<ActivityNodeId, Set<ActivityNodeId>> getBlocked2BlockerMap() {
- return blocked2blockerMap;
- }
-
- public Map<OperatorDescriptorId, Set<ActivityNodeId>> getOperatorTaskMap() {
- return operatorTaskMap;
- }
-
- public Map<ActivityNodeId, List<Integer>> getTaskInputMap() {
- return taskInputMap;
- }
-
- public Map<ActivityNodeId, List<Integer>> getTaskOutputMap() {
- return taskOutputMap;
- }
-
- public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorInputMap() {
- return operatorInputMap;
- }
-
- public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorOutputMap() {
- return operatorOutputMap;
- }
-
- public List<IConnectorDescriptor> getTaskInputs(ActivityNodeId hanId) {
- List<Integer> inputIndexes = taskInputMap.get(hanId);
- if (inputIndexes == null) {
- return null;
- }
- OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
- List<IConnectorDescriptor> inputs = new ArrayList<IConnectorDescriptor>();
- for (Integer i : inputIndexes) {
- inputs.add(jobSpec.getInputConnectorDescriptor(ownerId, i));
- }
- return inputs;
- }
-
- public List<IConnectorDescriptor> getTaskOutputs(ActivityNodeId hanId) {
- List<Integer> outputIndexes = taskOutputMap.get(hanId);
- if (outputIndexes == null) {
- return null;
- }
- OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
- List<IConnectorDescriptor> outputs = new ArrayList<IConnectorDescriptor>();
- for (Integer i : outputIndexes) {
- outputs.add(jobSpec.getOutputConnectorDescriptor(ownerId, i));
- }
- return outputs;
- }
-
- public RecordDescriptor getTaskInputRecordDescriptor(ActivityNodeId hanId, int inputIndex) {
- int opInputIndex = getTaskInputMap().get(hanId).get(inputIndex);
- return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
- }
-
- public RecordDescriptor getTaskOutputRecordDescriptor(ActivityNodeId hanId, int outputIndex) {
- int opOutputIndex = getTaskOutputMap().get(hanId).get(outputIndex);
- return jobSpec.getOperatorOutputRecordDescriptor(hanId.getOperatorDescriptorId(), opOutputIndex);
- }
-
- @Override
- public String toString() {
- StringBuilder buffer = new StringBuilder();
- buffer.append("ActivityNodes: " + activityNodes);
- buffer.append('\n');
- buffer.append("Blocker->Blocked: " + blocker2blockedMap);
- buffer.append('\n');
- buffer.append("Blocked->Blocker: " + blocked2blockerMap);
- buffer.append('\n');
- return buffer.toString();
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 59072f8..e3d28ea 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -26,11 +26,12 @@
import org.json.JSONException;
import org.json.JSONObject;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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.util.Pair;
@@ -51,10 +52,16 @@
private final Map<String, Serializable> properties;
- private final Set<ConstraintExpression> userConstraints;
+ private final Set<Constraint> userConstraints;
+
+ private IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy;
private int maxAttempts;
+ private transient int operatorIdCounter;
+
+ private transient int connectorIdCounter;
+
public JobSpecification() {
roots = new ArrayList<OperatorDescriptorId>();
opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
@@ -63,7 +70,17 @@
opOutputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
connectorOpMap = new HashMap<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>>();
properties = new HashMap<String, Serializable>();
- userConstraints = new HashSet<ConstraintExpression>();
+ userConstraints = new HashSet<Constraint>();
+ operatorIdCounter = 0;
+ connectorIdCounter = 0;
+ }
+
+ public OperatorDescriptorId createOperatorDescriptorId() {
+ return new OperatorDescriptorId(operatorIdCounter++);
+ }
+
+ public ConnectorDescriptorId createConnectorDescriptor() {
+ return new ConnectorDescriptorId(connectorIdCounter++);
}
public void addRoot(IOperatorDescriptor op) {
@@ -173,6 +190,14 @@
return roots;
}
+ public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
+ return connectorPolicyAssignmentPolicy;
+ }
+
+ public void setConnectorPolicyAssignmentPolicy(IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy) {
+ this.connectorPolicyAssignmentPolicy = connectorPolicyAssignmentPolicy;
+ }
+
public void setMaxAttempts(int maxAttempts) {
this.maxAttempts = maxAttempts;
}
@@ -181,11 +206,11 @@
return maxAttempts;
}
- public void addUserConstraint(ConstraintExpression constraint) {
+ public void addUserConstraint(Constraint constraint) {
userConstraints.add(constraint);
}
- public Set<ConstraintExpression> getUserConstraints() {
+ public Set<Constraint> getUserConstraints() {
return userConstraints;
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
deleted file mode 100644
index cb43f66..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/StageletProfile.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.job.profiling.om;
-
-import java.util.UUID;
-
-import org.json.JSONException;
-import org.json.JSONObject;
-
-public class StageletProfile extends AbstractProfile {
- private static final long serialVersionUID = 1L;
-
- private final UUID stageId;
-
- public StageletProfile(UUID stageId) {
- this.stageId = stageId;
- }
-
- public UUID getStageId() {
- return stageId;
- }
-
- @Override
- public JSONObject toJSON() throws JSONException {
- JSONObject json = new JSONObject();
-
- json.put("type", "joblet-profile");
- json.put("stage-id", stageId.toString());
- populateCounters(json);
-
- return json;
- }
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.java
new file mode 100644
index 0000000..7310606
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/naming/MultipartName.java
@@ -0,0 +1,42 @@
+/*
+ * 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.naming;
+
+import java.util.Arrays;
+
+public final class MultipartName {
+ private Object[] parts;
+
+ public MultipartName(Object... parts) {
+ this.parts = parts;
+ }
+
+ public Object[] getParts() {
+ return parts;
+ }
+
+ @Override
+ public int hashCode() {
+ return Arrays.deepHashCode(parts);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof MultipartName)) {
+ return false;
+ }
+ return Arrays.deepEquals(parts, ((MultipartName) o).parts);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
similarity index 74%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
index 45c589a..02eb891 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
@@ -12,11 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.partitions;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface IPartition extends IDeallocatable {
+ public void writeTo(IFrameWriter writer);
+
+ public boolean isReusable();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.java
new file mode 100644
index 0000000..be25d12
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/PartitionId.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.api.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class PartitionId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ConnectorDescriptorId cdId;
+
+ private final int senderIndex;
+
+ private final int receiverIndex;
+
+ public PartitionId(JobId jobId, ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
+ this.jobId = jobId;
+ this.cdId = cdId;
+ this.senderIndex = senderIndex;
+ this.receiverIndex = receiverIndex;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ConnectorDescriptorId getConnectorDescriptorId() {
+ return cdId;
+ }
+
+ public int getSenderIndex() {
+ return senderIndex;
+ }
+
+ public int getReceiverIndex() {
+ return receiverIndex;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((cdId == null) ? 0 : cdId.hashCode());
+ result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+ result = prime * result + receiverIndex;
+ result = prime * result + senderIndex;
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ PartitionId other = (PartitionId) obj;
+ if (cdId == null) {
+ if (other.cdId != null)
+ return false;
+ } else if (!cdId.equals(other.cdId))
+ return false;
+ if (jobId == null) {
+ if (other.jobId != null)
+ return false;
+ } else if (!jobId.equals(other.jobId))
+ return false;
+ if (receiverIndex != other.receiverIndex)
+ return false;
+ if (senderIndex != other.senderIndex)
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return jobId.toString() + ":" + cdId + ":" + senderIndex + ":" + receiverIndex;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java
deleted file mode 100644
index 3a6f930..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ISpaceSharedResource.java
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * 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.resources;
-
-public interface ISpaceSharedResource {
-
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java
deleted file mode 100644
index 1fb0465..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/ITimeSharedResource.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.resources;
-
-public interface ITimeSharedResource {
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
similarity index 69%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
index 45c589a..135a77c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ISliver.java
@@ -12,11 +12,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.things;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import java.nio.ByteBuffer;
+
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface ISliver {
+ public void open() throws HyracksDataException;
+
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+ public void commit() throws HyracksDataException;
+
+ public void abort();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
similarity index 82%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
index 2d76c26..1269e35 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/resources/IResource.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingDescriptor.java
@@ -12,9 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.resources;
+package edu.uci.ics.hyracks.api.things;
import java.io.Serializable;
-public interface IResource extends Serializable {
+public interface IThingDescriptor extends Serializable {
+ public ThingDescriptorId getThingId();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
similarity index 84%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
index c884998..cb79b6f 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/IThingPartition.java
@@ -12,9 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.things;
-public enum Direction {
- INPUT,
- OUTPUT,
+public interface IThingPartition {
+ public ISliver createSliver();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.java
new file mode 100644
index 0000000..5d42ce9
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/things/ThingDescriptorId.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.things;
+
+import java.io.Serializable;
+
+public final class ThingDescriptorId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final long id;
+
+ public ThingDescriptorId(long id) {
+ this.id = id;
+ }
+
+ public long getId() {
+ return id;
+ }
+
+ @Override
+ public String toString() {
+ return "ThID: " + id;
+ }
+
+ public int hashCode() {
+ return (int) (id & 0xffffffff);
+ }
+
+ public boolean equals(Object o) {
+ if (!(o instanceof ThingDescriptorId)) {
+ return false;
+ }
+ return id == ((ThingDescriptorId) o).id;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
index 1451c69..1185b5e 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/util/JavaSerializationUtils.java
@@ -34,6 +34,9 @@
}
public static Object deserialize(byte[] bytes, ClassLoader classLoader) throws IOException, ClassNotFoundException {
+ if (bytes == null) {
+ return null;
+ }
ClassLoader ctxCL = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(classLoader);
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java
new file mode 100644
index 0000000..b13afe4
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableDescriptor.java
@@ -0,0 +1,40 @@
+/*
+ * 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.workflow.variables;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+
+public final class WorkflowVariableDescriptor implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final MultipartName name;
+
+ private final WorkflowVariableValueScope scope;
+
+ public WorkflowVariableDescriptor(MultipartName name, WorkflowVariableValueScope scope) {
+ this.name = name;
+ this.scope = scope;
+ }
+
+ public MultipartName getName() {
+ return name;
+ }
+
+ public WorkflowVariableValueScope getScope() {
+ return scope;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
similarity index 84%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
index c884998..44ed129 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/workflow/variables/WorkflowVariableValueScope.java
@@ -12,9 +12,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.api.workflow.variables;
-public enum Direction {
- INPUT,
- OUTPUT,
+public enum WorkflowVariableValueScope {
+ LOCAL,
+ GLOBAL,
}
\ No newline at end of file
diff --git a/hyracks-cli/.classpath b/hyracks-cli/.classpath
index 93843eb..ba0bb5a 100644
--- a/hyracks-cli/.classpath
+++ b/hyracks-cli/.classpath
@@ -1,7 +1,7 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
<classpathentry kind="src" output="target/classes" path="src/main/java"/>
- <classpathentry kind="src" output="target/classes" path="target/generated-sources/javacc"/>
+ <classpathentry kind="src" path="target/generated-sources/javacc"/>
<classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/JavaSE-1.6"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java
new file mode 100644
index 0000000..77694b4
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCClientInterface.java
@@ -0,0 +1,61 @@
+package edu.uci.ics.hyracks.control.cc;
+
+import java.rmi.RemoteException;
+import java.rmi.server.UnicastRemoteObject;
+import java.util.EnumSet;
+
+import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
+import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+
+public class CCClientInterface extends UnicastRemoteObject implements IHyracksClientInterface {
+ private static final long serialVersionUID = 1L;
+
+ private final ClusterControllerService ccs;
+
+ public CCClientInterface(ClusterControllerService ccs) throws RemoteException {
+ this.ccs = ccs;
+ }
+
+ @Override
+ public ClusterControllerInfo getClusterControllerInfo() throws Exception {
+ return ccs.getClusterControllerInfo();
+ }
+
+ @Override
+ public void createApplication(String appName) throws Exception {
+ ccs.createApplication(appName);
+ }
+
+ @Override
+ public void startApplication(String appName) throws Exception {
+ ccs.startApplication(appName);
+ }
+
+ @Override
+ public void destroyApplication(String appName) throws Exception {
+ ccs.destroyApplication(appName);
+ }
+
+ @Override
+ public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ return ccs.createJob(appName, jobSpec, jobFlags);
+ }
+
+ @Override
+ public JobStatus getJobStatus(JobId jobId) throws Exception {
+ return ccs.getJobStatus(jobId);
+ }
+
+ @Override
+ public void start(JobId jobId) throws Exception {
+ ccs.start(jobId);
+ }
+
+ @Override
+ public void waitForCompletion(JobId jobId) throws Exception {
+ ccs.waitForCompletion(jobId);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
index 3e36a37..4bf3da2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/CCDriver.java
@@ -16,7 +16,7 @@
import org.kohsuke.args4j.CmdLineParser;
-import edu.uci.ics.hyracks.api.control.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
public class CCDriver {
public static void main(String args[]) throws Exception {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index e5efe92..d864c05 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -26,7 +26,6 @@
import java.util.Set;
import java.util.Timer;
import java.util.TimerTask;
-import java.util.UUID;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.logging.Level;
@@ -34,17 +33,13 @@
import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
import edu.uci.ics.hyracks.api.client.IHyracksClientInterface;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.context.ICCContext;
-import edu.uci.ics.hyracks.api.control.CCConfig;
-import edu.uci.ics.hyracks.api.control.IClusterController;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
-import edu.uci.ics.hyracks.api.control.NodeParameters;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -56,18 +51,28 @@
import edu.uci.ics.hyracks.control.cc.job.manager.events.JobStartEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.NodeHeartbeatEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterNodeEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionAvailibilityEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.RegisterPartitionRequestEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.RemoveDeadNodesEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.ReportProfilesEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.StageletCompleteEvent;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.StageletFailureEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskCompleteEvent;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.TaskFailureEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.UnregisterNodeEvent;
import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
import edu.uci.ics.hyracks.control.cc.jobqueue.JobQueue;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-import edu.uci.ics.hyracks.control.cc.scheduler.naive.NaiveScheduler;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
IHyracksClientInterface {
@@ -89,18 +94,20 @@
private ClusterControllerInfo info;
- private final Map<UUID, JobRun> runMap;
+ private final Map<JobId, JobRun> runMap;
private final JobQueue jobQueue;
- private final IScheduler scheduler;
-
private final Executor taskExecutor;
private final Timer timer;
+ private final CCClientInterface ccci;
+
private final ICCContext ccContext;
+ private long jobCounter;
+
public ClusterControllerService(CCConfig ccConfig) throws Exception {
this.ccConfig = ccConfig;
nodeRegistry = new LinkedHashMap<String, NodeControllerState>();
@@ -110,23 +117,24 @@
ClusterControllerService.class.getName()));
taskExecutor = Executors.newCachedThreadPool();
webServer = new WebServer(this);
- runMap = new HashMap<UUID, JobRun>();
+ runMap = new HashMap<JobId, JobRun>();
jobQueue = new JobQueue();
- scheduler = new NaiveScheduler(this);
this.timer = new Timer(true);
+ ccci = new CCClientInterface(this);
ccContext = new ICCContext() {
@Override
public Map<String, Set<String>> getIPAddressNodeMap() {
return ipAddressNodeNameMap;
}
};
+ jobCounter = 0;
}
@Override
public void start() throws Exception {
LOGGER.log(Level.INFO, "Starting ClusterControllerService");
Registry registry = LocateRegistry.createRegistry(ccConfig.port);
- registry.rebind(IHyracksClientInterface.class.getName(), this);
+ registry.rebind(IHyracksClientInterface.class.getName(), ccci);
registry.rebind(IClusterController.class.getName(), this);
webServer.setPort(ccConfig.httpPort);
webServer.start();
@@ -147,7 +155,7 @@
return applications;
}
- public Map<UUID, JobRun> getRunMap() {
+ public Map<JobId, JobRun> getRunMap() {
return runMap;
}
@@ -155,10 +163,6 @@
return jobQueue;
}
- public IScheduler getScheduler() {
- return scheduler;
- }
-
public Executor getExecutor() {
return taskExecutor;
}
@@ -175,9 +179,13 @@
return ccConfig;
}
+ private JobId createJobId() {
+ return new JobId(jobCounter++);
+ }
+
@Override
- public UUID createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- UUID jobId = UUID.randomUUID();
+ public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ JobId jobId = createJobId();
JobCreateEvent jce = new JobCreateEvent(this, jobId, appName, jobSpec, jobFlags);
jobQueue.schedule(jce);
jce.sync();
@@ -185,10 +193,12 @@
}
@Override
- public NodeParameters registerNode(INodeController nodeController) throws Exception {
- String id = nodeController.getId();
- NCConfig ncConfig = nodeController.getConfiguration();
- NodeControllerState state = new NodeControllerState(nodeController, ncConfig);
+ public NodeParameters registerNode(NodeRegistration reg) throws Exception {
+ INodeController nodeController = reg.getNodeController();
+ String id = reg.getNodeId();
+ NCConfig ncConfig = reg.getNCConfig();
+ NetworkAddress dataPort = reg.getDataPort();
+ NodeControllerState state = new NodeControllerState(nodeController, ncConfig, dataPort);
jobQueue.scheduleAndSync(new RegisterNodeEvent(this, id, state));
nodeController.notifyRegistration(this);
LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
@@ -207,33 +217,34 @@
}
@Override
- public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
+ public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
throws Exception {
- StageletCompleteEvent sce = new StageletCompleteEvent(this, jobId, stageId, attempt, nodeId);
+ TaskCompleteEvent sce = new TaskCompleteEvent(this, jobId, taskId, nodeId);
jobQueue.schedule(sce);
}
@Override
- public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception {
- StageletFailureEvent sfe = new StageletFailureEvent(this, jobId, stageId, attempt, nodeId);
- jobQueue.schedule(sfe);
+ public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, Exception exception)
+ throws Exception {
+ TaskFailureEvent tfe = new TaskFailureEvent(this, jobId, taskId, nodeId, exception);
+ jobQueue.schedule(tfe);
}
@Override
- public JobStatus getJobStatus(UUID jobId) throws Exception {
+ public JobStatus getJobStatus(JobId jobId) throws Exception {
GetJobStatusEvent gse = new GetJobStatusEvent(this, jobId);
jobQueue.scheduleAndSync(gse);
return gse.getStatus();
}
@Override
- public void start(UUID jobId) throws Exception {
+ public void start(JobId jobId) throws Exception {
JobStartEvent jse = new JobStartEvent(this, jobId);
- jobQueue.scheduleAndSync(jse);
+ jobQueue.schedule(jse);
}
@Override
- public void waitForCompletion(UUID jobId) throws Exception {
+ public void waitForCompletion(JobId jobId) throws Exception {
GetJobStatusConditionVariableEvent e = new GetJobStatusConditionVariableEvent(this, jobId);
jobQueue.scheduleAndSync(e);
IJobStatusConditionVariable var = e.getConditionVariable();
@@ -249,9 +260,6 @@
@Override
public synchronized void nodeHeartbeat(String id) throws Exception {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Heartbeat from: " + id);
- }
jobQueue.schedule(new NodeHeartbeatEvent(this, id));
}
@@ -285,6 +293,16 @@
return info;
}
+ @Override
+ public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) {
+ jobQueue.schedule(new RegisterPartitionAvailibilityEvent(this, partitionDescriptor));
+ }
+
+ @Override
+ public void registerPartitionRequest(PartitionRequest partitionRequest) {
+ jobQueue.schedule(new RegisterPartitionRequestEvent(this, partitionRequest));
+ }
+
private class DeadNodeSweeper extends TimerTask {
@Override
public void run() {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index 40eb9ae..2b059b2 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -1,25 +1,43 @@
+/*
+ * 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;
import java.util.HashSet;
import java.util.Set;
-import java.util.UUID;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
public class NodeControllerState {
private final INodeController nodeController;
private final NCConfig ncConfig;
- private final Set<UUID> activeJobIds;
+ private final NetworkAddress dataPort;
+
+ private final Set<JobId> activeJobIds;
private int lastHeartbeatDuration;
- public NodeControllerState(INodeController nodeController, NCConfig ncConfig) {
+ public NodeControllerState(INodeController nodeController, NCConfig ncConfig, NetworkAddress dataPort) {
this.nodeController = nodeController;
this.ncConfig = ncConfig;
- activeJobIds = new HashSet<UUID>();
+ this.dataPort = dataPort;
+ activeJobIds = new HashSet<JobId>();
}
public void notifyHeartbeat() {
@@ -42,7 +60,11 @@
return ncConfig;
}
- public Set<UUID> getActiveJobIds() {
+ public Set<JobId> getActiveJobIds() {
return activeJobIds;
}
+
+ public NetworkAddress getDataPort() {
+ return dataPort;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index 96e29d3..c1c0161 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -1,10 +1,23 @@
+/*
+ * 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.application;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
-import java.util.UUID;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
import edu.uci.ics.hyracks.api.application.ICCBootstrap;
@@ -12,6 +25,7 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.control.cc.job.DeserializingJobSpecificationFactory;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
@@ -46,7 +60,7 @@
this.jobSpecFactory = jobSpecFactory;
}
- public JobSpecification createJobSpecification(UUID jobId, byte[] bytes) throws HyracksException {
+ public JobSpecification createJobSpecification(byte[] bytes) throws HyracksException {
return jobSpecFactory.createJobSpecification(bytes, (ICCBootstrap) bootstrap, this);
}
@@ -67,19 +81,19 @@
jobLifecycleListeners.add(jobLifecycleListener);
}
- public synchronized void notifyJobStart(UUID jobId) throws HyracksException {
+ public synchronized void notifyJobStart(JobId jobId) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
l.notifyJobStart(jobId);
}
}
- public synchronized void notifyJobFinish(UUID jobId) throws HyracksException {
+ public synchronized void notifyJobFinish(JobId jobId) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
l.notifyJobFinish(jobId);
}
}
- public synchronized void notifyJobCreation(UUID jobId, JobSpecification specification) throws HyracksException {
+ public synchronized void notifyJobCreation(JobId jobId, JobSpecification specification) throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
l.notifyJobCreation(jobId, specification);
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
new file mode 100644
index 0000000..14cebb2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.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.control.cc.job;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+
+public class ActivityCluster {
+ private final JobRun jobRun;
+
+ private final Set<ActivityId> activities;
+
+ private final Set<ActivityCluster> dependencies;
+
+ private final Set<ActivityCluster> dependents;
+
+ private ActivityClusterId id;
+
+ private ActivityClusterPlan acp;
+
+ public ActivityCluster(JobRun jobRun, Set<ActivityId> activities) {
+ this.jobRun = jobRun;
+ this.activities = activities;
+ dependencies = new HashSet<ActivityCluster>();
+ dependents = new HashSet<ActivityCluster>();
+ }
+
+ public Set<ActivityId> getActivities() {
+ return activities;
+ }
+
+ public void addDependency(ActivityCluster stage) {
+ dependencies.add(stage);
+ }
+
+ public void addDependent(ActivityCluster stage) {
+ dependents.add(stage);
+ }
+
+ public Set<ActivityCluster> getDependencies() {
+ return dependencies;
+ }
+
+ public Set<ActivityCluster> getDependents() {
+ return dependents;
+ }
+
+ public JobRun getJobRun() {
+ return jobRun;
+ }
+
+ public ActivityClusterId getActivityClusterId() {
+ return id;
+ }
+
+ public void setActivityClusterId(ActivityClusterId id) {
+ this.id = id;
+ }
+
+ public int getMaxTaskClusterAttempts() {
+ return jobRun.getJobActivityGraph().getJobSpecification().getMaxAttempts();
+ }
+
+ public ActivityClusterPlan getPlan() {
+ return acp;
+ }
+
+ public void setPlan(ActivityClusterPlan acp) {
+ this.acp = acp;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.java
new file mode 100644
index 0000000..46628a4
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterId.java
@@ -0,0 +1,58 @@
+/*
+ * 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.job;
+
+import java.io.Serializable;
+
+public final class ActivityClusterId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final int id;
+
+ public ActivityClusterId(int id) {
+ this.id = id;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + id;
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ActivityClusterId other = (ActivityClusterId) obj;
+ if (id != other.id)
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "AC:" + id;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.java
new file mode 100644
index 0000000..2b1d3e2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityClusterPlan.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.control.cc.job;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+
+public class ActivityClusterPlan {
+ private final Map<ActivityId, ActivityPlan> activityPlanMap;
+
+ private final TaskCluster[] taskClusters;
+
+ public ActivityClusterPlan(TaskCluster[] taskClusters, Map<ActivityId, ActivityPlan> activityPlanMap) {
+ this.activityPlanMap = activityPlanMap;
+ this.taskClusters = taskClusters;
+ }
+
+ public Map<ActivityId, ActivityPlan> getActivityPlanMap() {
+ return activityPlanMap;
+ }
+
+ public TaskCluster[] getTaskClusters() {
+ return taskClusters;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.java
new file mode 100644
index 0000000..beeca27
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityPlan.java
@@ -0,0 +1,39 @@
+/*
+ * 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.job;
+
+import edu.uci.ics.hyracks.control.cc.scheduler.ActivityPartitionDetails;
+
+public class ActivityPlan {
+ private final ActivityPartitionDetails apd;
+
+ private Task[] tasks;
+
+ public ActivityPlan(ActivityPartitionDetails apd) {
+ this.apd = apd;
+ }
+
+ public ActivityPartitionDetails getActivityPartitionDetails() {
+ return apd;
+ }
+
+ public Task[] getTasks() {
+ return tasks;
+ }
+
+ public void setTasks(Task[] tasks) {
+ this.tasks = tasks;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
new file mode 100644
index 0000000..6effd84
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
@@ -0,0 +1,92 @@
+package edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class JobActivityGraphBuilder implements IActivityGraphBuilder {
+ private static final Logger LOGGER = Logger.getLogger(JobActivityGraphBuilder.class.getName());
+
+ private JobActivityGraph jag;
+
+ @Override
+ public void addBlockingEdge(IActivity blocker, IActivity blocked) {
+ addToValueSet(jag.getBlocker2BlockedMap(), blocker.getActivityId(), blocked.getActivityId());
+ addToValueSet(jag.getBlocked2BlockerMap(), blocked.getActivityId(), blocker.getActivityId());
+ }
+
+ @Override
+ public void addSourceEdge(int operatorInputIndex, IActivity task, int taskInputIndex) {
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("Adding source edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+ + operatorInputIndex + " -> " + task.getActivityId() + ":" + taskInputIndex);
+ }
+ insertIntoIndexedMap(jag.getActivityInputMap(), task.getActivityId(), taskInputIndex, operatorInputIndex);
+ insertIntoIndexedMap(jag.getOperatorInputMap(), task.getActivityId().getOperatorDescriptorId(),
+ operatorInputIndex, task.getActivityId());
+ }
+
+ @Override
+ public void addTargetEdge(int operatorOutputIndex, IActivity task, int taskOutputIndex) {
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("Adding target edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+ + operatorOutputIndex + " -> " + task.getActivityId() + ":" + taskOutputIndex);
+ }
+ insertIntoIndexedMap(jag.getActivityOutputMap(), task.getActivityId(), taskOutputIndex, operatorOutputIndex);
+ insertIntoIndexedMap(jag.getOperatorOutputMap(), task.getActivityId().getOperatorDescriptorId(),
+ operatorOutputIndex, task.getActivityId());
+ }
+
+ @Override
+ public void addActivity(IActivity task) {
+ ActivityId activityId = task.getActivityId();
+ jag.getActivityNodeMap().put(activityId, task);
+ addToValueSet(jag.getOperatorActivityMap(), activityId.getOperatorDescriptorId(), activityId);
+ }
+
+ private <K, V> void addToValueSet(Map<K, Set<V>> map, K n1, V n2) {
+ Set<V> targets = map.get(n1);
+ if (targets == null) {
+ targets = new HashSet<V>();
+ map.put(n1, targets);
+ }
+ targets.add(n2);
+ }
+
+ private <T> void extend(List<T> list, int index) {
+ int n = list.size();
+ for (int i = n; i <= index; ++i) {
+ list.add(null);
+ }
+ }
+
+ public void init(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+ jag = new JobActivityGraph(appName, jobSpec, jobFlags);
+ }
+
+ private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
+ List<V> vList = map.get(key);
+ if (vList == null) {
+ vList = new ArrayList<V>();
+ map.put(key, vList);
+ }
+ extend(vList, index);
+ vList.set(index, value);
+ }
+
+ public JobActivityGraph getActivityGraph() {
+ return jag;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java
deleted file mode 100644
index 284c92f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobAttempt.java
+++ /dev/null
@@ -1,130 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.cc.scheduler.IJobAttemptSchedulerState;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-
-public class JobAttempt {
- private final JobRun jobRun;
-
- private final JobPlan plan;
-
- private final int attempt;
-
- private final JobStage endStage;
-
- private final JobProfile profile;
-
- private final Map<UUID, JobStage> stageMap;
-
- private final Map<UUID, JobStageAttempt> stageAttemptMap;
-
- private final Set<UUID> pendingStages;
-
- private final Set<UUID> completedStages;
-
- private final Set<UUID> inProgressStages;
-
- private final Set<String> participatingNodeIds;
-
- private final IJobAttemptSchedulerState schedulerState;
-
- public JobAttempt(JobRun jobRun, JobPlan plan, int attempt, IScheduler scheduler) {
- this.jobRun = jobRun;
- this.plan = plan;
- this.attempt = attempt;
- this.endStage = new JobPlanner().createStageDAG(plan);
- stageMap = new HashMap<UUID, JobStage>();
- stageAttemptMap = new HashMap<UUID, JobStageAttempt>();
- completedStages = new HashSet<UUID>();
- inProgressStages = new HashSet<UUID>();
- profile = new JobProfile(plan.getJobId(), attempt);
- populateJobStageMap(stageMap, endStage);
- pendingStages = new HashSet<UUID>(stageMap.keySet());
- participatingNodeIds = new HashSet<String>();
- schedulerState = scheduler.createJobAttemptState(this);
- }
-
- private static void populateJobStageMap(Map<UUID, JobStage> stageMap, JobStage stage) {
- stageMap.put(stage.getId(), stage);
- for (JobStage s : stage.getDependencies()) {
- populateJobStageMap(stageMap, s);
- }
- }
-
- public JobRun getJobRun() {
- return jobRun;
- }
-
- public JobPlan getPlan() {
- return plan;
- }
-
- public int getAttempt() {
- return attempt;
- }
-
- public JobStage getEndStage() {
- return endStage;
- }
-
- public void findRunnableStages(Set<JobStage> runnableStages) {
- findRunnableStages(runnableStages, endStage);
- }
-
- private void findRunnableStages(Set<JobStage> runnableStages, JobStage stage) {
- if (completedStages.contains(stage.getId()) || inProgressStages.contains(stage.getId())
- || runnableStages.contains(stage)) {
- return;
- }
- boolean runnable = true;
- for (JobStage s : stage.getDependencies()) {
- if (!completedStages.contains(s.getId())) {
- runnable = false;
- findRunnableStages(runnableStages, s);
- }
- }
- if (runnable) {
- runnableStages.add(stage);
- }
- }
-
- public Set<UUID> getPendingStageIds() {
- return pendingStages;
- }
-
- public Set<UUID> getInProgressStageIds() {
- return inProgressStages;
- }
-
- public Set<UUID> getCompletedStageIds() {
- return completedStages;
- }
-
- public Map<UUID, JobStage> getStageMap() {
- return stageMap;
- }
-
- public Map<UUID, JobStageAttempt> getStageAttemptMap() {
- return stageAttemptMap;
- }
-
- public Set<String> getParticipatingNodeIds() {
- return participatingNodeIds;
- }
-
- public JobProfile getJobProfile() {
- return profile;
- }
-
- public IJobAttemptSchedulerState getSchedulerState() {
- return schedulerState;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
deleted file mode 100644
index bafa7e5..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
+++ /dev/null
@@ -1,91 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-
-public class JobPlanBuilder implements IActivityGraphBuilder {
- private static final Logger LOGGER = Logger.getLogger(JobPlanBuilder.class.getName());
-
- private JobPlan plan;
-
- @Override
- public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked) {
- addToValueSet(plan.getBlocker2BlockedMap(), blocker.getActivityNodeId(), blocked.getActivityNodeId());
- addToValueSet(plan.getBlocked2BlockerMap(), blocked.getActivityNodeId(), blocker.getActivityNodeId());
- }
-
- @Override
- public void addSourceEdge(int operatorInputIndex, IActivityNode task, int taskInputIndex) {
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Adding source edge: " + task.getOwner().getOperatorId() + ":" + operatorInputIndex + " -> "
- + task.getActivityNodeId() + ":" + taskInputIndex);
- }
- insertIntoIndexedMap(plan.getTaskInputMap(), task.getActivityNodeId(), taskInputIndex, operatorInputIndex);
- insertIntoIndexedMap(plan.getOperatorInputMap(), task.getOwner().getOperatorId(), operatorInputIndex,
- task.getActivityNodeId());
- }
-
- @Override
- public void addTargetEdge(int operatorOutputIndex, IActivityNode task, int taskOutputIndex) {
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Adding target edge: " + task.getOwner().getOperatorId() + ":" + operatorOutputIndex + " -> "
- + task.getActivityNodeId() + ":" + taskOutputIndex);
- }
- insertIntoIndexedMap(plan.getTaskOutputMap(), task.getActivityNodeId(), taskOutputIndex, operatorOutputIndex);
- insertIntoIndexedMap(plan.getOperatorOutputMap(), task.getOwner().getOperatorId(), operatorOutputIndex,
- task.getActivityNodeId());
- }
-
- @Override
- public void addTask(IActivityNode task) {
- plan.getActivityNodeMap().put(task.getActivityNodeId(), task);
- addToValueSet(plan.getOperatorTaskMap(), task.getOwner().getOperatorId(), task.getActivityNodeId());
- }
-
- private <K, V> void addToValueSet(Map<K, Set<V>> map, K n1, V n2) {
- Set<V> targets = map.get(n1);
- if (targets == null) {
- targets = new HashSet<V>();
- map.put(n1, targets);
- }
- targets.add(n2);
- }
-
- private <T> void extend(List<T> list, int index) {
- int n = list.size();
- for (int i = n; i <= index; ++i) {
- list.add(null);
- }
- }
-
- public void init(String appName, UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
- plan = new JobPlan(appName, jobId, jobSpec, jobFlags);
- }
-
- private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
- List<V> vList = map.get(key);
- if (vList == null) {
- vList = new ArrayList<V>();
- map.put(key, vList);
- }
- extend(vList, index);
- vList.set(index, value);
- }
-
- public JobPlan getPlan() {
- return plan;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
deleted file mode 100644
index 900e5eee..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanner.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * 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.job;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.util.Pair;
-
-public class JobPlanner {
- private static final Logger LOGGER = Logger.getLogger(JobPlanner.class.getName());
-
- private Pair<ActivityNodeId, ActivityNodeId> findMergePair(JobPlan plan, JobSpecification spec, Set<JobStage> eqSets) {
- Map<ActivityNodeId, IActivityNode> activityNodeMap = plan.getActivityNodeMap();
- for (JobStage eqSet : eqSets) {
- for (ActivityNodeId t : eqSet.getTasks()) {
- IOperatorDescriptor owner = activityNodeMap.get(t).getOwner();
- List<Integer> inputList = plan.getTaskInputMap().get(t);
- if (inputList != null) {
- for (Integer idx : inputList) {
- IConnectorDescriptor conn = spec.getInputConnectorDescriptor(owner, idx);
- OperatorDescriptorId producerId = spec.getProducer(conn).getOperatorId();
- int producerOutputIndex = spec.getProducerOutputIndex(conn);
- ActivityNodeId inTask = plan.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
- if (!eqSet.getTasks().contains(inTask)) {
- return new Pair<ActivityNodeId, ActivityNodeId>(t, inTask);
- }
- }
- }
- List<Integer> outputList = plan.getTaskOutputMap().get(t);
- if (outputList != null) {
- for (Integer idx : outputList) {
- IConnectorDescriptor conn = spec.getOutputConnectorDescriptor(owner, idx);
- OperatorDescriptorId consumerId = spec.getConsumer(conn).getOperatorId();
- int consumerInputIndex = spec.getConsumerInputIndex(conn);
- ActivityNodeId outTask = plan.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
- if (!eqSet.getTasks().contains(outTask)) {
- return new Pair<ActivityNodeId, ActivityNodeId>(t, outTask);
- }
- }
- }
- }
- }
- return null;
- }
-
- private JobStage inferStages(JobPlan plan) {
- JobSpecification spec = plan.getJobSpecification();
-
- /*
- * Build initial equivalence sets map. We create a map such that for each IOperatorTask, t -> { t }
- */
- Map<ActivityNodeId, JobStage> stageMap = new HashMap<ActivityNodeId, JobStage>();
- Set<JobStage> stages = new HashSet<JobStage>();
- for (Set<ActivityNodeId> taskIds : plan.getOperatorTaskMap().values()) {
- for (ActivityNodeId taskId : taskIds) {
- Set<ActivityNodeId> eqSet = new HashSet<ActivityNodeId>();
- eqSet.add(taskId);
- JobStage stage = new JobStage(eqSet);
- stageMap.put(taskId, stage);
- stages.add(stage);
- }
- }
-
- boolean changed = true;
- while (changed) {
- changed = false;
- Pair<ActivityNodeId, ActivityNodeId> pair = findMergePair(plan, spec, stages);
- if (pair != null) {
- merge(stageMap, stages, pair.first, pair.second);
- changed = true;
- }
- }
-
- JobStage endStage = new JobStage(new HashSet<ActivityNodeId>());
- Map<ActivityNodeId, Set<ActivityNodeId>> blocker2BlockedMap = plan.getBlocker2BlockedMap();
- for (JobStage s : stages) {
- endStage.addDependency(s);
- s.addDependent(endStage);
- Set<JobStage> blockedStages = new HashSet<JobStage>();
- for (ActivityNodeId t : s.getTasks()) {
- Set<ActivityNodeId> blockedTasks = blocker2BlockedMap.get(t);
- if (blockedTasks != null) {
- for (ActivityNodeId bt : blockedTasks) {
- blockedStages.add(stageMap.get(bt));
- }
- }
- }
- for (JobStage bs : blockedStages) {
- bs.addDependency(s);
- s.addDependent(bs);
- }
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Inferred " + (stages.size() + 1) + " stages");
- for (JobStage s : stages) {
- LOGGER.info(s.toString());
- }
- LOGGER.info("SID: ENDSTAGE");
- }
- return endStage;
- }
-
- private void merge(Map<ActivityNodeId, JobStage> eqSetMap, Set<JobStage> eqSets, ActivityNodeId t1,
- ActivityNodeId t2) {
- JobStage stage1 = eqSetMap.get(t1);
- Set<ActivityNodeId> s1 = stage1.getTasks();
- JobStage stage2 = eqSetMap.get(t2);
- Set<ActivityNodeId> s2 = stage2.getTasks();
-
- Set<ActivityNodeId> mergedSet = new HashSet<ActivityNodeId>();
- mergedSet.addAll(s1);
- mergedSet.addAll(s2);
-
- eqSets.remove(stage1);
- eqSets.remove(stage2);
- JobStage mergedStage = new JobStage(mergedSet);
- eqSets.add(mergedStage);
-
- for (ActivityNodeId t : mergedSet) {
- eqSetMap.put(t, mergedStage);
- }
- }
-
- public JobStage createStageDAG(JobPlan jag) {
- JobStage endStage = inferStages(jag);
- return endStage;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 5766288..6231080 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -14,53 +14,86 @@
*/
package edu.uci.ics.hyracks.control.cc.job;
-import java.util.ArrayList;
-import java.util.List;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
import java.util.Set;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import org.json.JSONArray;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.scheduler.ActivityPartitionDetails;
+import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
public class JobRun implements IJobStatusConditionVariable {
- private final JobPlan plan;
- private final List<JobAttempt> attempts;
+ private final JobId jobId;
+
+ private final JobActivityGraph jag;
+
+ private final PartitionMatchMaker pmm;
+
+ private final Set<String> participatingNodeIds;
+
+ private final JobProfile profile;
+
+ private Set<ActivityCluster> activityClusters;
+
+ private final Map<ActivityId, ActivityCluster> activityClusterMap;
+
+ private final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap;
+
+ private JobScheduler js;
+
private JobStatus status;
- private Set<ConstraintExpression> constraints;
- public JobRun(JobPlan plan, Set<ConstraintExpression> constraints) {
- this.plan = plan;
- attempts = new ArrayList<JobAttempt>();
- this.constraints = constraints;
+ private Exception exception;
+
+ public JobRun(JobId jobId, JobActivityGraph plan) {
+ this.jobId = jobId;
+ this.jag = plan;
+ pmm = new PartitionMatchMaker();
+ participatingNodeIds = new HashSet<String>();
+ profile = new JobProfile(jobId);
+ activityClusterMap = new HashMap<ActivityId, ActivityCluster>();
+ connectorPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
}
- public JobPlan getJobPlan() {
- return plan;
+ public JobId getJobId() {
+ return jobId;
}
- public synchronized void setStatus(JobStatus status) {
+ public JobActivityGraph getJobActivityGraph() {
+ return jag;
+ }
+
+ public PartitionMatchMaker getPartitionMatchMaker() {
+ return pmm;
+ }
+
+ public synchronized void setStatus(JobStatus status, Exception exception) {
this.status = status;
+ this.exception = exception;
notifyAll();
}
- public JobStatus getStatus() {
+ public synchronized JobStatus getStatus() {
return status;
}
- public List<JobAttempt> getAttempts() {
- return attempts;
- }
-
- public Set<ConstraintExpression> getConstraints() {
- return constraints;
- }
-
- public JobAttempt createAttempt(IScheduler scheduler) {
- int attemptNumber = attempts.size();
- JobAttempt attempt = new JobAttempt(this, plan, attemptNumber, scheduler);
- attempts.add(attempt);
- return attempt;
+ public synchronized Exception getException() {
+ return exception;
}
@Override
@@ -68,5 +101,164 @@
while (status != JobStatus.TERMINATED && status != JobStatus.FAILURE) {
wait();
}
+ if (exception != null) {
+ throw new HyracksException("Job Failed", exception);
+ }
+ }
+
+ public Set<String> getParticipatingNodeIds() {
+ return participatingNodeIds;
+ }
+
+ public JobProfile getJobProfile() {
+ return profile;
+ }
+
+ public void setScheduler(JobScheduler js) {
+ this.js = js;
+ }
+
+ public JobScheduler getScheduler() {
+ return js;
+ }
+
+ public Map<ActivityId, ActivityCluster> getActivityClusterMap() {
+ return activityClusterMap;
+ }
+
+ public Set<ActivityCluster> getActivityClusters() {
+ return activityClusters;
+ }
+
+ public void setActivityClusters(Set<ActivityCluster> activityClusters) {
+ this.activityClusters = activityClusters;
+ }
+
+ public Map<ConnectorDescriptorId, IConnectorPolicy> getConnectorPolicyMap() {
+ return connectorPolicyMap;
+ }
+
+ public JSONObject toJSON() throws JSONException {
+ JSONObject result = new JSONObject();
+
+ result.put("job-id", jobId.toString());
+
+ JSONArray aClusters = new JSONArray();
+ for (ActivityCluster ac : activityClusters) {
+ JSONObject acJSON = new JSONObject();
+
+ acJSON.put("activity-cluster-id", String.valueOf(ac.getActivityClusterId()));
+
+ JSONArray activitiesJSON = new JSONArray();
+ for (ActivityId aid : ac.getActivities()) {
+ activitiesJSON.put(aid);
+ }
+ acJSON.put("activities", activitiesJSON);
+
+ JSONArray dependentsJSON = new JSONArray();
+ for (ActivityCluster dependent : ac.getDependents()) {
+ dependentsJSON.put(String.valueOf(dependent.getActivityClusterId()));
+ }
+ acJSON.put("dependents", dependentsJSON);
+
+ JSONArray dependenciesJSON = new JSONArray();
+ for (ActivityCluster dependency : ac.getDependencies()) {
+ dependenciesJSON.put(String.valueOf(dependency.getActivityClusterId()));
+ }
+ acJSON.put("dependencies", dependenciesJSON);
+
+ ActivityClusterPlan acp = ac.getPlan();
+ if (acp == null) {
+ acJSON.put("plan", (Object) null);
+ } else {
+ JSONObject planJSON = new JSONObject();
+
+ JSONArray acTasks = new JSONArray();
+ for (Map.Entry<ActivityId, ActivityPlan> e : acp.getActivityPlanMap().entrySet()) {
+ ActivityPlan acPlan = e.getValue();
+ JSONObject entry = new JSONObject();
+ entry.put("activity-id", e.getKey().toString());
+
+ ActivityPartitionDetails apd = acPlan.getActivityPartitionDetails();
+ entry.put("partition-count", apd.getPartitionCount());
+
+ JSONArray inPartCountsJSON = new JSONArray();
+ int[] inPartCounts = apd.getInputPartitionCounts();
+ if (inPartCounts != null) {
+ for (int i : inPartCounts) {
+ inPartCountsJSON.put(i);
+ }
+ }
+ entry.put("input-partition-counts", inPartCountsJSON);
+
+ JSONArray outPartCountsJSON = new JSONArray();
+ int[] outPartCounts = apd.getOutputPartitionCounts();
+ if (outPartCounts != null) {
+ for (int o : outPartCounts) {
+ outPartCountsJSON.put(o);
+ }
+ }
+ entry.put("output-partition-counts", outPartCountsJSON);
+
+ JSONArray tasks = new JSONArray();
+ for (Task t : acPlan.getTasks()) {
+ JSONObject task = new JSONObject();
+
+ task.put("task-id", t.getTaskId().toString());
+
+ JSONArray dependentTasksJSON = new JSONArray();
+ for (TaskId dependent : t.getDependents()) {
+ dependentTasksJSON.put(dependent.toString());
+ }
+ task.put("dependents", dependentTasksJSON);
+
+ JSONArray dependencyTasksJSON = new JSONArray();
+ for (TaskId dependency : t.getDependencies()) {
+ dependencyTasksJSON.put(dependency.toString());
+ }
+ task.put("dependencies", dependencyTasksJSON);
+
+ tasks.put(task);
+ }
+ entry.put("tasks", tasks);
+
+ acTasks.put(entry);
+ }
+ planJSON.put("task-map", acTasks);
+
+ JSONArray tClusters = new JSONArray();
+ for (TaskCluster tc : acp.getTaskClusters()) {
+ JSONObject c = new JSONObject();
+ c.put("task-cluster-id", String.valueOf(tc.getTaskClusterId()));
+
+ JSONArray tasks = new JSONArray();
+ for (Task t : tc.getTasks()) {
+ tasks.put(t.getTaskId().toString());
+ }
+ c.put("tasks", tasks);
+
+ JSONArray prodParts = new JSONArray();
+ for (PartitionId p : tc.getProducedPartitions()) {
+ prodParts.put(p.toString());
+ }
+ c.put("produced-partitions", prodParts);
+
+ JSONArray reqdParts = new JSONArray();
+ for (PartitionId p : tc.getRequiredPartitions()) {
+ reqdParts.put(p.toString());
+ }
+ c.put("required-partitions", reqdParts);
+
+ tClusters.put(c);
+ }
+ planJSON.put("task-clusters", tClusters);
+
+ acJSON.put("plan", planJSON);
+ }
+ aClusters.put(acJSON);
+ }
+ result.put("activity-clusters", aClusters);
+
+ return result;
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
deleted file mode 100644
index 8e4013d..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStage.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * 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.job;
-
-import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-
-public class JobStage implements Serializable {
- private static final long serialVersionUID = 1L;
-
- private final UUID id;
-
- private final Set<ActivityNodeId> tasks;
-
- private final Set<JobStage> dependencies;
-
- private final Set<JobStage> dependents;
-
- private boolean started;
-
- public JobStage(Set<ActivityNodeId> tasks) {
- this.id = UUID.randomUUID();
- this.tasks = tasks;
- dependencies = new HashSet<JobStage>();
- dependents = new HashSet<JobStage>();
- }
-
- public UUID getId() {
- return id;
- }
-
- public Set<ActivityNodeId> getTasks() {
- return tasks;
- }
-
- public void addDependency(JobStage stage) {
- dependencies.add(stage);
- }
-
- public void addDependent(JobStage stage) {
- dependents.add(stage);
- }
-
- public Set<JobStage> getDependencies() {
- return dependencies;
- }
-
- @Override
- public int hashCode() {
- return id == null ? 0 : id.hashCode();
- }
-
- @Override
- public boolean equals(Object o) {
- if (!(o instanceof JobStage)) {
- return false;
- }
- return id == ((JobStage) o).id;
- }
-
- @Override
- public String toString() {
- return "SID:" + id + ": " + tasks;
- }
-
- public boolean isStarted() {
- return started;
- }
-
- public void setStarted() {
- started = true;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java
deleted file mode 100644
index 61db770..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobStageAttempt.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.job;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-
-public class JobStageAttempt {
- private final JobStage stage;
-
- private final JobAttempt jobAttempt;
-
- private final Set<String> participatingNodes;
-
- private final Set<String> completedNodes;
-
- private ISchedule schedule;
-
- public JobStageAttempt(JobStage stage, JobAttempt jobAttempt) {
- this.stage = stage;
- this.jobAttempt = jobAttempt;
- participatingNodes = new HashSet<String>();
- completedNodes = new HashSet<String>();
- }
-
- public JobStage getJobStage() {
- return stage;
- }
-
- public JobAttempt getJobAttempt() {
- return jobAttempt;
- }
-
- public void setSchedule(ISchedule schedule) {
- this.schedule = schedule;
- for (ActivityNodeId aid : stage.getTasks()) {
- String[] partitions = schedule.getPartitions(aid);
- for (String nid : partitions) {
- participatingNodes.add(nid);
- }
- }
- }
-
- public ISchedule getSchedule() {
- return schedule;
- }
-
- public Set<String> getParticipatingNodes() {
- return participatingNodes;
- }
-
- public Set<String> getCompletedNodes() {
- return completedNodes;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.java
new file mode 100644
index 0000000..84ae6e5
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/Task.java
@@ -0,0 +1,68 @@
+/*
+ * 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.job;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+
+public class Task {
+ private final TaskId taskId;
+
+ private final ActivityPlan activityPlan;
+
+ private final Set<TaskId> dependencies;
+
+ private final Set<TaskId> dependents;
+
+ private TaskCluster taskCluster;
+
+ public Task(TaskId taskId, ActivityPlan activityPlan) {
+ this.taskId = taskId;
+ this.activityPlan = activityPlan;
+ this.dependencies = new HashSet<TaskId>();
+ this.dependents = new HashSet<TaskId>();
+ }
+
+ public TaskId getTaskId() {
+ return taskId;
+ }
+
+ public ActivityPlan getActivityPlan() {
+ return activityPlan;
+ }
+
+ public Set<TaskId> getDependencies() {
+ return dependencies;
+ }
+
+ public Set<TaskId> getDependents() {
+ return dependents;
+ }
+
+ public TaskCluster getTaskCluster() {
+ return taskCluster;
+ }
+
+ public void setTaskCluster(TaskCluster taskCluster) {
+ this.taskCluster = taskCluster;
+ }
+
+ @Override
+ public String toString() {
+ return String.valueOf(taskId);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
new file mode 100644
index 0000000..22f5587
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -0,0 +1,78 @@
+/*
+ * 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.job;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskAttempt {
+ public enum TaskStatus {
+ INITIALIZED,
+ RUNNING,
+ COMPLETED,
+ FAILED,
+ ABORTED,
+ }
+
+ private final TaskClusterAttempt tcAttempt;
+
+ private final TaskAttemptId taskId;
+
+ private final Task taskState;
+
+ private String nodeId;
+
+ private TaskStatus status;
+
+ private Exception exception;
+
+ public TaskAttempt(TaskClusterAttempt tcAttempt, TaskAttemptId taskId, Task taskState) {
+ this.tcAttempt = tcAttempt;
+ this.taskId = taskId;
+ this.taskState = taskState;
+ }
+
+ public TaskClusterAttempt getTaskClusterAttempt() {
+ return tcAttempt;
+ }
+
+ public TaskAttemptId getTaskAttemptId() {
+ return taskId;
+ }
+
+ public Task getTaskState() {
+ return taskState;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public void setNodeId(String nodeId) {
+ this.nodeId = nodeId;
+ }
+
+ public TaskStatus getStatus() {
+ return status;
+ }
+
+ public Exception getException() {
+ return exception;
+ }
+
+ public void setStatus(TaskStatus status, Exception exception) {
+ this.status = status;
+ this.exception = exception;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java
new file mode 100644
index 0000000..f41cda9
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java
@@ -0,0 +1,89 @@
+/*
+ * 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.job;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class TaskCluster {
+ private final TaskClusterId taskClusterId;
+
+ private final ActivityCluster ac;
+
+ private final Task[] tasks;
+
+ private final Set<PartitionId> producedPartitions;
+
+ private final Set<PartitionId> requiredPartitions;
+
+ private final Set<TaskCluster> dependencyTaskClusters;
+
+ private final Set<TaskCluster> dependentTaskClusters;
+
+ private final List<TaskClusterAttempt> taskClusterAttempts;
+
+ public TaskCluster(TaskClusterId taskClusterId, ActivityCluster ac, Task[] tasks) {
+ this.taskClusterId = taskClusterId;
+ this.ac = ac;
+ this.tasks = tasks;
+ producedPartitions = new HashSet<PartitionId>();
+ requiredPartitions = new HashSet<PartitionId>();
+ dependencyTaskClusters = new HashSet<TaskCluster>();
+ dependentTaskClusters = new HashSet<TaskCluster>();
+ taskClusterAttempts = new ArrayList<TaskClusterAttempt>();
+ }
+
+ public TaskClusterId getTaskClusterId() {
+ return taskClusterId;
+ }
+
+ public ActivityCluster getActivityCluster() {
+ return ac;
+ }
+
+ public Task[] getTasks() {
+ return tasks;
+ }
+
+ public Set<PartitionId> getProducedPartitions() {
+ return producedPartitions;
+ }
+
+ public Set<PartitionId> getRequiredPartitions() {
+ return requiredPartitions;
+ }
+
+ public Set<TaskCluster> getDependencyTaskClusters() {
+ return dependencyTaskClusters;
+ }
+
+ public Set<TaskCluster> getDependentTaskClusters() {
+ return dependentTaskClusters;
+ }
+
+ public List<TaskClusterAttempt> getAttempts() {
+ return taskClusterAttempts;
+ }
+
+ @Override
+ public String toString() {
+ return "TC:" + Arrays.toString(tasks);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
new file mode 100644
index 0000000..9d33d12
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
@@ -0,0 +1,75 @@
+/*
+ * 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.job;
+
+public class TaskClusterAttempt {
+ public enum TaskClusterStatus {
+ RUNNING,
+ COMPLETED,
+ FAILED,
+ ABORTED,
+ }
+
+ private final TaskCluster taskCluster;
+
+ private final int attempt;
+
+ private TaskAttempt[] taskAttempts;
+
+ private TaskClusterStatus status;
+
+ private int pendingTaskCounter;
+
+ public TaskClusterAttempt(TaskCluster taskCluster, int attempt) {
+ this.taskCluster = taskCluster;
+ this.attempt = attempt;
+ }
+
+ public TaskCluster getTaskCluster() {
+ return taskCluster;
+ }
+
+ public void setTaskAttempts(TaskAttempt[] taskAttempts) {
+ this.taskAttempts = taskAttempts;
+ }
+
+ public TaskAttempt[] getTaskAttempts() {
+ return taskAttempts;
+ }
+
+ public int getAttempt() {
+ return attempt;
+ }
+
+ public void setStatus(TaskClusterStatus status) {
+ this.status = status;
+ }
+
+ public TaskClusterStatus getStatus() {
+ return status;
+ }
+
+ public void initializePendingTaskCounter() {
+ pendingTaskCounter = taskAttempts.length;
+ }
+
+ public int getPendingTaskCounter() {
+ return pendingTaskCounter;
+ }
+
+ public int decrementPendingTasksCounter() {
+ return --pendingTaskCounter;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.java
new file mode 100644
index 0000000..b1000dd
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterId.java
@@ -0,0 +1,71 @@
+/*
+ * 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.job;
+
+import java.io.Serializable;
+
+public final class TaskClusterId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityClusterId activityClusterId;
+
+ private final int id;
+
+ public TaskClusterId(ActivityClusterId activityClusterId, int id) {
+ this.activityClusterId = activityClusterId;
+ this.id = id;
+ }
+
+ public ActivityClusterId getActivityClusterId() {
+ return activityClusterId;
+ }
+
+ public int getId() {
+ return id;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((activityClusterId == null) ? 0 : activityClusterId.hashCode());
+ result = prime * result + id;
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ TaskClusterId other = (TaskClusterId) obj;
+ if (activityClusterId == null) {
+ if (other.activityClusterId != null)
+ return false;
+ } else if (!activityClusterId.equals(other.activityClusterId))
+ return false;
+ if (id != other.id)
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return "TC:" + activityClusterId + ":" + id;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
deleted file mode 100644
index ef87c4f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManager.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.job.manager;
-
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-
-public interface IJobManager {
- public void createJob(UUID jobId, String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags)
- throws Exception;
-
- public void start(UUID jobId) throws Exception;
-
- public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId,
- Map<String, Long> statistics) throws Exception;
-
- public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
-
- public JobStatus getJobStatus(UUID jobId);
-
- public void waitForCompletion(UUID jobId) throws Exception;
-
- public void notifyNodeFailure(String nodeId) throws Exception;
-
- public void registerNode(String nodeId) throws Exception;
-
- public void reportProfile(String id, Map<UUID, Map<String, Long>> counterDump) throws Exception;
-
- public IJobManagerQueryInterface getQueryInterface();
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
deleted file mode 100644
index fbdf941..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/IJobManagerQueryInterface.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * 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.job.manager;
-
-import java.util.UUID;
-
-import org.json.JSONArray;
-import org.json.JSONObject;
-
-public interface IJobManagerQueryInterface {
- public JSONArray getAllJobSummaries() throws Exception;
-
- public JSONObject getJobSpecification(UUID jobId) throws Exception;
-
- public JSONObject getJobPlan(UUID jobId) throws Exception;
-
- public JSONObject getJobProfile(UUID jobId, int attempt) throws Exception;
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
deleted file mode 100644
index be42e22..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/JobLifecycleHelper.java
+++ /dev/null
@@ -1,27 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.job.manager;
-
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.manager.events.JobAttemptStartEvent;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.JobletAborter;
-
-public class JobLifecycleHelper {
- public static void abortJob(ClusterControllerService ccs, UUID jobId, int attempt, Set<String> targetNodes) {
- if (!targetNodes.isEmpty()) {
- JobletAborter[] jas = new JobletAborter[targetNodes.size()];
- int i = 0;
- for (String nodeId : targetNodes) {
- jas[i++] = new JobletAborter(nodeId, jobId, attempt);
- }
- try {
- RemoteRunner.runRemote(ccs, jas, null);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- ccs.getJobQueue().schedule(new JobAttemptStartEvent(ccs, jobId));
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.java
new file mode 100644
index 0000000..0028b6f
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/AbstractTaskLifecycleEvent.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.control.cc.job.manager.events;
+
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+
+public abstract class AbstractTaskLifecycleEvent extends AbstractEvent {
+ protected final ClusterControllerService ccs;
+ protected final JobId jobId;
+ protected final TaskAttemptId taId;
+ protected final String nodeId;
+
+ public AbstractTaskLifecycleEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.taId = taId;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public final void run() {
+ JobRun run = ccs.getRunMap().get(jobId);
+ if (run != null) {
+ TaskId tid = taId.getTaskId();
+ Map<ActivityId, ActivityCluster> activityClusterMap = run.getActivityClusterMap();
+ ActivityCluster ac = activityClusterMap.get(tid.getActivityId());
+ if (ac != null) {
+ Map<ActivityId, ActivityPlan> taskStateMap = ac.getPlan().getActivityPlanMap();
+ Task[] taskStates = taskStateMap.get(tid.getActivityId()).getTasks();
+ if (taskStates != null && taskStates.length > tid.getPartition()) {
+ Task ts = taskStates[tid.getPartition()];
+ TaskCluster tc = ts.getTaskCluster();
+ List<TaskClusterAttempt> taskClusterAttempts = tc.getAttempts();
+ if (taskClusterAttempts != null && taskClusterAttempts.size() > taId.getAttempt()) {
+ TaskClusterAttempt tca = taskClusterAttempts.get(taId.getAttempt());
+ for (TaskAttempt ta : tca.getTaskAttempts()) {
+ if (ta.getTaskAttemptId().equals(taId)) {
+ performEvent(ta);
+ break;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ protected abstract void performEvent(TaskAttempt ta);
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
index 69c244a..050c6ed 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationDestroyEvent.java
@@ -19,13 +19,14 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationDestroyer;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-public class ApplicationDestroyEvent implements Runnable {
+public class ApplicationDestroyEvent extends AbstractEvent {
private final ClusterControllerService ccs;
private final String appName;
private FutureValue fv;
@@ -57,7 +58,7 @@
fv.setException(e);
return;
}
- ccs.getJobQueue().schedule(new Runnable() {
+ ccs.getJobQueue().schedule(new AbstractEvent() {
@Override
public void run() {
try {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
index 9f4ad8f..ab05d0f 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ApplicationStartEvent.java
@@ -20,13 +20,14 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
import edu.uci.ics.hyracks.control.cc.jobqueue.FutureValue;
import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
import edu.uci.ics.hyracks.control.cc.remote.ops.ApplicationStarter;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
-public class ApplicationStartEvent implements Runnable {
+public class ApplicationStartEvent extends AbstractEvent {
private final ClusterControllerService ccs;
private final String appName;
private final FutureValue fv;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.java
new file mode 100644
index 0000000..14b56df
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobActivityGraphJSONEvent.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.job.manager.events;
+
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+
+public class GetJobActivityGraphJSONEvent extends SynchronizableEvent {
+ private final ClusterControllerService ccs;
+ private final JobId jobId;
+ private JSONObject json;
+
+ public GetJobActivityGraphJSONEvent(ClusterControllerService ccs, JobId jobId) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ }
+
+ @Override
+ protected void doRun() throws Exception {
+ JobRun run = ccs.getRunMap().get(jobId);
+ if (run == null) {
+ json = new JSONObject();
+ return;
+ }
+ json = run.getJobActivityGraph().toJSON();
+ }
+
+ public JSONObject getJSON() {
+ return json;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
index dc397ef..4355962 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
@@ -14,44 +14,35 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.List;
-import java.util.UUID;
-
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetJobProfileJSONEvent extends SynchronizableRunnable {
+public class GetJobProfileJSONEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
- private final UUID jobId;
- private final int attempt;
- private JSONObject profile;
+ private final JobId jobId;
+ private JSONObject json;
- public GetJobProfileJSONEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
+ public GetJobProfileJSONEvent(ClusterControllerService ccs, JobId jobId) {
this.ccs = ccs;
this.jobId = jobId;
- this.attempt = attempt;
}
@Override
protected void doRun() throws Exception {
- JobRun run = ccs.getRunMap().get(jobId);
- profile = new JSONObject();
- if (run == null) {
+ json = new JSONObject();
+ JobRun jobRun = ccs.getRunMap().get(jobId);
+ if (jobRun == null) {
+ json = new JSONObject();
return;
}
- List<JobAttempt> attempts = run.getAttempts();
- if (attempts.size() <= attempt) {
- return;
- }
- JobAttempt ja = attempts.get(attempt);
- profile = ja.getJobProfile().toJSON();
+ json = jobRun.getJobProfile().toJSON();
}
- public JSONObject getProfile() {
- return profile;
+ public JSONObject getJSON() {
+ return json;
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
index 6248925..db7e5a7 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSpecificationJSONEvent.java
@@ -14,20 +14,19 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.UUID;
-
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetJobSpecificationJSONEvent extends SynchronizableRunnable {
+public class GetJobSpecificationJSONEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
- private final UUID jobId;
- private JSONObject spec;
+ private final JobId jobId;
+ private JSONObject json;
- public GetJobSpecificationJSONEvent(ClusterControllerService ccs, UUID jobId) {
+ public GetJobSpecificationJSONEvent(ClusterControllerService ccs, JobId jobId) {
this.ccs = ccs;
this.jobId = jobId;
}
@@ -36,13 +35,13 @@
protected void doRun() throws Exception {
JobRun run = ccs.getRunMap().get(jobId);
if (run == null) {
- spec = new JSONObject();
+ json = new JSONObject();
return;
}
- spec = run.getJobPlan().getJobSpecification().toJSON();
+ json = run.getJobActivityGraph().getJobSpecification().toJSON();
}
- public JSONObject getSpecification() {
- return spec;
+ public JSONObject getJSON() {
+ return json;
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
index c80974a..357d62b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusConditionVariableEvent.java
@@ -14,18 +14,17 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.IJobStatusConditionVariable;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetJobStatusConditionVariableEvent extends SynchronizableRunnable {
+public class GetJobStatusConditionVariableEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
- private final UUID jobId;
+ private final JobId jobId;
private IJobStatusConditionVariable cVar;
- public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, UUID jobId) {
+ public GetJobStatusConditionVariableEvent(ClusterControllerService ccs, JobId jobId) {
this.ccs = ccs;
this.jobId = jobId;
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
index 9b15e5c..fbe6e4c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobStatusEvent.java
@@ -14,19 +14,18 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetJobStatusEvent extends SynchronizableRunnable {
+public class GetJobStatusEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
- private final UUID jobId;
+ private final JobId jobId;
private JobStatus status;
- public GetJobStatusEvent(ClusterControllerService ccs, UUID jobId) {
+ public GetJobStatusEvent(ClusterControllerService ccs, JobId jobId) {
this.ccs = ccs;
this.jobId = jobId;
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
index 74851e5..bf5e5c3 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobSummariesJSONEvent.java
@@ -19,9 +19,9 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetJobSummariesJSONEvent extends SynchronizableRunnable {
+public class GetJobSummariesJSONEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private JSONArray summaries;
@@ -35,8 +35,7 @@
for (JobRun run : ccs.getRunMap().values()) {
JSONObject jo = new JSONObject();
jo.put("type", "job-summary");
- jo.put("id", run.getJobPlan().getJobId().toString());
- jo.put("attempts", run.getAttempts().size());
+ jo.put("id", run.getJobId().toString());
jo.put("status", run.getStatus().toString());
summaries.put(jo);
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
index 7554e62..0db7f2a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetNodeEvent.java
@@ -16,9 +16,9 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class GetNodeEvent extends SynchronizableRunnable {
+public class GetNodeEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private final String nodeId;
private NodeControllerState state;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
deleted file mode 100644
index 9a0eff1..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAbortEvent.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.job.manager.events;
-
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.manager.JobLifecycleHelper;
-
-public class JobAbortEvent implements Runnable {
- private final ClusterControllerService ccs;
-
- private final UUID jobId;
-
- private final int attempt;
-
- public JobAbortEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
- this.ccs = ccs;
- this.jobId = jobId;
- this.attempt = attempt;
- }
-
- @Override
- public void run() {
- Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
- Map<UUID, JobRun> runMap = ccs.getRunMap();
- final JobRun run = runMap.get(jobId);
- final Set<String> targetNodes = new HashSet<String>();
- if (run != null) {
- List<JobAttempt> jas = run.getAttempts();
- JobAttempt ja = jas.get(attempt);
- for (String runningNodeId : ja.getParticipatingNodeIds()) {
- if (nodeMap.containsKey(runningNodeId)) {
- targetNodes.add(runningNodeId);
- nodeMap.get(runningNodeId).getActiveJobIds().remove(jobId);
- }
- }
- }
-
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
- }
- });
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java
deleted file mode 100644
index 7dc6496..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobAttemptStartEvent.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.job.manager.events;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-
-public class JobAttemptStartEvent implements Runnable {
- private ClusterControllerService ccs;
- private UUID jobId;
-
- public JobAttemptStartEvent(ClusterControllerService ccs, UUID jobId) {
- this.ccs = ccs;
- this.jobId = jobId;
- }
-
- @Override
- public void run() {
- JobRun run = ccs.getRunMap().get(jobId);
- int maxAttempts = run.getJobPlan().getJobSpecification().getMaxAttempts();
- if (maxAttempts == 0) {
- maxAttempts = ccs.getConfig().defaultMaxJobAttempts;
- }
- if (run.getAttempts().size() > maxAttempts) {
- run.setStatus(JobStatus.FAILURE);
- return;
- }
- JobAttempt attempt = run.createAttempt(ccs.getScheduler());
- new ScheduleRunnableStagesEvent(ccs, jobId, attempt.getAttempt()).run();
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
index c4ed077..4274914 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCleanupEvent.java
@@ -14,42 +14,44 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.Map;
import java.util.Set;
-import java.util.UUID;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
-public class JobCleanupEvent implements Runnable {
+public class JobCleanupEvent extends AbstractEvent {
private ClusterControllerService ccs;
- private UUID jobId;
- private int attempt;
+ private JobId jobId;
private JobStatus status;
+ private Exception exception;
- public JobCleanupEvent(ClusterControllerService ccs, UUID jobId, int attempt, JobStatus status) {
+ public JobCleanupEvent(ClusterControllerService ccs, JobId jobId, JobStatus status, Exception exception) {
this.ccs = ccs;
this.jobId = jobId;
- this.attempt = attempt;
this.status = status;
+ this.exception = exception;
}
@Override
public void run() {
final JobRun run = ccs.getRunMap().get(jobId);
- final JobAttempt ja = run.getAttempts().get(attempt);
- Set<String> targetNodes = ja.getParticipatingNodeIds();
+ Set<String> targetNodes = run.getParticipatingNodeIds();
final JobCompleteNotifier[] jcns = new JobCompleteNotifier[targetNodes.size()];
int i = 0;
for (String n : targetNodes) {
jcns[i++] = new JobCompleteNotifier(n, jobId);
+ NodeControllerState ncs = ccs.getNodeMap().get(n);
+ if (ncs != null) {
+ ncs.getActiveJobIds().remove(jobId);
+ }
}
ccs.getExecutor().execute(new Runnable() {
@Override
@@ -61,10 +63,11 @@
e.printStackTrace();
}
}
- ccs.getJobQueue().schedule(new Runnable() {
+ ccs.getJobQueue().schedule(new AbstractEvent() {
@Override
public void run() {
- CCApplicationContext appCtx = ccs.getApplicationMap().get(ja.getPlan().getApplicationName());
+ CCApplicationContext appCtx = ccs.getApplicationMap().get(
+ run.getJobActivityGraph().getApplicationName());
if (appCtx != null) {
try {
appCtx.notifyJobFinish(jobId);
@@ -72,12 +75,7 @@
e.printStackTrace();
}
}
- Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
- for (String nodeId : ja.getParticipatingNodeIds()) {
- NodeControllerState state = nodeMap.get(nodeId);
- state.getActiveJobIds().remove(jobId);
- }
- run.setStatus(status);
+ run.setStatus(status, exception);
}
});
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
index d7f244e..5fadb00 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobCreateEvent.java
@@ -15,36 +15,31 @@
package edu.uci.ics.hyracks.control.cc.job.manager.events;
import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
-import edu.uci.ics.hyracks.control.cc.job.JobPlanBuilder;
+import edu.uci.ics.hyracks.control.cc.job.JobActivityGraphBuilder;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
+import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
-public class JobCreateEvent extends SynchronizableRunnable {
+public class JobCreateEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private final byte[] jobSpec;
private final EnumSet<JobFlag> jobFlags;
- private final UUID jobId;
+ private final JobId jobId;
private final String appName;
- public JobCreateEvent(ClusterControllerService ccs, UUID jobId, String appName, byte[] jobSpec,
+ public JobCreateEvent(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
EnumSet<JobFlag> jobFlags) {
this.jobId = jobId;
this.ccs = ccs;
@@ -59,53 +54,29 @@
if (appCtx == null) {
throw new HyracksException("No application with id " + appName + " found");
}
- JobSpecification spec = appCtx.createJobSpecification(jobId, jobSpec);
- JobRun run = plan(jobId, spec, appCtx, jobFlags);
- run.setStatus(JobStatus.INITIALIZED);
+ JobSpecification spec = appCtx.createJobSpecification(jobSpec);
+
+ final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
+ builder.init(appName, spec, jobFlags);
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+ @Override
+ public void visit(IOperatorDescriptor op) {
+ op.contributeActivities(builder);
+ }
+ });
+ final JobActivityGraph jag = builder.getActivityGraph();
+
+ JobRun run = new JobRun(jobId, jag);
+
+ run.setStatus(JobStatus.INITIALIZED, null);
ccs.getRunMap().put(jobId, run);
+ JobScheduler jrs = new JobScheduler(ccs, run);
+ run.setScheduler(jrs);
appCtx.notifyJobCreation(jobId, spec);
}
- public UUID getJobId() {
+ public JobId getJobId() {
return jobId;
}
-
- private JobRun plan(UUID jobId, JobSpecification jobSpec, final CCApplicationContext appCtx,
- EnumSet<JobFlag> jobFlags) throws Exception {
- final JobPlanBuilder builder = new JobPlanBuilder();
- builder.init(appName, jobId, jobSpec, jobFlags);
- PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
- @Override
- public void visit(IOperatorDescriptor op) {
- op.contributeTaskGraph(builder);
- }
- });
- final JobPlan plan = builder.getPlan();
-
- final Set<ConstraintExpression> contributedConstraints = new HashSet<ConstraintExpression>();
- final IConstraintExpressionAcceptor acceptor = new IConstraintExpressionAcceptor() {
- @Override
- public void addConstraintExpression(ConstraintExpression constraintExpression) {
- contributedConstraints.add(constraintExpression);
- }
- };
- PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
- @Override
- public void visit(IOperatorDescriptor op) {
- op.contributeSchedulingConstraints(acceptor, plan, appCtx);
- }
- });
- PlanUtils.visit(jobSpec, new IConnectorDescriptorVisitor() {
- @Override
- public void visit(IConnectorDescriptor conn) {
- conn.contributeSchedulingConstraints(acceptor, plan, appCtx);
- }
- });
- contributedConstraints.addAll(jobSpec.getUserConstraints());
-
- JobRun run = new JobRun(plan, contributedConstraints);
-
- return run;
- }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
index 742885e..8f6357c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
@@ -14,18 +14,17 @@
*/
package edu.uci.ics.hyracks.control.cc.job.manager.events;
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class JobStartEvent extends SynchronizableRunnable {
+public class JobStartEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
- private final UUID jobId;
+ private final JobId jobId;
- public JobStartEvent(ClusterControllerService ccs, UUID jobId) {
+ public JobStartEvent(ClusterControllerService ccs, JobId jobId) {
this.ccs = ccs;
this.jobId = jobId;
}
@@ -39,8 +38,11 @@
if (run.getStatus() != JobStatus.INITIALIZED) {
throw new Exception("Job already started");
}
- run.setStatus(JobStatus.RUNNING);
-
- new JobAttemptStartEvent(ccs, jobId).run();
+ run.setStatus(JobStatus.RUNNING, null);
+ try {
+ run.getScheduler().startJob();
+ } catch (Exception e) {
+ ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, run.getJobId(), JobStatus.FAILURE, e));
+ }
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
index 348d703..8643e6a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/NodeHeartbeatEvent.java
@@ -15,12 +15,13 @@
package edu.uci.ics.hyracks.control.cc.job.manager.events;
import java.util.Map;
+import java.util.logging.Level;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class NodeHeartbeatEvent extends SynchronizableRunnable {
+public class NodeHeartbeatEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private final String nodeId;
@@ -37,4 +38,9 @@
state.notifyHeartbeat();
}
}
+
+ @Override
+ public Level logLevel() {
+ return Level.FINEST;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
index 3df302f..5f9377b 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterNodeEvent.java
@@ -20,9 +20,9 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class RegisterNodeEvent extends SynchronizableRunnable {
+public class RegisterNodeEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private final String nodeId;
private final NodeControllerState state;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java
new file mode 100644
index 0000000..3f49b97
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.java
@@ -0,0 +1,57 @@
+/*
+ * 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.job.manager.events;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class RegisterPartitionAvailibilityEvent extends AbstractEvent {
+ private final ClusterControllerService ccs;
+ private final PartitionDescriptor partitionDescriptor;
+
+ public RegisterPartitionAvailibilityEvent(ClusterControllerService ccs, PartitionDescriptor partitionDescriptor) {
+ this.ccs = ccs;
+ this.partitionDescriptor = partitionDescriptor;
+ }
+
+ @Override
+ public void run() {
+ final PartitionId pid = partitionDescriptor.getPartitionId();
+ JobRun run = ccs.getRunMap().get(pid.getJobId());
+ if (run == null) {
+ return;
+ }
+ PartitionMatchMaker pmm = run.getPartitionMatchMaker();
+ List<Pair<PartitionDescriptor, PartitionRequest>> matches = pmm
+ .registerPartitionDescriptor(partitionDescriptor);
+ for (Pair<PartitionDescriptor, PartitionRequest> match : matches) {
+ PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "PartitionAvailable@" + partitionDescriptor;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.java
new file mode 100644
index 0000000..bac8dd1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.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.control.cc.job.manager.events;
+
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionUtils;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class RegisterPartitionRequestEvent extends AbstractEvent {
+ private final ClusterControllerService ccs;
+ private final PartitionRequest partitionRequest;
+
+ public RegisterPartitionRequestEvent(ClusterControllerService ccs, PartitionRequest partitionRequest) {
+ this.ccs = ccs;
+ this.partitionRequest = partitionRequest;
+ }
+
+ @Override
+ public void run() {
+ PartitionId pid = partitionRequest.getPartitionId();
+ JobRun run = ccs.getRunMap().get(pid.getJobId());
+ if (run == null) {
+ return;
+ }
+ PartitionMatchMaker pmm = run.getPartitionMatchMaker();
+ Pair<PartitionDescriptor, PartitionRequest> match = pmm.matchPartitionRequest(partitionRequest);
+ if (match != null) {
+ PartitionUtils.reportPartitionMatch(ccs, pid, match);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "PartitionRequest@" + partitionRequest;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
index dca0dee..b812594 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
@@ -17,14 +17,16 @@
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
-import java.util.UUID;
+import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
-public class RemoveDeadNodesEvent implements Runnable {
+public class RemoveDeadNodesEvent extends AbstractEvent {
private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesEvent.class.getName());
private final ClusterControllerService ccs;
@@ -44,15 +46,14 @@
LOGGER.info(e.getKey() + " considered dead");
}
}
+ Set<JobId> affectedJobIds = new HashSet<JobId>();
Map<String, Set<String>> ipAddressNodeNameMap = ccs.getIPAddressNodeNameMap();
for (String deadNode : deadNodes) {
NodeControllerState state = nodeMap.remove(deadNode);
- for (final UUID jid : state.getActiveJobIds()) {
- JobRun run = ccs.getRunMap().get(jid);
- int lastAttempt = run.getAttempts().size() - 1;
- LOGGER.info("Aborting: " + jid);
- ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jid, lastAttempt));
- }
+
+ // Deal with dead tasks.
+ affectedJobIds.addAll(state.getActiveJobIds());
+
String ipAddress = state.getNCConfig().dataIPAddress;
Set<String> ipNodes = ipAddressNodeNameMap.get(ipAddress);
if (ipNodes != null) {
@@ -61,5 +62,19 @@
}
}
}
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Number of affected jobs: " + affectedJobIds.size());
+ }
+ for (JobId jobId : affectedJobIds) {
+ JobRun run = ccs.getRunMap().get(jobId);
+ if (run != null) {
+ run.getScheduler().notifyNodeFailures(deadNodes);
+ }
+ }
+ }
+
+ @Override
+ public Level logLevel() {
+ return Level.FINE;
}
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
index 21c16c8..bd82f75 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ReportProfilesEvent.java
@@ -16,14 +16,15 @@
import java.util.List;
import java.util.Map;
-import java.util.UUID;
+import java.util.logging.Level;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.jobqueue.AbstractEvent;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-public class ReportProfilesEvent implements Runnable {
+public class ReportProfilesEvent extends AbstractEvent {
private final ClusterControllerService ccs;
private final List<JobProfile> profiles;
@@ -34,13 +35,14 @@
@Override
public void run() {
- Map<UUID, JobRun> runMap = ccs.getRunMap();
+ Map<JobId, JobRun> runMap = ccs.getRunMap();
for (JobProfile profile : profiles) {
JobRun run = runMap.get(profile.getJobId());
- if (run != null) {
- JobAttempt ja = run.getAttempts().get(profile.getAttempt());
- ja.getJobProfile().merge(profile);
- }
}
}
+
+ @Override
+ public Level logLevel() {
+ return Level.FINEST;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
deleted file mode 100644
index ba1c711..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/ScheduleRunnableStagesEvent.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * 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.job.manager.events;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.JobStage;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteRunner;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase1Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase2Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.Phase3Installer;
-import edu.uci.ics.hyracks.control.cc.remote.ops.PortMapMergingAccumulator;
-import edu.uci.ics.hyracks.control.cc.remote.ops.StageStarter;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-
-public class ScheduleRunnableStagesEvent implements Runnable {
- private static final Logger LOGGER = Logger.getLogger(ScheduleRunnableStagesEvent.class.getName());
-
- private ClusterControllerService ccs;
- private UUID jobId;
- private int attempt;
-
- public ScheduleRunnableStagesEvent(ClusterControllerService ccs, UUID jobId, int attempt) {
- this.ccs = ccs;
- this.jobId = jobId;
- this.attempt = attempt;
- }
-
- @Override
- public void run() {
- JobRun run = ccs.getRunMap().get(jobId);
- JobAttempt ja = run.getAttempts().get(attempt);
- Set<UUID> pendingStages = ja.getPendingStageIds();
- Set<UUID> scheduledStages = ja.getInProgressStageIds();
-
- LOGGER.info(jobId + ":" + attempt + ":Pending stages: " + pendingStages + " Scheduled stages: "
- + scheduledStages);
- if (pendingStages.size() == 1 && scheduledStages.isEmpty()) {
- LOGGER.info(jobId + ":" + attempt + ":No more runnable stages");
- ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobId, attempt, JobStatus.TERMINATED));
- return;
- }
-
- Map<UUID, JobStageAttempt> stageAttemptMap = ja.getStageAttemptMap();
-
- Set<JobStage> runnableStages = new HashSet<JobStage>();
- ja.findRunnableStages(runnableStages);
- LOGGER.info(jobId + ":" + attempt + ": Found " + runnableStages.size() + " runnable stages");
-
- Set<JobStageAttempt> runnableStageAttempts = new HashSet<JobStageAttempt>();
- for (JobStage rs : runnableStages) {
- UUID stageId = rs.getId();
- LOGGER.info("Runnable Stage: " + jobId + ":" + rs.getId());
- pendingStages.remove(stageId);
- scheduledStages.add(stageId);
- JobStageAttempt jsa = new JobStageAttempt(rs, ja);
- stageAttemptMap.put(stageId, jsa);
- runnableStageAttempts.add(jsa);
- }
-
- try {
- ccs.getScheduler().schedule(runnableStageAttempts);
- } catch (HyracksException e) {
- e.printStackTrace();
- ccs.getJobQueue().schedule(new JobAbortEvent(ccs, jobId, attempt));
- return;
- }
-
- final JobPlan plan = run.getJobPlan();
- for (final JobStageAttempt jsa : runnableStageAttempts) {
- ISchedule schedule = jsa.getSchedule();
- final Map<OperatorDescriptorId, Integer> partCountMap = new HashMap<OperatorDescriptorId, Integer>();
- final Map<String, Map<ActivityNodeId, Set<Integer>>> targetMap = new HashMap<String, Map<ActivityNodeId, Set<Integer>>>();
- for (ActivityNodeId aid : jsa.getJobStage().getTasks()) {
- String[] locations = schedule.getPartitions(aid);
- partCountMap.put(aid.getOperatorDescriptorId(), locations.length);
- int nLoc = locations.length;
- for (int i = 0; i < nLoc; ++i) {
- Map<ActivityNodeId, Set<Integer>> target = targetMap.get(locations[i]);
- if (target == null) {
- target = new HashMap<ActivityNodeId, Set<Integer>>();
- targetMap.put(locations[i], target);
- }
- Set<Integer> partIdxs = target.get(aid);
- if (partIdxs == null) {
- partIdxs = new HashSet<Integer>();
- target.put(aid, partIdxs);
- }
- partIdxs.add(i);
- }
- }
-
- Set<String> participatingNodeIds = ja.getParticipatingNodeIds();
- for (String nid : targetMap.keySet()) {
- ccs.getNodeMap().get(nid).getActiveJobIds().add(jobId);
- participatingNodeIds.add(nid);
- }
-
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- Phase1Installer p1is[] = new Phase1Installer[targetMap.size()];
- int i = 0;
- for (String nid : targetMap.keySet()) {
- p1is[i] = new Phase1Installer(nid, plan.getJobId(), plan.getApplicationName(), plan, jsa
- .getJobStage().getId(), jsa.getJobAttempt().getAttempt(), targetMap.get(nid),
- partCountMap);
- ++i;
- }
- LOGGER.info("Stage start - Phase 1");
- try {
- Map<PortInstanceId, Endpoint> globalPortMap = RemoteRunner.runRemote(ccs, p1is,
- new PortMapMergingAccumulator());
-
- Phase2Installer[] p2is = new Phase2Installer[targetMap.size()];
- Phase3Installer[] p3is = new Phase3Installer[targetMap.size()];
- StageStarter[] ss = new StageStarter[targetMap.size()];
-
- i = 0;
- for (String nid : targetMap.keySet()) {
- p2is[i] = new Phase2Installer(nid, plan.getJobId(), plan.getApplicationName(), plan, jsa
- .getJobStage().getId(), targetMap.get(nid), partCountMap, globalPortMap);
- p3is[i] = new Phase3Installer(nid, plan.getJobId(), jsa.getJobStage().getId());
- ss[i] = new StageStarter(nid, plan.getJobId(), jsa.getJobStage().getId());
- ++i;
- }
- LOGGER.info("Stage start - Phase 2");
- RemoteRunner.runRemote(ccs, p2is, null);
- LOGGER.info("Stage start - Phase 3");
- RemoteRunner.runRemote(ccs, p3is, null);
- LOGGER.info("Stage start");
- RemoteRunner.runRemote(ccs, ss, null);
- LOGGER.info("Stage started");
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- });
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java
deleted file mode 100644
index 8f07d09..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletCompleteEvent.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.job.manager.events;
-
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-
-public class StageletCompleteEvent implements Runnable {
- private final ClusterControllerService ccs;
- private final UUID jobId;
- private final UUID stageId;
- private final int attempt;
- private final String nodeId;
-
- public StageletCompleteEvent(ClusterControllerService ccs, UUID jobId, UUID stageId, int attempt, String nodeId) {
- this.ccs = ccs;
- this.jobId = jobId;
- this.stageId = stageId;
- this.attempt = attempt;
- this.nodeId = nodeId;
- }
-
- @Override
- public void run() {
- JobRun run = ccs.getRunMap().get(jobId);
- JobAttempt jobAttempt = run.getAttempts().get(attempt);
- JobStageAttempt jsAttempt = jobAttempt.getStageAttemptMap().get(stageId);
-
- Set<String> participatingNodes = jsAttempt.getParticipatingNodes();
- Set<String> completedNodes = jsAttempt.getCompletedNodes();
- completedNodes.add(nodeId);
-
- if (completedNodes.containsAll(participatingNodes)) {
- Set<UUID> completedStageIds = jobAttempt.getCompletedStageIds();
- completedStageIds.add(stageId);
-
- Set<UUID> inProgressStageIds = jobAttempt.getInProgressStageIds();
- inProgressStageIds.remove(stageId);
-
- ccs.getJobQueue().schedule(new ScheduleRunnableStagesEvent(ccs, jobId, attempt));
- }
- }
-
- @Override
- public String toString() {
- return "StageletCompleteEvent[" + jobId + ":" + stageId + ":" + attempt + ":" + nodeId + "]";
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
deleted file mode 100644
index 34dbeb4..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/StageletFailureEvent.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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.job.manager.events;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.manager.JobLifecycleHelper;
-
-public class StageletFailureEvent implements Runnable {
- private final ClusterControllerService ccs;
- private final UUID jobId;
- private final UUID stageId;
- private final int attempt;
- private final String nodeId;
-
- public StageletFailureEvent(ClusterControllerService ccs, UUID jobId, UUID stageId, int attempt, String nodeId) {
- this.ccs = ccs;
- this.jobId = jobId;
- this.stageId = stageId;
- this.attempt = attempt;
- this.nodeId = nodeId;
- }
-
- @Override
- public void run() {
- final JobRun run = ccs.getRunMap().get(jobId);
- List<JobAttempt> attempts = run.getAttempts();
- JobAttempt ja = attempts.get(attempt);
- final Set<String> targetNodes = new HashSet<String>(ja.getParticipatingNodeIds());
- Map<String, NodeControllerState> nodeMap = new HashMap<String, NodeControllerState>();
- for (String nodeId : targetNodes) {
- nodeMap.get(nodeId).getActiveJobIds().remove(jobId);
- }
- ccs.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- JobLifecycleHelper.abortJob(ccs, jobId, attempt, targetNodes);
- }
- });
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java
new file mode 100644
index 0000000..19fd95b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.java
@@ -0,0 +1,43 @@
+/*
+ * 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.job.manager.events;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+
+public class TaskCompleteEvent extends AbstractTaskLifecycleEvent {
+ public TaskCompleteEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId) {
+ super(ccs, jobId, taId, nodeId);
+ }
+
+ @Override
+ protected void performEvent(TaskAttempt ta) {
+ try {
+ ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+ ac.getJobRun().getScheduler().notifyTaskComplete(ta);
+ } catch (HyracksException e) {
+ e.printStackTrace();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "TaskCompleteEvent@[" + nodeId + "[" + jobId + ":" + taId + "]";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
new file mode 100644
index 0000000..8b7e821
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
@@ -0,0 +1,42 @@
+/*
+ * 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.job.manager.events;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+
+public class TaskFailureEvent extends AbstractTaskLifecycleEvent {
+ private final Exception exception;
+
+ public TaskFailureEvent(ClusterControllerService ccs, JobId jobId, TaskAttemptId taId, String nodeId,
+ Exception exception) {
+ super(ccs, jobId, taId, nodeId);
+ this.exception = exception;
+ }
+
+ @Override
+ protected void performEvent(TaskAttempt ta) {
+ ActivityCluster ac = ta.getTaskState().getTaskCluster().getActivityCluster();
+ ac.getJobRun().getScheduler().notifyTaskFailure(ta, ac, exception);
+ }
+
+ @Override
+ public String toString() {
+ return "TaskFailureEvent[" + jobId + ":" + taId + ":" + nodeId + "]";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
index fc1ba03..b89de07 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/UnregisterNodeEvent.java
@@ -18,9 +18,9 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
-public class UnregisterNodeEvent extends SynchronizableRunnable {
+public class UnregisterNodeEvent extends SynchronizableEvent {
private final ClusterControllerService ccs;
private final String nodeId;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
similarity index 75%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
index c884998..5fcd56a 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/Direction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/AbstractEvent.java
@@ -12,9 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.control.cc.jobqueue;
-public enum Direction {
- INPUT,
- OUTPUT,
+import java.util.logging.Level;
+
+public abstract class AbstractEvent implements Runnable {
+ public Level logLevel() {
+ return Level.INFO;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
index f533ad0..84a844c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/JobQueue.java
@@ -20,21 +20,23 @@
public class JobQueue {
private static final Logger LOGGER = Logger.getLogger(JobQueue.class.getName());
- private final LinkedBlockingQueue<Runnable> queue;
+ private final LinkedBlockingQueue<AbstractEvent> queue;
private final JobThread thread;
public JobQueue() {
- queue = new LinkedBlockingQueue<Runnable>();
+ queue = new LinkedBlockingQueue<AbstractEvent>();
thread = new JobThread();
thread.start();
}
- public void schedule(Runnable runnable) {
- LOGGER.info("Scheduling: " + runnable);
- queue.offer(runnable);
+ public void schedule(AbstractEvent event) {
+ if (LOGGER.isLoggable(event.logLevel())) {
+ LOGGER.info("Scheduling: " + event);
+ }
+ queue.offer(event);
}
- public void scheduleAndSync(SynchronizableRunnable sRunnable) throws Exception {
+ public void scheduleAndSync(SynchronizableEvent sRunnable) throws Exception {
schedule(sRunnable);
sRunnable.sync();
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
new file mode 100644
index 0000000..88c9097
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableEvent.java
@@ -0,0 +1,55 @@
+/*
+ * 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.jobqueue;
+
+public abstract class SynchronizableEvent extends AbstractEvent {
+ private boolean done;
+
+ private Exception e;
+
+ protected abstract void doRun() throws Exception;
+
+ public void init() {
+ done = false;
+ e = null;
+ }
+
+ @Override
+ public final void run() {
+ try {
+ doRun();
+ } catch (Exception e) {
+ this.e = e;
+ } finally {
+ synchronized (this) {
+ done = true;
+ notifyAll();
+ }
+ }
+ }
+
+ public final synchronized void sync() throws Exception {
+ while (!done) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ if (e != null) {
+ throw e;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
deleted file mode 100644
index 53315b7..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/jobqueue/SynchronizableRunnable.java
+++ /dev/null
@@ -1,41 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.jobqueue;
-
-public abstract class SynchronizableRunnable implements Runnable {
- private boolean done;
-
- private Exception e;
-
- protected abstract void doRun() throws Exception;
-
- public void init() {
- done = false;
- e = null;
- }
-
- @Override
- public final void run() {
- try {
- doRun();
- } catch (Exception e) {
- this.e = e;
- } finally {
- synchronized (this) {
- done = true;
- notifyAll();
- }
- }
- }
-
- public final synchronized void sync() throws Exception {
- while (!done) {
- try {
- wait();
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
- }
- if (e != null) {
- throw e;
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
new file mode 100644
index 0000000..d2c14b2
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionMatchMaker.java
@@ -0,0 +1,201 @@
+/*
+ * 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.partitions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+
+public class PartitionMatchMaker {
+ private final Map<PartitionId, List<PartitionDescriptor>> partitionDescriptors;
+
+ private final Map<PartitionId, List<PartitionRequest>> partitionRequests;
+
+ public PartitionMatchMaker() {
+ partitionDescriptors = new HashMap<PartitionId, List<PartitionDescriptor>>();
+ partitionRequests = new HashMap<PartitionId, List<PartitionRequest>>();
+ }
+
+ public List<Pair<PartitionDescriptor, PartitionRequest>> registerPartitionDescriptor(
+ PartitionDescriptor partitionDescriptor) {
+ List<Pair<PartitionDescriptor, PartitionRequest>> matches = new ArrayList<Pair<PartitionDescriptor, PartitionRequest>>();
+ PartitionId pid = partitionDescriptor.getPartitionId();
+ boolean matched = false;
+ List<PartitionRequest> requests = partitionRequests.get(pid);
+ if (requests != null) {
+ Iterator<PartitionRequest> i = requests.iterator();
+ while (i.hasNext()) {
+ PartitionRequest req = i.next();
+ if (partitionDescriptor.getState().isAtLeast(req.getMinimumState())) {
+ matches.add(new Pair<PartitionDescriptor, PartitionRequest>(partitionDescriptor, req));
+ i.remove();
+ matched = true;
+ if (!partitionDescriptor.isReusable()) {
+ break;
+ }
+ }
+ }
+ if (requests.isEmpty()) {
+ partitionRequests.remove(pid);
+ }
+ }
+
+ if (!matched) {
+ List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+ if (descriptors == null) {
+ descriptors = new ArrayList<PartitionDescriptor>();
+ partitionDescriptors.put(pid, descriptors);
+ }
+ descriptors.add(partitionDescriptor);
+ }
+
+ return matches;
+ }
+
+ public Pair<PartitionDescriptor, PartitionRequest> matchPartitionRequest(PartitionRequest partitionRequest) {
+ Pair<PartitionDescriptor, PartitionRequest> match = null;
+
+ PartitionId pid = partitionRequest.getPartitionId();
+
+ List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+ if (descriptors != null) {
+ Iterator<PartitionDescriptor> i = descriptors.iterator();
+ while (i.hasNext()) {
+ PartitionDescriptor descriptor = i.next();
+ if (descriptor.getState().isAtLeast(partitionRequest.getMinimumState())) {
+ match = new Pair<PartitionDescriptor, PartitionRequest>(descriptor, partitionRequest);
+ if (!descriptor.isReusable()) {
+ i.remove();
+ }
+ break;
+ }
+ }
+ if (descriptors.isEmpty()) {
+ partitionDescriptors.remove(pid);
+ }
+ }
+
+ if (match == null) {
+ List<PartitionRequest> requests = partitionRequests.get(pid);
+ if (requests == null) {
+ requests = new ArrayList<PartitionRequest>();
+ partitionRequests.put(pid, requests);
+ }
+ requests.add(partitionRequest);
+ }
+
+ return match;
+ }
+
+ public PartitionState getMaximumAvailableState(PartitionId pid) {
+ List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+ if (descriptors == null) {
+ return null;
+ }
+ for (PartitionDescriptor descriptor : descriptors) {
+ if (descriptor.getState() == PartitionState.COMMITTED) {
+ return PartitionState.COMMITTED;
+ }
+ }
+ return PartitionState.STARTED;
+ }
+
+ private interface IEntryFilter<T> {
+ public boolean matches(T o);
+ }
+
+ private static <T> void removeEntries(List<T> list, IEntryFilter<T> filter) {
+ Iterator<T> j = list.iterator();
+ while (j.hasNext()) {
+ T o = j.next();
+ if (filter.matches(o)) {
+ j.remove();
+ }
+ }
+ }
+
+ private static <T> void removeEntries(Map<PartitionId, List<T>> map, IEntryFilter<T> filter) {
+ Iterator<Map.Entry<PartitionId, List<T>>> i = map.entrySet().iterator();
+ while (i.hasNext()) {
+ Map.Entry<PartitionId, List<T>> e = i.next();
+ List<T> list = e.getValue();
+ removeEntries(list, filter);
+ if (list.isEmpty()) {
+ i.remove();
+ }
+ }
+ }
+
+ public void notifyNodeFailures(final Set<String> deadNodes) {
+ removeEntries(partitionDescriptors, new IEntryFilter<PartitionDescriptor>() {
+ @Override
+ public boolean matches(PartitionDescriptor o) {
+ return deadNodes.contains(o.getNodeId());
+ }
+ });
+ removeEntries(partitionRequests, new IEntryFilter<PartitionRequest>() {
+ @Override
+ public boolean matches(PartitionRequest o) {
+ return deadNodes.contains(o.getNodeId());
+ }
+ });
+ }
+
+ public void removeUncommittedPartitions(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
+ IEntryFilter<PartitionDescriptor> filter = new IEntryFilter<PartitionDescriptor>() {
+ @Override
+ public boolean matches(PartitionDescriptor o) {
+ return o.getState() != PartitionState.COMMITTED && taIds.contains(o.getProducingTaskAttemptId());
+ }
+ };
+ for (PartitionId pid : partitionIds) {
+ List<PartitionDescriptor> descriptors = partitionDescriptors.get(pid);
+ if (descriptors != null) {
+ removeEntries(descriptors, filter);
+ if (descriptors.isEmpty()) {
+ partitionDescriptors.remove(pid);
+ }
+ }
+ }
+ }
+
+ public void removePartitionRequests(Set<PartitionId> partitionIds, final Set<TaskAttemptId> taIds) {
+ IEntryFilter<PartitionRequest> filter = new IEntryFilter<PartitionRequest>() {
+ @Override
+ public boolean matches(PartitionRequest o) {
+ return taIds.contains(o.getRequestingTaskAttemptId());
+ }
+ };
+ for (PartitionId pid : partitionIds) {
+ List<PartitionRequest> requests = partitionRequests.get(pid);
+ if (requests != null) {
+ removeEntries(requests, filter);
+ if (requests.isEmpty()) {
+ partitionRequests.remove(pid);
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
new file mode 100644
index 0000000..90c6ad1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/partitions/PartitionUtils.java
@@ -0,0 +1,33 @@
+package edu.uci.ics.hyracks.control.cc.partitions;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+
+public class PartitionUtils {
+ public static void reportPartitionMatch(ClusterControllerService ccs, final PartitionId pid,
+ Pair<PartitionDescriptor, PartitionRequest> match) {
+ PartitionDescriptor desc = match.first;
+ PartitionRequest req = match.second;
+
+ NodeControllerState producerNCS = ccs.getNodeMap().get(desc.getNodeId());
+ NodeControllerState requestorNCS = ccs.getNodeMap().get(req.getNodeId());
+ final NetworkAddress dataport = producerNCS.getDataPort();
+ final INodeController requestorNC = requestorNCS.getNodeController();
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ requestorNC.reportPartitionAvailability(pid, dataport);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
index c0a35cb..ec15186 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteOp.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.control.cc.remote;
-import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
public interface RemoteOp<T> {
public String getNodeId();
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
index 9bd4375..a37c786 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/RemoteRunner.java
@@ -18,9 +18,9 @@
import java.util.Vector;
import java.util.concurrent.Semaphore;
-import edu.uci.ics.hyracks.api.control.INodeController;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
public class RemoteRunner {
public static <T, R> R runRemote(ClusterControllerService ccs, final RemoteOp<T>[] remoteOps,
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
index f8ca6af..52e726c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationDestroyer.java
@@ -14,8 +14,8 @@
*/
package edu.uci.ics.hyracks.control.cc.remote.ops;
-import edu.uci.ics.hyracks.api.control.INodeController;
import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
public class ApplicationDestroyer implements RemoteOp<Void> {
private String nodeId;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
index af61249..d6a1d26 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/ApplicationStarter.java
@@ -14,8 +14,8 @@
*/
package edu.uci.ics.hyracks.control.cc.remote.ops;
-import edu.uci.ics.hyracks.api.control.INodeController;
import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
public class ApplicationStarter implements RemoteOp<Void> {
private String nodeId;
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
index 81a8610..414cf12 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobCompleteNotifier.java
@@ -14,16 +14,15 @@
*/
package edu.uci.ics.hyracks.control.cc.remote.ops;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
public class JobCompleteNotifier implements RemoteOp<Void> {
private String nodeId;
- private UUID jobId;
+ private JobId jobId;
- public JobCompleteNotifier(String nodeId, UUID jobId) {
+ public JobCompleteNotifier(String nodeId, JobId jobId) {
this.nodeId = nodeId;
this.jobId = jobId;
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
deleted file mode 100644
index b50aa8e..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/JobletAborter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class JobletAborter implements RemoteOp<Void> {
- private String nodeId;
- private UUID jobId;
- private int attempt;
-
- public JobletAborter(String nodeId, UUID jobId, int attempt) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.attempt = attempt;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.abortJoblet(jobId, attempt);
- return null;
- }
-
- @Override
- public String toString() {
- return jobId + " Aborting";
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java
deleted file mode 100644
index 2fe67ac..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase1Installer.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase1Installer implements RemoteOp<Map<PortInstanceId, Endpoint>> {
- private String nodeId;
- private UUID jobId;
- private String appName;
- private JobPlan plan;
- private UUID stageId;
- private int attempt;
- private Map<ActivityNodeId, Set<Integer>> tasks;
- private Map<OperatorDescriptorId, Integer> opNumPartitions;
-
- public Phase1Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId, int attempt,
- Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.appName = appName;
- this.plan = plan;
- this.stageId = stageId;
- this.attempt = attempt;
- this.tasks = tasks;
- this.opNumPartitions = opNumPartitions;
- }
-
- @Override
- public Map<PortInstanceId, Endpoint> execute(INodeController node) throws Exception {
- return node.initializeJobletPhase1(appName, jobId, attempt, JavaSerializationUtils.serialize(plan), stageId,
- tasks, opNumPartitions);
- }
-
- @Override
- public String toString() {
- return jobId + " Distribution Phase 1";
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java
deleted file mode 100644
index 99d7c9f..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase2Installer.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase2Installer implements RemoteOp<Void> {
- private String nodeId;
- private UUID jobId;
- private String appName;
- private JobPlan plan;
- private UUID stageId;
- private Map<ActivityNodeId, Set<Integer>> tasks;
- private Map<OperatorDescriptorId, Integer> opNumPartitions;
- private Map<PortInstanceId, Endpoint> globalPortMap;
-
- public Phase2Installer(String nodeId, UUID jobId, String appName, JobPlan plan, UUID stageId,
- Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
- Map<PortInstanceId, Endpoint> globalPortMap) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.appName = appName;
- this.plan = plan;
- this.stageId = stageId;
- this.tasks = tasks;
- this.opNumPartitions = opNumPartitions;
- this.globalPortMap = globalPortMap;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.initializeJobletPhase2(appName, jobId, JavaSerializationUtils.serialize(plan), stageId, tasks,
- opNumPartitions, globalPortMap);
- return null;
- }
-
- @Override
- public String toString() {
- return jobId + " Distribution Phase 2";
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java
deleted file mode 100644
index 02f9aa1..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/Phase3Installer.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class Phase3Installer implements RemoteOp<Void> {
- private String nodeId;
- private UUID jobId;
- private UUID stageId;
-
- public Phase3Installer(String nodeId, UUID jobId, UUID stageId) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.stageId = stageId;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.commitJobletInitialization(jobId, stageId);
- return null;
- }
-
- @Override
- public String toString() {
- return jobId + " Distribution Phase 3";
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java
deleted file mode 100644
index 8cffd08..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/PortMapMergingAccumulator.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
-import edu.uci.ics.hyracks.control.cc.remote.Accumulator;
-
-public class PortMapMergingAccumulator implements
- Accumulator<Map<PortInstanceId, Endpoint>, Map<PortInstanceId, Endpoint>> {
- Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
-
- @Override
- public void accumulate(Map<PortInstanceId, Endpoint> o) {
- portMap.putAll(o);
- }
-
- @Override
- public Map<PortInstanceId, Endpoint> getResult() {
- return portMap;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
deleted file mode 100644
index d62254b..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.remote.ops;
-
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.control.cc.remote.RemoteOp;
-
-public class StageStarter implements RemoteOp<Void> {
- private String nodeId;
- private UUID jobId;
- private UUID stageId;
-
- public StageStarter(String nodeId, UUID jobId, UUID stageId) {
- this.nodeId = nodeId;
- this.jobId = jobId;
- this.stageId = stageId;
- }
-
- @Override
- public Void execute(INodeController node) throws Exception {
- node.startStage(jobId, stageId);
- return null;
- }
-
- @Override
- public String toString() {
- return jobId + " Started Stage: " + stageId;
- }
-
- @Override
- public String getNodeId() {
- return nodeId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
new file mode 100644
index 0000000..0e59c8df
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
@@ -0,0 +1,165 @@
+/*
+ * 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.scheduler;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+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.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityClusterId;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+
+public class ActivityClusterGraphBuilder {
+ private static final Logger LOGGER = Logger.getLogger(ActivityClusterGraphBuilder.class.getName());
+
+ private final JobRun jobRun;
+
+ public ActivityClusterGraphBuilder(JobRun jobRun) {
+ this.jobRun = jobRun;
+ }
+
+ private static Pair<ActivityId, ActivityId> findMergePair(JobActivityGraph jag, JobSpecification spec,
+ Set<ActivityCluster> eqSets) {
+ Map<ActivityId, IActivity> activityNodeMap = jag.getActivityNodeMap();
+ for (ActivityCluster eqSet : eqSets) {
+ for (ActivityId t : eqSet.getActivities()) {
+ IActivity activity = activityNodeMap.get(t);
+ List<Integer> inputList = jag.getActivityInputMap().get(t);
+ if (inputList != null) {
+ for (Integer idx : inputList) {
+ IConnectorDescriptor conn = spec.getInputConnectorDescriptor(activity.getActivityId()
+ .getOperatorDescriptorId(), idx);
+ OperatorDescriptorId producerId = spec.getProducer(conn).getOperatorId();
+ int producerOutputIndex = spec.getProducerOutputIndex(conn);
+ ActivityId inTask = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
+ if (!eqSet.getActivities().contains(inTask)) {
+ return new Pair<ActivityId, ActivityId>(t, inTask);
+ }
+ }
+ }
+ List<Integer> outputList = jag.getActivityOutputMap().get(t);
+ if (outputList != null) {
+ for (Integer idx : outputList) {
+ IConnectorDescriptor conn = spec.getOutputConnectorDescriptor(activity.getActivityId()
+ .getOperatorDescriptorId(), idx);
+ OperatorDescriptorId consumerId = spec.getConsumer(conn).getOperatorId();
+ int consumerInputIndex = spec.getConsumerInputIndex(conn);
+ ActivityId outTask = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
+ if (!eqSet.getActivities().contains(outTask)) {
+ return new Pair<ActivityId, ActivityId>(t, outTask);
+ }
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ public Set<ActivityCluster> inferActivityClusters(JobActivityGraph jag) {
+ JobSpecification spec = jag.getJobSpecification();
+
+ /*
+ * Build initial equivalence sets map. We create a map such that for each IOperatorTask, t -> { t }
+ */
+ Map<ActivityId, ActivityCluster> stageMap = new HashMap<ActivityId, ActivityCluster>();
+ Set<ActivityCluster> stages = new HashSet<ActivityCluster>();
+ for (Set<ActivityId> taskIds : jag.getOperatorActivityMap().values()) {
+ for (ActivityId taskId : taskIds) {
+ Set<ActivityId> eqSet = new HashSet<ActivityId>();
+ eqSet.add(taskId);
+ ActivityCluster stage = new ActivityCluster(jobRun, eqSet);
+ stageMap.put(taskId, stage);
+ stages.add(stage);
+ }
+ }
+
+ boolean changed = true;
+ while (changed) {
+ changed = false;
+ Pair<ActivityId, ActivityId> pair = findMergePair(jag, spec, stages);
+ if (pair != null) {
+ merge(stageMap, stages, pair.first, pair.second);
+ changed = true;
+ }
+ }
+
+ Map<ActivityId, Set<ActivityId>> blocker2BlockedMap = jag.getBlocker2BlockedMap();
+ for (ActivityCluster s : stages) {
+ Set<ActivityCluster> blockedStages = new HashSet<ActivityCluster>();
+ for (ActivityId t : s.getActivities()) {
+ Set<ActivityId> blockedTasks = blocker2BlockedMap.get(t);
+ if (blockedTasks != null) {
+ for (ActivityId bt : blockedTasks) {
+ blockedStages.add(stageMap.get(bt));
+ }
+ }
+ }
+ for (ActivityCluster bs : blockedStages) {
+ bs.addDependency(s);
+ s.addDependent(bs);
+ }
+ }
+ Set<ActivityCluster> roots = new HashSet<ActivityCluster>();
+ int idCounter = 0;
+ for (ActivityCluster s : stages) {
+ s.setActivityClusterId(new ActivityClusterId(idCounter++));
+ if (s.getDependents().isEmpty()) {
+ roots.add(s);
+ }
+ }
+ jobRun.setActivityClusters(stages);
+ jobRun.getActivityClusterMap().putAll(stageMap);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inferred " + stages.size() + " stages");
+ for (ActivityCluster s : stages) {
+ LOGGER.info(s.toString());
+ }
+ }
+ return roots;
+ }
+
+ private void merge(Map<ActivityId, ActivityCluster> eqSetMap, Set<ActivityCluster> eqSets, ActivityId t1,
+ ActivityId t2) {
+ ActivityCluster stage1 = eqSetMap.get(t1);
+ Set<ActivityId> s1 = stage1.getActivities();
+ ActivityCluster stage2 = eqSetMap.get(t2);
+ Set<ActivityId> s2 = stage2.getActivities();
+
+ Set<ActivityId> mergedSet = new HashSet<ActivityId>();
+ mergedSet.addAll(s1);
+ mergedSet.addAll(s2);
+
+ eqSets.remove(stage1);
+ eqSets.remove(stage2);
+ ActivityCluster mergedStage = new ActivityCluster(jobRun, mergedSet);
+ eqSets.add(mergedStage);
+
+ for (ActivityId t : mergedSet) {
+ eqSetMap.put(t, mergedStage);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
new file mode 100644
index 0000000..e5d9f64
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -0,0 +1,337 @@
+/*
+ * 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.scheduler;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
+import edu.uci.ics.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.Pair;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.ActivityClusterPlan;
+import edu.uci.ics.hyracks.control.cc.job.ActivityPlan;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterId;
+
+public class ActivityClusterPlanner {
+ private static final Logger LOGGER = Logger.getLogger(ActivityClusterPlanner.class.getName());
+
+ private final JobScheduler scheduler;
+
+ private final Map<PartitionId, TaskCluster> partitionProducingTaskClusterMap;
+
+ public ActivityClusterPlanner(JobScheduler newJobScheduler) {
+ this.scheduler = newJobScheduler;
+ partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
+ }
+
+ public void planActivityCluster(ActivityCluster ac) throws HyracksException {
+ JobRun jobRun = scheduler.getJobRun();
+ Map<ActivityId, ActivityPartitionDetails> pcMap = computePartitionCounts(ac);
+
+ Map<ActivityId, ActivityPlan> activityPlanMap = new HashMap<ActivityId, ActivityPlan>();
+ Set<ActivityId> activities = ac.getActivities();
+
+ Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
+
+ Set<ActivityId> depAnIds = new HashSet<ActivityId>();
+ for (ActivityId anId : activities) {
+ depAnIds.clear();
+ getDependencyActivityIds(depAnIds, anId);
+ ActivityPartitionDetails apd = pcMap.get(anId);
+ Task[] tasks = new Task[apd.getPartitionCount()];
+ ActivityPlan activityPlan = new ActivityPlan(apd);
+ for (int i = 0; i < tasks.length; ++i) {
+ TaskId tid = new TaskId(anId, i);
+ tasks[i] = new Task(tid, activityPlan);
+ for (ActivityId danId : depAnIds) {
+ ActivityCluster dAC = jobRun.getActivityClusterMap().get(danId);
+ ActivityClusterPlan dACP = dAC.getPlan();
+ assert dACP != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+ + danId;
+ Task[] dATasks = dACP.getActivityPlanMap().get(danId).getTasks();
+ assert dATasks != null : "IllegalStateEncountered: Dependent AC is being planned without a plan for dependency AC: Encountered no plan for ActivityID "
+ + danId;
+ assert dATasks.length == tasks.length : "Dependency activity partitioned differently from dependent: "
+ + dATasks.length + " != " + tasks.length;
+ Task dTask = dATasks[i];
+ TaskId dTaskId = dTask.getTaskId();
+ tasks[i].getDependencies().add(dTaskId);
+ dTask.getDependents().add(tid);
+ }
+ Set<TaskId> cluster = new HashSet<TaskId>();
+ cluster.add(tid);
+ taskClusterMap.put(tid, cluster);
+ }
+ activityPlan.setTasks(tasks);
+ activityPlanMap.put(anId, activityPlan);
+ }
+
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = assignConnectorPolicy(ac, activityPlanMap);
+ scheduler.getJobRun().getConnectorPolicyMap().putAll(connectorPolicies);
+
+ Map<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>> taskConnectivity = new HashMap<TaskId, List<Pair<TaskId, ConnectorDescriptorId>>>();
+ JobActivityGraph jag = jobRun.getJobActivityGraph();
+ BitSet targetBitmap = new BitSet();
+ for (ActivityId ac1 : activities) {
+ Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
+ int nProducers = ac1TaskStates.length;
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+ if (outputConns != null) {
+ for (IConnectorDescriptor c : outputConns) {
+ ConnectorDescriptorId cdId = c.getConnectorId();
+ ActivityId ac2 = jag.getConsumerActivity(cdId);
+ Task[] ac2TaskStates = activityPlanMap.get(ac2).getTasks();
+ int nConsumers = ac2TaskStates.length;
+ for (int i = 0; i < nProducers; ++i) {
+ c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+ List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(ac1TaskStates[i]
+ .getTaskId());
+ if (cInfoList == null) {
+ cInfoList = new ArrayList<Pair<TaskId, ConnectorDescriptorId>>();
+ taskConnectivity.put(ac1TaskStates[i].getTaskId(), cInfoList);
+ }
+ Set<TaskId> cluster = taskClusterMap.get(ac1TaskStates[i].getTaskId());
+ for (int j = targetBitmap.nextSetBit(0); j >= 0; j = targetBitmap.nextSetBit(j + 1)) {
+ cInfoList.add(new Pair<TaskId, ConnectorDescriptorId>(ac2TaskStates[j].getTaskId(), cdId));
+ IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
+ if (cPolicy.requiresProducerConsumerCoscheduling()) {
+ cluster.add(ac2TaskStates[j].getTaskId());
+ }
+ }
+ }
+ }
+ }
+ }
+
+ boolean done = false;
+ while (!done) {
+ done = true;
+ Set<TaskId> set = new HashSet<TaskId>();
+ Set<TaskId> oldSet = null;
+ for (Map.Entry<TaskId, Set<TaskId>> e : taskClusterMap.entrySet()) {
+ set.clear();
+ oldSet = e.getValue();
+ set.addAll(e.getValue());
+ for (TaskId tid : e.getValue()) {
+ set.addAll(taskClusterMap.get(tid));
+ }
+ for (TaskId tid : set) {
+ Set<TaskId> targetSet = taskClusterMap.get(tid);
+ if (!targetSet.equals(set)) {
+ done = false;
+ break;
+ }
+ }
+ if (!done) {
+ break;
+ }
+ }
+ for (TaskId tid : oldSet) {
+ taskClusterMap.put(tid, set);
+ }
+ }
+
+ Set<Set<TaskId>> clusters = new HashSet<Set<TaskId>>(taskClusterMap.values());
+ Set<TaskCluster> tcSet = new HashSet<TaskCluster>();
+ int counter = 0;
+ for (Set<TaskId> cluster : clusters) {
+ Set<Task> taskStates = new HashSet<Task>();
+ for (TaskId tid : cluster) {
+ taskStates.add(activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()]);
+ }
+ TaskCluster tc = new TaskCluster(new TaskClusterId(ac.getActivityClusterId(), counter++), ac,
+ taskStates.toArray(new Task[taskStates.size()]));
+ tcSet.add(tc);
+ for (TaskId tid : cluster) {
+ activityPlanMap.get(tid.getActivityId()).getTasks()[tid.getPartition()].setTaskCluster(tc);
+ }
+ }
+ TaskCluster[] taskClusters = tcSet.toArray(new TaskCluster[tcSet.size()]);
+
+ for (TaskCluster tc : taskClusters) {
+ Set<TaskCluster> tcDependencyTaskClusters = tc.getDependencyTaskClusters();
+ for (Task ts : tc.getTasks()) {
+ TaskId tid = ts.getTaskId();
+ List<Pair<TaskId, ConnectorDescriptorId>> cInfoList = taskConnectivity.get(tid);
+ if (cInfoList != null) {
+ for (Pair<TaskId, ConnectorDescriptorId> p : cInfoList) {
+ Task targetTS = activityPlanMap.get(p.first.getActivityId()).getTasks()[p.first.getPartition()];
+ TaskCluster targetTC = targetTS.getTaskCluster();
+ if (targetTC != tc) {
+ ConnectorDescriptorId cdId = p.second;
+ PartitionId pid = new PartitionId(jobRun.getJobId(), cdId, tid.getPartition(),
+ p.first.getPartition());
+ tc.getProducedPartitions().add(pid);
+ targetTC.getRequiredPartitions().add(pid);
+ partitionProducingTaskClusterMap.put(pid, tc);
+ }
+ }
+ }
+
+ for (TaskId dTid : ts.getDependencies()) {
+ TaskCluster dTC = getTaskCluster(dTid);
+ dTC.getDependentTaskClusters().add(tc);
+ tcDependencyTaskClusters.add(dTC);
+ }
+ }
+ }
+
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Plan for " + ac);
+ LOGGER.info("Built " + tcSet.size() + " Task Clusters");
+ for (TaskCluster tc : tcSet) {
+ LOGGER.info("Tasks: " + Arrays.toString(tc.getTasks()));
+ }
+ }
+
+ ac.setPlan(new ActivityClusterPlan(taskClusters, activityPlanMap));
+ }
+
+ private TaskCluster getTaskCluster(TaskId tid) {
+ ActivityCluster ac = scheduler.getJobRun().getActivityClusterMap().get(tid.getActivityId());
+ ActivityClusterPlan acp = ac.getPlan();
+ Task[] tasks = acp.getActivityPlanMap().get(tid.getActivityId()).getTasks();
+ Task task = tasks[tid.getPartition()];
+ assert task.getTaskId().equals(tid);
+ return task.getTaskCluster();
+ }
+
+ private void getDependencyActivityIds(Set<ActivityId> depAnIds, ActivityId anId) {
+ JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
+ Set<ActivityId> blockers = jag.getBlocked2BlockerMap().get(anId);
+ if (blockers != null) {
+ depAnIds.addAll(blockers);
+ }
+ }
+
+ private Map<ConnectorDescriptorId, IConnectorPolicy> assignConnectorPolicy(ActivityCluster ac,
+ Map<ActivityId, ActivityPlan> taskMap) {
+ JobActivityGraph jag = scheduler.getJobRun().getJobActivityGraph();
+ Map<ConnectorDescriptorId, IConnectorPolicy> cPolicyMap = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
+ Set<ActivityId> activities = ac.getActivities();
+ BitSet targetBitmap = new BitSet();
+ for (ActivityId a1 : activities) {
+ Task[] ac1TaskStates = taskMap.get(a1).getTasks();
+ int nProducers = ac1TaskStates.length;
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(a1);
+ if (outputConns != null) {
+ for (IConnectorDescriptor c : outputConns) {
+ ConnectorDescriptorId cdId = c.getConnectorId();
+ ActivityId a2 = jag.getConsumerActivity(cdId);
+ Task[] ac2TaskStates = taskMap.get(a2).getTasks();
+ int nConsumers = ac2TaskStates.length;
+
+ int[] fanouts = new int[nProducers];
+ for (int i = 0; i < nProducers; ++i) {
+ c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+ fanouts[i] = targetBitmap.cardinality();
+ }
+ IConnectorPolicy cp = assignConnectorPolicy(c, nProducers, nConsumers, fanouts);
+ cPolicyMap.put(cdId, cp);
+ }
+ }
+ }
+ return cPolicyMap;
+ }
+
+ private IConnectorPolicy assignConnectorPolicy(IConnectorDescriptor c, int nProducers, int nConsumers, int[] fanouts) {
+ IConnectorPolicyAssignmentPolicy cpap = scheduler.getJobRun().getJobActivityGraph().getJobSpecification()
+ .getConnectorPolicyAssignmentPolicy();
+ if (cpap != null) {
+ return cpap.getConnectorPolicyAssignment(c, nProducers, nConsumers, fanouts);
+ }
+ return new PipeliningConnectorPolicy();
+ }
+
+ private Map<ActivityId, ActivityPartitionDetails> computePartitionCounts(ActivityCluster ac)
+ throws HyracksException {
+ PartitionConstraintSolver solver = scheduler.getSolver();
+ JobRun jobRun = scheduler.getJobRun();
+ Set<LValueConstraintExpression> lValues = new HashSet<LValueConstraintExpression>();
+ for (ActivityId anId : ac.getActivities()) {
+ lValues.add(new PartitionCountExpression(anId.getOperatorDescriptorId()));
+ }
+ solver.solve(lValues);
+ Map<OperatorDescriptorId, Integer> nPartMap = new HashMap<OperatorDescriptorId, Integer>();
+ for (LValueConstraintExpression lv : lValues) {
+ Object value = solver.getValue(lv);
+ if (value == null) {
+ throw new HyracksException("No value found for " + lv);
+ }
+ if (!(value instanceof Number)) {
+ throw new HyracksException("Unexpected type of value bound to " + lv + ": " + value.getClass() + "("
+ + value + ")");
+ }
+ int nParts = ((Number) value).intValue();
+ if (nParts <= 0) {
+ throw new HyracksException("Unsatisfiable number of partitions for " + lv + ": " + nParts);
+ }
+ nPartMap.put(((PartitionCountExpression) lv).getOperatorDescriptorId(), Integer.valueOf(nParts));
+ }
+ Map<ActivityId, ActivityPartitionDetails> activityPartsMap = new HashMap<ActivityId, ActivityPartitionDetails>();
+ for (ActivityId anId : ac.getActivities()) {
+ int nParts = nPartMap.get(anId.getOperatorDescriptorId());
+ int[] nInputPartitions = null;
+ List<IConnectorDescriptor> inputs = jobRun.getJobActivityGraph().getActivityInputConnectorDescriptors(anId);
+ if (inputs != null) {
+ nInputPartitions = new int[inputs.size()];
+ for (int i = 0; i < nInputPartitions.length; ++i) {
+ nInputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
+ .getProducerActivity(inputs.get(i).getConnectorId()).getOperatorDescriptorId());
+ }
+ }
+ int[] nOutputPartitions = null;
+ List<IConnectorDescriptor> outputs = jobRun.getJobActivityGraph().getActivityOutputConnectorDescriptors(
+ anId);
+ if (outputs != null) {
+ nOutputPartitions = new int[outputs.size()];
+ for (int i = 0; i < nOutputPartitions.length; ++i) {
+ nOutputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
+ .getConsumerActivity(outputs.get(i).getConnectorId()).getOperatorDescriptorId());
+ }
+ }
+ ActivityPartitionDetails apd = new ActivityPartitionDetails(nParts, nInputPartitions, nOutputPartitions);
+ activityPartsMap.put(anId, apd);
+ }
+ return activityPartsMap;
+ }
+
+ public Map<? extends PartitionId, ? extends TaskCluster> getPartitionProducingTaskClusterMap() {
+ return partitionProducingTaskClusterMap;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.java
new file mode 100644
index 0000000..414b543
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityPartitionDetails.java
@@ -0,0 +1,49 @@
+/*
+ * 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.scheduler;
+
+import java.util.Arrays;
+
+public class ActivityPartitionDetails {
+ private final int nPartitions;
+
+ private final int[] nInputPartitions;
+
+ private final int[] nOutputPartitions;
+
+ public ActivityPartitionDetails(int nPartitions, int[] nInputPartitions, int[] nOutputPartitions) {
+ this.nPartitions = nPartitions;
+ this.nInputPartitions = nInputPartitions;
+ this.nOutputPartitions = nOutputPartitions;
+ }
+
+ public int getPartitionCount() {
+ return nPartitions;
+ }
+
+ public int[] getInputPartitionCounts() {
+ return nInputPartitions;
+ }
+
+ public int[] getOutputPartitionCounts() {
+ return nOutputPartitions;
+ }
+
+ @Override
+ public String toString() {
+ return nPartitions + ":" + (nInputPartitions == null ? "[]" : Arrays.toString(nInputPartitions)) + ":"
+ + (nOutputPartitions == null ? "[]" : Arrays.toString(nOutputPartitions));
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
deleted file mode 100644
index f8fcb89..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
+++ /dev/null
@@ -1,18 +0,0 @@
-/*
- * 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.scheduler;
-
-public interface IJobAttemptSchedulerState {
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
deleted file mode 100644
index 2906ba6..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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.scheduler;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-
-public interface ISchedule {
- public String[] getPartitions(ActivityNodeId aid);
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
deleted file mode 100644
index 3893a2d..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.scheduler;
-
-import java.util.Set;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-
-public interface IScheduler {
- public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja);
-
- public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException;
-}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
new file mode 100644
index 0000000..2fc9624
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -0,0 +1,664 @@
+/*
+ * 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.scheduler;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+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.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
+import edu.uci.ics.hyracks.control.cc.job.Task;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskClusterAttempt;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCleanupEvent;
+import edu.uci.ics.hyracks.control.cc.partitions.PartitionMatchMaker;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+
+public class JobScheduler {
+ private static final Logger LOGGER = Logger.getLogger(JobScheduler.class.getName());
+
+ private final ClusterControllerService ccs;
+
+ private final JobRun jobRun;
+
+ private final PartitionConstraintSolver solver;
+
+ private final Map<PartitionId, TaskCluster> partitionProducingTaskClusterMap;
+
+ private final Set<TaskCluster> inProgressTaskClusters;
+
+ private Set<ActivityCluster> rootActivityClusters;
+
+ public JobScheduler(ClusterControllerService ccs, JobRun jobRun) {
+ this.ccs = ccs;
+ this.jobRun = jobRun;
+ solver = new PartitionConstraintSolver();
+ partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
+ inProgressTaskClusters = new HashSet<TaskCluster>();
+ }
+
+ public JobRun getJobRun() {
+ return jobRun;
+ }
+
+ public PartitionConstraintSolver getSolver() {
+ return solver;
+ }
+
+ public void startJob() throws HyracksException {
+ analyze();
+ startRunnableActivityClusters();
+ }
+
+ private void analyze() throws HyracksException {
+ final JobActivityGraph jag = jobRun.getJobActivityGraph();
+ final ICCApplicationContext appCtx = ccs.getApplicationMap().get(jag.getApplicationName());
+ JobSpecification spec = jag.getJobSpecification();
+ final Set<Constraint> contributedConstraints = new HashSet<Constraint>();
+ final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
+ @Override
+ public void addConstraint(Constraint constraint) {
+ contributedConstraints.add(constraint);
+ }
+ };
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+ @Override
+ public void visit(IOperatorDescriptor op) {
+ op.contributeSchedulingConstraints(acceptor, jag, appCtx);
+ }
+ });
+ PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
+ @Override
+ public void visit(IConnectorDescriptor conn) {
+ conn.contributeSchedulingConstraints(acceptor, jag, appCtx);
+ }
+ });
+ contributedConstraints.addAll(spec.getUserConstraints());
+ solver.addConstraints(contributedConstraints);
+
+ ActivityClusterGraphBuilder acgb = new ActivityClusterGraphBuilder(jobRun);
+ rootActivityClusters = acgb.inferActivityClusters(jag);
+ }
+
+ private void findRunnableTaskClusterRoots(Set<TaskCluster> frontier, Set<ActivityCluster> roots)
+ throws HyracksException {
+ for (ActivityCluster root : roots) {
+ findRunnableTaskClusterRoots(frontier, root);
+ }
+ }
+
+ private void findRunnableTaskClusterRoots(Set<TaskCluster> frontier, ActivityCluster candidate)
+ throws HyracksException {
+ boolean depsComplete = true;
+ for (ActivityCluster depAC : candidate.getDependencies()) {
+ if (!isPlanned(depAC)) {
+ depsComplete = false;
+ findRunnableTaskClusterRoots(frontier, depAC);
+ } else {
+ boolean tcRootsComplete = true;
+ Set<TaskCluster> depACTCRoots = new HashSet<TaskCluster>();
+ for (TaskCluster tc : depAC.getPlan().getTaskClusters()) {
+ if (tc.getProducedPartitions().isEmpty()) {
+ TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+ if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+ tcRootsComplete = false;
+ }
+ depACTCRoots.add(tc);
+ }
+ }
+ if (!tcRootsComplete) {
+ depsComplete = false;
+ findRunnableTaskClusterRoots(frontier, depAC);
+ }
+ }
+ }
+ if (depsComplete) {
+ if (!isPlanned(candidate)) {
+ ActivityClusterPlanner acp = new ActivityClusterPlanner(this);
+ acp.planActivityCluster(candidate);
+ partitionProducingTaskClusterMap.putAll(acp.getPartitionProducingTaskClusterMap());
+ }
+ for (TaskCluster tc : candidate.getPlan().getTaskClusters()) {
+ if (tc.getProducedPartitions().isEmpty()) {
+ TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+ if (tca == null || tca.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+ frontier.add(tc);
+ }
+ }
+ }
+ }
+ }
+
+ private boolean isPlanned(ActivityCluster ac) {
+ return ac.getPlan() != null;
+ }
+
+ private void startRunnableActivityClusters() throws HyracksException {
+ Set<TaskCluster> taskClusterRoots = new HashSet<TaskCluster>();
+ findRunnableTaskClusterRoots(taskClusterRoots, rootActivityClusters);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Runnable TC roots: " + taskClusterRoots + ", inProgressTaskClusters: "
+ + inProgressTaskClusters);
+ }
+ if (taskClusterRoots.isEmpty() && inProgressTaskClusters.isEmpty()) {
+ ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
+ return;
+ }
+ startRunnableTaskClusters(taskClusterRoots);
+ }
+
+ private void startRunnableTaskClusters(Set<TaskCluster> tcRoots) throws HyracksException {
+ Map<TaskCluster, Runnability> runnabilityMap = new HashMap<TaskCluster, Runnability>();
+ for (TaskCluster tc : tcRoots) {
+ assignRunnabilityRank(tc, runnabilityMap);
+ }
+
+ PriorityQueue<RankedRunnableTaskCluster> queue = new PriorityQueue<RankedRunnableTaskCluster>();
+ for (Map.Entry<TaskCluster, Runnability> e : runnabilityMap.entrySet()) {
+ TaskCluster tc = e.getKey();
+ Runnability runnability = e.getValue();
+ if (runnability.getTag() != Runnability.Tag.RUNNABLE) {
+ continue;
+ }
+ int priority = runnability.getPriority();
+ if (priority >= 0 && priority < Integer.MAX_VALUE) {
+ queue.add(new RankedRunnableTaskCluster(priority, tc));
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Ranked TCs: " + queue);
+ }
+
+ Map<String, List<TaskAttemptDescriptor>> taskAttemptMap = new HashMap<String, List<TaskAttemptDescriptor>>();
+ for (RankedRunnableTaskCluster rrtc : queue) {
+ TaskCluster tc = rrtc.getTaskCluster();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Found runnable TC: " + tc);
+ List<TaskClusterAttempt> attempts = tc.getAttempts();
+ LOGGER.info("Attempts so far:" + attempts.size());
+ for (TaskClusterAttempt tcAttempt : attempts) {
+ LOGGER.info("Status: " + tcAttempt.getStatus());
+ }
+ }
+ assignTaskLocations(tc, taskAttemptMap);
+ }
+
+ if (taskAttemptMap.isEmpty()) {
+ return;
+ }
+
+ startTasks(taskAttemptMap);
+ }
+
+ /*
+ * Runnability rank has the following semantics
+ * Runnability(Runnable TaskCluster depending on completed TaskClusters) = {RUNNABLE, 0}
+ * Runnability(Runnable TaskCluster) = max(Rank(Dependent TaskClusters)) + 1
+ * Runnability(Non-schedulable TaskCluster) = {NOT_RUNNABLE, _}
+ */
+ private Runnability assignRunnabilityRank(TaskCluster goal, Map<TaskCluster, Runnability> runnabilityMap) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Computing runnability: " + goal);
+ }
+ if (runnabilityMap.containsKey(goal)) {
+ return runnabilityMap.get(goal);
+ }
+ TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(goal);
+ if (lastAttempt != null) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Last Attempt Status: " + lastAttempt.getStatus());
+ }
+ if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+ Runnability runnability = new Runnability(Runnability.Tag.COMPLETED, Integer.MIN_VALUE);
+ runnabilityMap.put(goal, runnability);
+ return runnability;
+ }
+ if (lastAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING) {
+ Runnability runnability = new Runnability(Runnability.Tag.RUNNING, Integer.MIN_VALUE);
+ runnabilityMap.put(goal, runnability);
+ return runnability;
+ }
+ }
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap = jobRun.getConnectorPolicyMap();
+ PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+ Runnability aggregateRunnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
+ for (PartitionId pid : goal.getRequiredPartitions()) {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inspecting required partition: " + pid);
+ }
+ Runnability runnability;
+ ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
+ IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
+ PartitionState maxState = pmm.getMaximumAvailableState(pid);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Policy: " + cPolicy + " maxState: " + maxState);
+ }
+ if (PartitionState.COMMITTED.equals(maxState)) {
+ runnability = new Runnability(Runnability.Tag.RUNNABLE, 0);
+ } else if (PartitionState.STARTED.equals(maxState) && !cPolicy.consumerWaitsForProducerToFinish()) {
+ runnability = new Runnability(Runnability.Tag.RUNNABLE, 1);
+ } else {
+ runnability = assignRunnabilityRank(partitionProducingTaskClusterMap.get(pid), runnabilityMap);
+ switch (runnability.getTag()) {
+ case RUNNABLE:
+ if (cPolicy.consumerWaitsForProducerToFinish()) {
+ runnability = new Runnability(Runnability.Tag.NOT_RUNNABLE, Integer.MAX_VALUE);
+ } else {
+ runnability = new Runnability(Runnability.Tag.RUNNABLE, runnability.getPriority() + 1);
+ }
+ break;
+
+ case NOT_RUNNABLE:
+ break;
+
+ case RUNNING:
+ if (cPolicy.consumerWaitsForProducerToFinish()) {
+ runnability = new Runnability(Runnability.Tag.NOT_RUNNABLE, Integer.MAX_VALUE);
+ } else {
+ runnability = new Runnability(Runnability.Tag.RUNNABLE, 1);
+ }
+ break;
+ }
+ }
+ aggregateRunnability = Runnability.getWorstCase(aggregateRunnability, runnability);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("aggregateRunnability: " + aggregateRunnability);
+ }
+ }
+ runnabilityMap.put(goal, aggregateRunnability);
+ return aggregateRunnability;
+ }
+
+ private void assignTaskLocations(TaskCluster tc, Map<String, List<TaskAttemptDescriptor>> taskAttemptMap)
+ throws HyracksException {
+ JobActivityGraph jag = jobRun.getJobActivityGraph();
+ Task[] tasks = tc.getTasks();
+ List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
+ int attempts = tcAttempts.size();
+ TaskClusterAttempt tcAttempt = new TaskClusterAttempt(tc, attempts);
+ TaskAttempt[] taskAttempts = new TaskAttempt[tasks.length];
+ Map<TaskId, LValueConstraintExpression> locationMap = new HashMap<TaskId, LValueConstraintExpression>();
+ for (int i = 0; i < tasks.length; ++i) {
+ Task ts = tasks[i];
+ TaskId tid = ts.getTaskId();
+ TaskAttempt taskAttempt = new TaskAttempt(tcAttempt, new TaskAttemptId(new TaskId(tid.getActivityId(),
+ tid.getPartition()), attempts), ts);
+ taskAttempt.setStatus(TaskAttempt.TaskStatus.INITIALIZED, null);
+ locationMap.put(tid,
+ new PartitionLocationExpression(tid.getActivityId().getOperatorDescriptorId(), tid.getPartition()));
+ taskAttempts[i] = taskAttempt;
+ }
+ tcAttempt.setTaskAttempts(taskAttempts);
+ solver.solve(locationMap.values());
+ for (int i = 0; i < tasks.length; ++i) {
+ Task ts = tasks[i];
+ TaskId tid = ts.getTaskId();
+ TaskAttempt taskAttempt = taskAttempts[i];
+ ActivityId aid = tid.getActivityId();
+ Set<ActivityId> blockers = jag.getBlocked2BlockerMap().get(aid);
+ String nodeId = null;
+ if (blockers != null) {
+ for (ActivityId blocker : blockers) {
+ nodeId = findLocationOfBlocker(jobRun, jag, new TaskId(blocker, tid.getPartition()));
+ if (nodeId != null) {
+ break;
+ }
+ }
+ }
+ Set<String> liveNodes = ccs.getNodeMap().keySet();
+ if (nodeId == null) {
+ LValueConstraintExpression pLocationExpr = locationMap.get(tid);
+ Object location = solver.getValue(pLocationExpr);
+ if (location == null) {
+ // pick any
+ nodeId = liveNodes.toArray(new String[liveNodes.size()])[Math.abs(new Random().nextInt())
+ % liveNodes.size()];
+ } else if (location instanceof String) {
+ nodeId = (String) location;
+ } else if (location instanceof String[]) {
+ for (String choice : (String[]) location) {
+ if (liveNodes.contains(choice)) {
+ nodeId = choice;
+ break;
+ }
+ }
+ if (nodeId == null) {
+ throw new HyracksException("No satisfiable location found for "
+ + taskAttempt.getTaskAttemptId());
+ }
+ } else {
+ throw new HyracksException("Unknown type of value for " + pLocationExpr + ": " + location + "("
+ + location.getClass() + ")");
+ }
+ }
+ if (nodeId == null) {
+ throw new HyracksException("No satisfiable location found for " + taskAttempt.getTaskAttemptId());
+ }
+ if (!liveNodes.contains(nodeId)) {
+ throw new HyracksException("Node " + nodeId + " not live");
+ }
+ taskAttempt.setNodeId(nodeId);
+ taskAttempt.setStatus(TaskAttempt.TaskStatus.RUNNING, null);
+ List<TaskAttemptDescriptor> tads = taskAttemptMap.get(nodeId);
+ if (tads == null) {
+ tads = new ArrayList<TaskAttemptDescriptor>();
+ taskAttemptMap.put(nodeId, tads);
+ }
+ ActivityPartitionDetails apd = ts.getActivityPlan().getActivityPartitionDetails();
+ tads.add(new TaskAttemptDescriptor(taskAttempt.getTaskAttemptId(), apd.getPartitionCount(), apd
+ .getInputPartitionCounts(), apd.getOutputPartitionCounts()));
+ }
+ tcAttempt.initializePendingTaskCounter();
+ tcAttempts.add(tcAttempt);
+ tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.RUNNING);
+ inProgressTaskClusters.add(tc);
+ }
+
+ private static String findLocationOfBlocker(JobRun jobRun, JobActivityGraph jag, TaskId tid) {
+ ActivityId blockerAID = tid.getActivityId();
+ ActivityCluster blockerAC = jobRun.getActivityClusterMap().get(blockerAID);
+ Task[] blockerTasks = blockerAC.getPlan().getActivityPlanMap().get(blockerAID).getTasks();
+ List<TaskClusterAttempt> tcAttempts = blockerTasks[tid.getPartition()].getTaskCluster().getAttempts();
+ if (tcAttempts == null || tcAttempts.isEmpty()) {
+ return null;
+ }
+ TaskClusterAttempt lastTCA = tcAttempts.get(tcAttempts.size() - 1);
+ for (TaskAttempt ta : lastTCA.getTaskAttempts()) {
+ TaskId blockerTID = ta.getTaskAttemptId().getTaskId();
+ if (tid.equals(blockerTID)) {
+ return ta.getNodeId();
+ }
+ }
+ return null;
+ }
+
+ private static TaskClusterAttempt findLastTaskClusterAttempt(TaskCluster tc) {
+ List<TaskClusterAttempt> attempts = tc.getAttempts();
+ if (!attempts.isEmpty()) {
+ return attempts.get(attempts.size() - 1);
+ }
+ return null;
+ }
+
+ private void startTasks(Map<String, List<TaskAttemptDescriptor>> taskAttemptMap) throws HyracksException {
+ Executor executor = ccs.getExecutor();
+ final JobId jobId = jobRun.getJobId();
+ final JobActivityGraph jag = jobRun.getJobActivityGraph();
+ final String appName = jag.getApplicationName();
+ final Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = jobRun.getConnectorPolicyMap();
+ for (Map.Entry<String, List<TaskAttemptDescriptor>> e : taskAttemptMap.entrySet()) {
+ String nodeId = e.getKey();
+ final List<TaskAttemptDescriptor> taskDescriptors = e.getValue();
+ final NodeControllerState node = ccs.getNodeMap().get(nodeId);
+ if (node != null) {
+ node.getActiveJobIds().add(jobRun.getJobId());
+ jobRun.getParticipatingNodeIds().add(nodeId);
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ node.getNodeController().startTasks(appName, jobId, JavaSerializationUtils.serialize(jag),
+ taskDescriptors, connectorPolicies, null);
+ } catch (IOException e) {
+ e.printStackTrace();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+ }
+ }
+
+ private void abortJob(Exception exception) {
+ for (TaskCluster tc : inProgressTaskClusters) {
+ abortTaskCluster(findLastTaskClusterAttempt(tc));
+ }
+ jobRun.setStatus(JobStatus.FAILURE, exception);
+ }
+
+ private void abortTaskCluster(TaskClusterAttempt tcAttempt) {
+ Set<TaskAttemptId> abortTaskIds = new HashSet<TaskAttemptId>();
+ Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+ for (TaskAttempt ta : tcAttempt.getTaskAttempts()) {
+ TaskAttemptId taId = ta.getTaskAttemptId();
+ abortTaskIds.add(taId);
+ if (ta.getStatus() == TaskAttempt.TaskStatus.RUNNING) {
+ ta.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
+ List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta.getNodeId());
+ if (abortTaskAttempts == null) {
+ abortTaskAttempts = new ArrayList<TaskAttemptId>();
+ abortTaskAttemptMap.put(ta.getNodeId(), abortTaskAttempts);
+ }
+ abortTaskAttempts.add(taId);
+ }
+ }
+ final JobId jobId = jobRun.getJobId();
+ for (Map.Entry<String, List<TaskAttemptId>> e : abortTaskAttemptMap.entrySet()) {
+ final NodeControllerState node = ccs.getNodeMap().get(e.getKey());
+ final List<TaskAttemptId> abortTaskAttempts = e.getValue();
+ if (node != null) {
+ LOGGER.info("Aborting: " + abortTaskAttempts + " at " + e.getKey());
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ node.getNodeController().abortTasks(jobId, abortTaskAttempts);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+ }
+ inProgressTaskClusters.remove(tcAttempt.getTaskCluster());
+ TaskCluster tc = tcAttempt.getTaskCluster();
+ PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+ pmm.removeUncommittedPartitions(tc.getProducedPartitions(), abortTaskIds);
+ pmm.removePartitionRequests(tc.getRequiredPartitions(), abortTaskIds);
+ }
+
+ private void abortDoomedTaskClusters() throws HyracksException {
+ Set<TaskCluster> doomedTaskClusters = new HashSet<TaskCluster>();
+ for (TaskCluster tc : inProgressTaskClusters) {
+ // Start search at TCs that produce no outputs (sinks)
+ if (tc.getProducedPartitions().isEmpty()) {
+ findDoomedTaskClusters(tc, doomedTaskClusters);
+ }
+ }
+
+ for (TaskCluster tc : doomedTaskClusters) {
+ TaskClusterAttempt tca = findLastTaskClusterAttempt(tc);
+ abortTaskCluster(tca);
+ tca.setStatus(TaskClusterAttempt.TaskClusterStatus.ABORTED);
+ }
+ }
+
+ private boolean findDoomedTaskClusters(TaskCluster tc, Set<TaskCluster> doomedTaskClusters) {
+ if (doomedTaskClusters.contains(tc)) {
+ return true;
+ }
+ TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+ if (lastAttempt != null) {
+ switch (lastAttempt.getStatus()) {
+ case ABORTED:
+ case FAILED:
+ return true;
+
+ case COMPLETED:
+ return false;
+ }
+ }
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicyMap = jobRun.getConnectorPolicyMap();
+ PartitionMatchMaker pmm = jobRun.getPartitionMatchMaker();
+ boolean doomed = false;
+ for (TaskCluster depTC : tc.getDependencyTaskClusters()) {
+ if (findDoomedTaskClusters(depTC, doomedTaskClusters)) {
+ doomed = true;
+ }
+ }
+ for (PartitionId pid : tc.getRequiredPartitions()) {
+ ConnectorDescriptorId cdId = pid.getConnectorDescriptorId();
+ IConnectorPolicy cPolicy = connectorPolicyMap.get(cdId);
+ PartitionState maxState = pmm.getMaximumAvailableState(pid);
+ if (maxState == null
+ || (cPolicy.consumerWaitsForProducerToFinish() && maxState != PartitionState.COMMITTED)) {
+ if (findDoomedTaskClusters(partitionProducingTaskClusterMap.get(pid), doomedTaskClusters)) {
+ doomed = true;
+ }
+ }
+ }
+ if (doomed) {
+ doomedTaskClusters.add(tc);
+ }
+ return doomed;
+ }
+
+ public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
+ TaskAttemptId taId = ta.getTaskAttemptId();
+ TaskCluster tc = ta.getTaskState().getTaskCluster();
+ TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+ if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
+ TaskAttempt.TaskStatus taStatus = ta.getStatus();
+ if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+ ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+ if (lastAttempt.decrementPendingTasksCounter() == 0) {
+ lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+ inProgressTaskClusters.remove(tc);
+ startRunnableActivityClusters();
+ }
+ } else {
+ LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
+ }
+ } else {
+ LOGGER.warning("Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+ }
+ }
+
+ /**
+ * Indicates that a single task attempt has encountered a failure.
+ *
+ * @param ta
+ * - Failed Task Attempt
+ * @param ac
+ * - Activity Cluster that owns this Task
+ * @param exception
+ * - Cause of the failure
+ */
+ public void notifyTaskFailure(TaskAttempt ta, ActivityCluster ac, Exception exception) {
+ try {
+ TaskAttemptId taId = ta.getTaskAttemptId();
+ TaskCluster tc = ta.getTaskState().getTaskCluster();
+ TaskClusterAttempt lastAttempt = findLastTaskClusterAttempt(tc);
+ if (lastAttempt != null && taId.getAttempt() == lastAttempt.getAttempt()) {
+ TaskAttempt.TaskStatus taStatus = ta.getStatus();
+ if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+ ta.setStatus(TaskAttempt.TaskStatus.FAILED, exception);
+ abortTaskCluster(lastAttempt);
+ lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
+ abortDoomedTaskClusters();
+ if (lastAttempt.getAttempt() >= ac.getMaxTaskClusterAttempts()) {
+ abortJob(null);
+ return;
+ }
+ startRunnableActivityClusters();
+ } else {
+ LOGGER.warning("Spurious task failure notification: " + taId + " Current state = " + taStatus);
+ }
+ } else {
+ LOGGER.warning("Ignoring task failure notification: " + taId + " -- Current last attempt = "
+ + lastAttempt);
+ }
+ } catch (Exception e) {
+ abortJob(e);
+ }
+ }
+
+ /**
+ * Indicates that the provided set of nodes have left the cluster.
+ *
+ * @param deadNodes
+ * - Set of failed nodes
+ */
+ public void notifyNodeFailures(Set<String> deadNodes) {
+ try {
+ jobRun.getPartitionMatchMaker().notifyNodeFailures(deadNodes);
+ for (ActivityCluster ac : jobRun.getActivityClusters()) {
+ TaskCluster[] taskClusters = ac.getPlan().getTaskClusters();
+ if (taskClusters != null) {
+ for (TaskCluster tc : taskClusters) {
+ TaskClusterAttempt lastTaskClusterAttempt = findLastTaskClusterAttempt(tc);
+ if (lastTaskClusterAttempt != null
+ && (lastTaskClusterAttempt.getStatus() == TaskClusterAttempt.TaskClusterStatus.COMPLETED || lastTaskClusterAttempt
+ .getStatus() == TaskClusterAttempt.TaskClusterStatus.RUNNING)) {
+ boolean abort = false;
+ for (TaskAttempt ta : lastTaskClusterAttempt.getTaskAttempts()) {
+ assert (ta.getStatus() == TaskAttempt.TaskStatus.COMPLETED || ta.getStatus() == TaskAttempt.TaskStatus.RUNNING);
+ if (deadNodes.contains(ta.getNodeId())) {
+ ta.setStatus(TaskAttempt.TaskStatus.FAILED,
+ new HyracksException("Node " + ta.getNodeId() + " failed"));
+ abort = true;
+ }
+ }
+ if (abort) {
+ abortTaskCluster(lastTaskClusterAttempt);
+ }
+ }
+ }
+ abortDoomedTaskClusters();
+ }
+ }
+ startRunnableActivityClusters();
+ } catch (Exception e) {
+ abortJob(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
new file mode 100644
index 0000000..ea10562
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/PartitionConstraintSolver.java
@@ -0,0 +1,125 @@
+/*
+ * 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.scheduler;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
+import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
+
+public class PartitionConstraintSolver {
+ private final Map<LValueConstraintExpression, Set<ConstraintExpression>> constraints;
+
+ public PartitionConstraintSolver() {
+ constraints = new HashMap<LValueConstraintExpression, Set<ConstraintExpression>>();
+ }
+
+ public void addConstraints(Collection<Constraint> constraintCollection) {
+ for (Constraint c : constraintCollection) {
+ addConstraint(c);
+ }
+ }
+
+ public void addConstraint(Constraint c) {
+ Set<ConstraintExpression> rValues = constraints.get(c.getLValue());
+ if (rValues == null) {
+ rValues = new HashSet<ConstraintExpression>();
+ constraints.put(c.getLValue(), rValues);
+ }
+ rValues.add(c.getRValue());
+ }
+
+ public void solve(Collection<LValueConstraintExpression> targetSet) {
+ Set<LValueConstraintExpression> inProcess = new HashSet<LValueConstraintExpression>();
+ for (LValueConstraintExpression lv : targetSet) {
+ solveLValue(lv, inProcess);
+ }
+ }
+
+ private Solution solve(ConstraintExpression ce, Set<LValueConstraintExpression> inProcess) {
+ switch (ce.getTag()) {
+ case CONSTANT:
+ return new Solution(((ConstantExpression) ce).getValue(), Solution.Status.FOUND);
+
+ case PARTITION_COUNT:
+ case PARTITION_LOCATION:
+ return solveLValue((LValueConstraintExpression) ce, inProcess);
+ }
+ return null;
+ }
+
+ private Solution solveLValue(LValueConstraintExpression lv, Set<LValueConstraintExpression> inProcess) {
+ if (inProcess.contains(lv)) {
+ return new Solution(null, Solution.Status.CYCLE);
+ }
+ Solution result = null;
+ inProcess.add(lv);
+ Set<ConstraintExpression> rValues = constraints.get(lv);
+ if (rValues == null) {
+ return new Solution(null, Solution.Status.NOT_BOUND);
+ }
+ for (ConstraintExpression ce : rValues) {
+ Solution solution = solve(ce, inProcess);
+ if (solution != null && solution.status == Solution.Status.FOUND) {
+ result = solution;
+ break;
+ }
+ }
+ if (result != null) {
+ rValues.clear();
+ rValues.add(new ConstantExpression(result.value));
+ }
+ inProcess.remove(lv);
+ return result;
+ }
+
+ public Object getValue(LValueConstraintExpression lValue) {
+ Set<ConstraintExpression> rValues = constraints.get(lValue);
+ if (rValues == null) {
+ return null;
+ }
+ if (rValues.size() != 1) {
+ return null;
+ }
+ for (ConstraintExpression ce : rValues) {
+ if (ce.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
+ return ((ConstantExpression) ce).getValue();
+ }
+ }
+ return null;
+ }
+
+ private static class Solution {
+ enum Status {
+ FOUND,
+ CYCLE,
+ NOT_BOUND,
+ }
+
+ final Object value;
+ final Status status;
+
+ public Solution(Object value, Status status) {
+ this.value = value;
+ this.status = status;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.java
new file mode 100644
index 0000000..27e9480
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/RankedRunnableTaskCluster.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.scheduler;
+
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+
+public class RankedRunnableTaskCluster implements Comparable<RankedRunnableTaskCluster> {
+ private final int rank;
+
+ private final TaskCluster taskCluster;
+
+ public RankedRunnableTaskCluster(int rank, TaskCluster taskCluster) {
+ this.rank = rank;
+ this.taskCluster = taskCluster;
+ }
+
+ public int getRank() {
+ return rank;
+ }
+
+ public TaskCluster getTaskCluster() {
+ return taskCluster;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + rank + ":" + taskCluster + "]";
+ }
+
+ @Override
+ public int compareTo(RankedRunnableTaskCluster o) {
+ int cmp = rank - o.rank;
+ return cmp < 0 ? -1 : (cmp > 0 ? 1 : 0);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.java
new file mode 100644
index 0000000..56ae9c3
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/Runnability.java
@@ -0,0 +1,101 @@
+/*
+ * 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.scheduler;
+
+public final class Runnability {
+ private final Tag tag;
+
+ private final int priority;
+
+ public Runnability(Tag tag, int priority) {
+ this.tag = tag;
+ this.priority = priority;
+ }
+
+ public Tag getTag() {
+ return tag;
+ }
+
+ public int getPriority() {
+ return priority;
+ }
+
+ public enum Tag {
+ COMPLETED,
+ NOT_RUNNABLE,
+ RUNNABLE,
+ RUNNING,
+ }
+
+ public static Runnability getWorstCase(Runnability r1, Runnability r2) {
+ switch (r1.tag) {
+ case COMPLETED:
+ switch (r2.tag) {
+ case COMPLETED:
+ case NOT_RUNNABLE:
+ case RUNNABLE:
+ case RUNNING:
+ return r2;
+ }
+ break;
+
+ case NOT_RUNNABLE:
+ switch (r2.tag) {
+ case COMPLETED:
+ case NOT_RUNNABLE:
+ case RUNNABLE:
+ case RUNNING:
+ return r1;
+ }
+ break;
+
+ case RUNNABLE:
+ switch (r2.tag) {
+ case COMPLETED:
+ return r1;
+
+ case RUNNING:
+ return r1.priority > 0 ? r1 : new Runnability(Tag.RUNNABLE, 1);
+
+ case NOT_RUNNABLE:
+ return r2;
+
+ case RUNNABLE:
+ return r1.priority > r2.priority ? r1 : r2;
+ }
+ break;
+
+ case RUNNING:
+ switch (r2.tag) {
+ case COMPLETED:
+ case RUNNING:
+ return r1;
+
+ case NOT_RUNNABLE:
+ return r2;
+
+ case RUNNABLE:
+ return r2.priority > 0 ? r2 : new Runnability(Tag.RUNNABLE, 1);
+ }
+ break;
+ }
+ throw new IllegalArgumentException("Could not aggregate: " + r1 + " and " + r2);
+ }
+
+ @Override
+ public String toString() {
+ return "{" + tag + ", " + priority + "}";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
deleted file mode 100644
index c433bf9..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
+++ /dev/null
@@ -1,208 +0,0 @@
-package edu.uci.ics.hyracks.control.cc.scheduler.naive;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.constraints.expressions.BelongsToExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression.ExpressionTag;
-import edu.uci.ics.hyracks.api.constraints.expressions.EnumeratedCollectionExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobAttempt;
-import edu.uci.ics.hyracks.control.cc.job.JobStageAttempt;
-import edu.uci.ics.hyracks.control.cc.scheduler.IJobAttemptSchedulerState;
-import edu.uci.ics.hyracks.control.cc.scheduler.ISchedule;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-
-public class NaiveScheduler implements IScheduler {
- private static final Logger LOGGER = Logger.getLogger(NaiveScheduler.class.getName());
-
- private final ClusterControllerService ccs;
-
- public NaiveScheduler(ClusterControllerService ccs) {
- this.ccs = ccs;
- }
-
- @Override
- public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja) {
- return new JobAttemptState(ja.getJobRun().getConstraints());
- }
-
- @Override
- public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException {
- for (JobStageAttempt jsa : runnableStageAttempts) {
- Set<OperatorDescriptorId> operators = new HashSet<OperatorDescriptorId>();
- for (ActivityNodeId aid : jsa.getJobStage().getTasks()) {
- operators.add(aid.getOperatorDescriptorId());
- }
- jsa.setSchedule(computeSchedule(jsa, operators));
- }
- }
-
- private ISchedule computeSchedule(JobStageAttempt jsa, Set<OperatorDescriptorId> operators) throws HyracksException {
- Set<String> nodeSet = ccs.getNodeMap().keySet();
- if (nodeSet.isEmpty()) {
- throw new HyracksException("0 usable nodes found");
- }
- String[] liveNodes = ccs.getNodeMap().keySet().toArray(new String[nodeSet.size()]);
- JobAttempt ja = jsa.getJobAttempt();
- final JobAttemptState jas = (JobAttemptState) ja.getSchedulerState();
-
- List<PartitionLocationExpression> rrAssignment = new ArrayList<PartitionLocationExpression>();
-
- for (OperatorDescriptorId oid : operators) {
- String[] opParts = null;
- if (!jas.allocations.containsKey(oid)) {
- Set<ConstraintExpression> opConstraints = jas.opConstraints.get(oid);
- for (ConstraintExpression ce : opConstraints) {
- int nParts = getNumPartitions(oid, ce);
- if (nParts != -1) {
- opParts = new String[nParts];
- break;
- }
- }
- if (opParts == null) {
- throw new HyracksException("Unable to satisfy constraints for operator : " + oid);
- }
- jas.allocations.put(oid, opParts);
- BitSet unassignedPartsIds = new BitSet(opParts.length);
- unassignedPartsIds.set(0, opParts.length);
- for (ConstraintExpression ce : opConstraints) {
- if (ce.getTag() == ConstraintExpression.ExpressionTag.BELONGS_TO) {
- BelongsToExpression bE = (BelongsToExpression) ce;
- if (bE.getItemExpression().getTag() == ConstraintExpression.ExpressionTag.PARTITION_LOCATION) {
- PartitionLocationExpression plE = (PartitionLocationExpression) bE.getItemExpression();
- if (plE.getOperatorDescriptorId().equals(oid)) {
- int part = plE.getPartition();
- if (bE.getSetExpression().getTag() == ConstraintExpression.ExpressionTag.ENUMERATED_SET) {
- EnumeratedCollectionExpression ecE = (EnumeratedCollectionExpression) bE
- .getSetExpression();
- for (ConstraintExpression value : ecE.getMembers()) {
- if (value.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
- ConstantExpression nodeConst = (ConstantExpression) value;
- String nodeId = (String) nodeConst.getValue();
- if (nodeSet.contains(nodeId)) {
- opParts[part] = nodeId;
- unassignedPartsIds.clear(part);
- LOGGER.info("Assigned: " + oid + ":" + part + ": " + nodeId);
- break;
- }
- }
- }
- }
- if (unassignedPartsIds.get(part)) {
- throw new HyracksException("Unsatisfiable constraint for operator: " + oid);
- }
- }
- }
- }
- }
-
- if (!unassignedPartsIds.isEmpty()) {
- // Do round robin assignment.
- for (int i = unassignedPartsIds.nextSetBit(0); i >= 0; i = unassignedPartsIds.nextSetBit(i + 1)) {
- rrAssignment.add(new PartitionLocationExpression(oid, i));
- }
- }
- }
- }
- int n = rrAssignment.size();
- for (int i = 0; i < n; ++i) {
- PartitionLocationExpression plE = rrAssignment.get(i);
- String[] opParts = jas.allocations.get(plE.getOperatorDescriptorId());
- String node = liveNodes[i % liveNodes.length];
- LOGGER.info("Assigned: " + plE.getOperatorDescriptorId() + ":" + plE.getPartition() + ": " + node);
- opParts[plE.getPartition()] = node;
- }
- return new ISchedule() {
- @Override
- public String[] getPartitions(ActivityNodeId aid) {
- return jas.allocations.get(aid.getOperatorDescriptorId());
- }
- };
- }
-
- private int getNumPartitions(OperatorDescriptorId oid, ConstraintExpression ce) {
- if (ce.getTag() == ExpressionTag.RELATIONAL) {
- RelationalExpression re = (RelationalExpression) ce;
- if (re.getOperator() == RelationalExpression.Operator.EQUAL) {
- if (re.getLeft().getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
- return getNumPartitions(oid, (PartitionCountExpression) re.getLeft(), re.getRight());
- } else if (re.getRight().getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
- return getNumPartitions(oid, (PartitionCountExpression) re.getRight(), re.getLeft());
- }
- }
- }
- return -1;
- }
-
- private int getNumPartitions(OperatorDescriptorId oid, PartitionCountExpression pce, ConstraintExpression ce) {
- if (pce.getOperatorDescriptorId().equals(oid)) {
- if (ce.getTag() == ConstraintExpression.ExpressionTag.CONSTANT) {
- ConstantExpression constExpr = (ConstantExpression) ce;
- Integer n = (Integer) constExpr.getValue();
- return n.intValue();
- }
- }
- return -1;
- }
-
- private static class JobAttemptState implements IJobAttemptSchedulerState {
- final Map<OperatorDescriptorId, String[]> allocations;
- final Map<OperatorDescriptorId, Set<ConstraintExpression>> opConstraints;
-
- public JobAttemptState(Set<ConstraintExpression> constraints) {
- allocations = new HashMap<OperatorDescriptorId, String[]>();
- opConstraints = new HashMap<OperatorDescriptorId, Set<ConstraintExpression>>();
- List<ConstraintExpression> ceList = new ArrayList<ConstraintExpression>();
- for (ConstraintExpression ce : constraints) {
- ceList.clear();
- ceList.add(ce);
- getAllConstraints(ceList);
- for (ConstraintExpression ce2 : ceList) {
- switch (ce2.getTag()) {
- case PARTITION_COUNT:
- addToOpConstraints(opConstraints,
- ((PartitionCountExpression) ce2).getOperatorDescriptorId(), ce);
- break;
-
- case PARTITION_LOCATION:
- addToOpConstraints(opConstraints,
- ((PartitionLocationExpression) ce2).getOperatorDescriptorId(), ce);
- break;
- }
- }
- }
- }
-
- private static void addToOpConstraints(Map<OperatorDescriptorId, Set<ConstraintExpression>> opc,
- OperatorDescriptorId opId, ConstraintExpression ce) {
- Set<ConstraintExpression> opSet = opc.get(opId);
- if (opSet == null) {
- opSet = new HashSet<ConstraintExpression>();
- opc.put(opId, opSet);
- }
- opSet.add(ce);
- }
-
- private static void getAllConstraints(List<ConstraintExpression> ceList) {
- for (int i = 0; i < ceList.size(); ++i) {
- ConstraintExpression cE = ceList.get(i);
- cE.getChildren(ceList);
- }
- }
- }
-}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
index 1a26e60..a0510f9 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/AdminConsoleHandler.java
@@ -27,7 +27,7 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
public class AdminConsoleHandler extends AbstractHandler {
private ClusterControllerService ccs;
@@ -51,7 +51,7 @@
writer.println("<h2>Node Controllers</h2>");
writer.println("<table><tr><td>Node Id</td><td>Host</td></tr>");
try {
- ccs.getJobQueue().scheduleAndSync(new SynchronizableRunnable() {
+ ccs.getJobQueue().scheduleAndSync(new SynchronizableEvent() {
@Override
protected void doRun() throws Exception {
for (Map.Entry<String, NodeControllerState> e : ccs.getNodeMap().entrySet()) {
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
index 65c156c..e047212 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/ApplicationInstallationHandler.java
@@ -28,7 +28,7 @@
import org.eclipse.jetty.server.handler.AbstractHandler;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableRunnable;
+import edu.uci.ics.hyracks.control.cc.jobqueue.SynchronizableEvent;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
public class ApplicationInstallationHandler extends AbstractHandler {
@@ -54,7 +54,7 @@
}
final String appName = parts[0];
if (HttpMethods.PUT.equals(request.getMethod())) {
- class OutputStreamGetter extends SynchronizableRunnable {
+ class OutputStreamGetter extends SynchronizableEvent {
private OutputStream os;
@Override
@@ -78,7 +78,7 @@
r.os.close();
}
} else if (HttpMethods.GET.equals(request.getMethod())) {
- class InputStreamGetter extends SynchronizableRunnable {
+ class InputStreamGetter extends SynchronizableEvent {
private InputStream is;
@Override
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
index 716dfcb..edb4e93 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/RESTAPIFunction.java
@@ -14,11 +14,11 @@
*/
package edu.uci.ics.hyracks.control.cc.web;
-import java.util.UUID;
-
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobActivityGraphJSONEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobProfileJSONEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSpecificationJSONEvent;
import edu.uci.ics.hyracks.control.cc.job.manager.events.GetJobSummariesJSONEvent;
@@ -47,25 +47,22 @@
}
case 2: {
- UUID jobId = UUID.fromString(arguments[0]);
+ JobId jobId = JobId.parse(arguments[0]);
- if ("spec".equalsIgnoreCase(arguments[1])) {
+ if ("job-specification".equalsIgnoreCase(arguments[1])) {
GetJobSpecificationJSONEvent gjse = new GetJobSpecificationJSONEvent(ccs, jobId);
ccs.getJobQueue().scheduleAndSync(gjse);
- result.put("result", gjse.getSpecification());
- }
- break;
- }
-
- case 3: {
- UUID jobId = UUID.fromString(arguments[0]);
- int attempt = Integer.parseInt(arguments[1]);
-
- if ("profile".equalsIgnoreCase(arguments[2])) {
- GetJobProfileJSONEvent gjpe = new GetJobProfileJSONEvent(ccs, jobId, attempt);
+ result.put("result", gjse.getJSON());
+ } else if ("job-activity-graph".equalsIgnoreCase(arguments[1])) {
+ GetJobActivityGraphJSONEvent gjage = new GetJobActivityGraphJSONEvent(ccs, jobId);
+ ccs.getJobQueue().scheduleAndSync(gjage);
+ result.put("result", gjage.getJSON());
+ } else if ("profile".equalsIgnoreCase(arguments[1])) {
+ GetJobProfileJSONEvent gjpe = new GetJobProfileJSONEvent(ccs, jobId);
ccs.getJobQueue().scheduleAndSync(gjpe);
- result.put("result", gjpe.getProfile());
+ result.put("result", gjpe.getJSON());
}
+
break;
}
}
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
index 806a14e..efbc8f7 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/WebServer.java
@@ -46,7 +46,7 @@
}
private void addHandlers() {
- ContextHandler handler = new ContextHandler("/state");
+ ContextHandler handler = new ContextHandler("/rest");
RoutingHandler rh = new RoutingHandler();
rh.addHandler("jobs", new JSONOutputRequestHandler(new RESTAPIFunction(ccs)));
handler.setHandler(rh);
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
new file mode 100644
index 0000000..6ff4901
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -0,0 +1,46 @@
+/*
+ * 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.common.base;
+
+import java.rmi.Remote;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+
+public interface IClusterController extends Remote {
+ public NodeParameters registerNode(NodeRegistration reg) throws Exception;
+
+ public void unregisterNode(INodeController nodeController) throws Exception;
+
+ public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics)
+ throws Exception;
+
+ public void notifyTaskFailure(JobId jobId, TaskAttemptId taskId, String nodeId, Exception e) throws Exception;
+
+ public void nodeHeartbeat(String id) throws Exception;
+
+ public void reportProfile(String id, List<JobProfile> profiles) throws Exception;
+
+ public void registerPartitionProvider(PartitionDescriptor partitionDescriptor) throws Exception;
+
+ public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.java
new file mode 100644
index 0000000..33548d9
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/INodeController.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.control.common.base;
+
+import java.rmi.Remote;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeCapability;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+
+public interface INodeController extends Remote {
+ public String getId() throws Exception;
+
+ public NCConfig getConfiguration() throws Exception;
+
+ public NodeCapability getNodeCapability() throws Exception;
+
+ public void startTasks(String appName, JobId jobId, byte[] planBytes, List<TaskAttemptDescriptor> taskDescriptors,
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies, byte[] ctxVarBytes) throws Exception;
+
+ public void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception;
+
+ public void cleanUpJob(JobId jobId) throws Exception;
+
+ public void notifyRegistration(IClusterController ccs) throws Exception;
+
+ public void createApplication(String appName, boolean deployHar, byte[] serializedDistributedState)
+ throws Exception;
+
+ public void destroyApplication(String appName) throws Exception;
+
+ public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception;
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
similarity index 96%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
index 4d9b5a0..d12d037 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/CCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/CCConfig.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
import org.kohsuke.args4j.Option;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
similarity index 97%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 4a0f64d..c887223 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NCConfig.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
import java.io.Serializable;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
similarity index 94%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
index 88efc1f..7ad5fa1 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeCapability.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
import java.io.Serializable;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
similarity index 96%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
index 8745b32..0161f96 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeParameters.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeParameters.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.controllers;
import java.io.Serializable;
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
new file mode 100644
index 0000000..374cf48
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -0,0 +1,55 @@
+/*
+ * 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.common.controllers;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
+
+public final class NodeRegistration implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final INodeController nc;
+
+ private final String nodeId;
+
+ private final NCConfig ncConfig;
+
+ private final NetworkAddress dataPort;
+
+ public NodeRegistration(INodeController nc, String nodeId, NCConfig ncConfig, NetworkAddress dataPort) {
+ this.nc = nc;
+ this.nodeId = nodeId;
+ this.ncConfig = ncConfig;
+ this.dataPort = dataPort;
+ }
+
+ public INodeController getNodeController() {
+ return nc;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public NCConfig getNCConfig() {
+ return ncConfig;
+ }
+
+ public NetworkAddress getDataPort() {
+ return dataPort;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.java
new file mode 100644
index 0000000..9fc0916
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionDescriptor implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final PartitionId pid;
+
+ private final String nodeId;
+
+ private final TaskAttemptId producingTaskAttemptId;
+
+ private final boolean reusable;
+
+ private PartitionState state;
+
+ public PartitionDescriptor(PartitionId pid, String nodeId, TaskAttemptId producingTaskAttemptId, boolean reusable) {
+ this.pid = pid;
+ this.nodeId = nodeId;
+ this.producingTaskAttemptId = producingTaskAttemptId;
+ this.reusable = reusable;
+ }
+
+ public PartitionId getPartitionId() {
+ return pid;
+ }
+
+ public String getNodeId() {
+ return nodeId;
+ }
+
+ public TaskAttemptId getProducingTaskAttemptId() {
+ return producingTaskAttemptId;
+ }
+
+ public PartitionState getState() {
+ return state;
+ }
+
+ public void setState(PartitionState state) {
+ this.state = state;
+ }
+
+ public boolean isReusable() {
+ return reusable;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + pid + ":" + nodeId + ":" + producingTaskAttemptId + (reusable ? "reusable" : "non-reusable") + " "
+ + state + "]";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.java
new file mode 100644
index 0000000..ca34501
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionRequest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PartitionRequest implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final PartitionId pid;
+
+ private final String requestingNodeId;
+
+ private final TaskAttemptId requestingTaskAttemptId;
+
+ private final PartitionState minState;
+
+ public PartitionRequest(PartitionId pid, String requestingNodeId, TaskAttemptId requestingTaskAttemptId,
+ PartitionState minState) {
+ this.pid = pid;
+ this.requestingNodeId = requestingNodeId;
+ this.requestingTaskAttemptId = requestingTaskAttemptId;
+ this.minState = minState;
+ }
+
+ public PartitionId getPartitionId() {
+ return pid;
+ }
+
+ public String getNodeId() {
+ return requestingNodeId;
+ }
+
+ public TaskAttemptId getRequestingTaskAttemptId() {
+ return requestingTaskAttemptId;
+ }
+
+ public PartitionState getMinimumState() {
+ return minState;
+ }
+
+ @Override
+ public String toString() {
+ return "[" + pid + ":" + requestingNodeId + ":" + requestingTaskAttemptId + ":" + minState + "]";
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
similarity index 64%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
index 88efc1f..edde0b8 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/control/NodeCapability.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/PartitionState.java
@@ -12,20 +12,20 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.control;
+package edu.uci.ics.hyracks.control.common.job;
-import java.io.Serializable;
+public enum PartitionState {
+ STARTED,
+ COMMITTED;
-public class NodeCapability implements Serializable {
- private static final long serialVersionUID = 1L;
+ public boolean isAtLeast(PartitionState minState) {
+ switch (this) {
+ case COMMITTED:
+ return true;
- private int cpuCount;
-
- public int getCPUCount() {
- return cpuCount;
- }
-
- public void setCPUCount(int cpuCount) {
- this.cpuCount = cpuCount;
+ case STARTED:
+ return minState == STARTED;
+ }
+ return false;
}
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.java
new file mode 100644
index 0000000..15ba75c
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/TaskAttemptDescriptor.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.control.common.job;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskAttemptDescriptor implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final TaskAttemptId taId;
+
+ private final int nPartitions;
+
+ private final int[] nInputPartitions;
+
+ private final int[] nOutputPartitions;
+
+ public TaskAttemptDescriptor(TaskAttemptId taId, int nPartitions, int[] nInputPartitions, int[] nOutputPartitions) {
+ this.taId = taId;
+ this.nPartitions = nPartitions;
+ this.nInputPartitions = nInputPartitions;
+ this.nOutputPartitions = nOutputPartitions;
+ }
+
+ public TaskAttemptId getTaskAttemptId() {
+ return taId;
+ }
+
+ public int getPartitionCount() {
+ return nPartitions;
+ }
+
+ public int[] getInputPartitionCounts() {
+ return nInputPartitions;
+ }
+
+ public int[] getOutputPartitionCounts() {
+ return nOutputPartitions;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
similarity index 95%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
index 6ef57a6..74d6000 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/AbstractProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/AbstractProfile.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
import java.io.Serializable;
import java.util.HashMap;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
similarity index 78%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index 53ad0ad..5457ac5 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -1,35 +1,29 @@
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
import java.util.HashMap;
import java.util.Map;
-import java.util.UUID;
import org.json.JSONException;
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.job.JobId;
+
public class JobProfile extends AbstractProfile {
private static final long serialVersionUID = 1L;
- private final UUID jobId;
-
- private final int attempt;
+ private final JobId jobId;
private final Map<String, JobletProfile> jobletProfiles;
- public JobProfile(UUID jobId, int attempt) {
+ public JobProfile(JobId jobId) {
this.jobId = jobId;
- this.attempt = attempt;
jobletProfiles = new HashMap<String, JobletProfile>();
}
- public UUID getJobId() {
+ public JobId getJobId() {
return jobId;
}
- public int getAttempt() {
- return attempt;
- }
-
public Map<String, JobletProfile> getJobletProfiles() {
return jobletProfiles;
}
@@ -40,7 +34,6 @@
json.put("type", "job-profile");
json.put("job-id", jobId.toString());
- json.put("attempt", attempt);
populateCounters(json);
for (JobletProfile p : jobletProfiles.values()) {
json.accumulate("joblets", p.toJSON());
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
similarity index 67%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index 66a203c..5069fc3 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/profiling/om/JobletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -12,33 +12,34 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.job.profiling.om;
+package edu.uci.ics.hyracks.control.common.job.profiling.om;
import java.util.HashMap;
import java.util.Map;
-import java.util.UUID;
import org.json.JSONException;
import org.json.JSONObject;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
public class JobletProfile extends AbstractProfile {
private static final long serialVersionUID = 1L;
private final String nodeId;
- private final Map<UUID, StageletProfile> stageletProfiles;
+ private final Map<TaskAttemptId, TaskProfile> taskProfiles;
public JobletProfile(String nodeId) {
this.nodeId = nodeId;
- stageletProfiles = new HashMap<UUID, StageletProfile>();
+ taskProfiles = new HashMap<TaskAttemptId, TaskProfile>();
}
public String getNodeId() {
return nodeId;
}
- public Map<UUID, StageletProfile> getStageletProfiles() {
- return stageletProfiles;
+ public Map<TaskAttemptId, TaskProfile> getTaskProfiles() {
+ return taskProfiles;
}
@Override
@@ -48,8 +49,8 @@
json.put("type", "joblet-profile");
json.put("node-id", nodeId.toString());
populateCounters(json);
- for (StageletProfile p : stageletProfiles.values()) {
- json.accumulate("stagelets", p.toJSON());
+ for (TaskProfile p : taskProfiles.values()) {
+ json.accumulate("tasks", p.toJSON());
}
return json;
@@ -57,11 +58,11 @@
public void merge(JobletProfile jp) {
super.merge(this);
- for (StageletProfile sp : jp.stageletProfiles.values()) {
- if (stageletProfiles.containsKey(sp.getStageId())) {
- stageletProfiles.get(sp.getStageId()).merge(sp);
+ for (TaskProfile tp : jp.taskProfiles.values()) {
+ if (taskProfiles.containsKey(tp.getTaskId())) {
+ taskProfiles.get(tp.getTaskId()).merge(tp);
} else {
- stageletProfiles.put(sp.getStageId(), sp);
+ taskProfiles.put(tp.getTaskId(), tp);
}
}
}
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
new file mode 100644
index 0000000..3ab1090
--- /dev/null
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.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.common.job.profiling.om;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskProfile extends AbstractProfile {
+ private static final long serialVersionUID = 1L;
+
+ private final TaskAttemptId taskAttemptId;
+
+ public TaskProfile(TaskAttemptId taskAttemptId) {
+ this.taskAttemptId = taskAttemptId;
+ }
+
+ public TaskAttemptId getTaskId() {
+ return taskAttemptId;
+ }
+
+ @Override
+ public JSONObject toJSON() throws JSONException {
+ JSONObject json = new JSONObject();
+
+ json.put("type", "task-profile");
+ json.put("activity-id", taskAttemptId.getTaskId().getActivityId().toString());
+ json.put("partition", taskAttemptId.getTaskId().getPartition());
+ json.put("attempt", taskAttemptId.getAttempt());
+ populateCounters(json);
+
+ return json;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 413243e..3026359 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -15,29 +15,37 @@
package edu.uci.ics.hyracks.control.nc;
import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
import java.util.HashMap;
-import java.util.Hashtable;
import java.util.Map;
-import java.util.UUID;
import java.util.concurrent.Executor;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
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.IIOManager;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.common.job.PartitionRequest;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
public class Joblet implements IHyracksJobletContext, ICounterContext {
@@ -47,54 +55,76 @@
private final INCApplicationContext appCtx;
- private final UUID jobId;
+ private final JobId jobId;
- private final int attempt;
-
- private final Map<UUID, Stagelet> stageletMap;
+ private final Map<PartitionId, IPartitionCollector> partitionRequestMap;
private final Map<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>> envMap;
+ private final Map<TaskAttemptId, Task> taskMap;
+
private final Map<String, Counter> counterMap;
+ private final Map<MultipartName, Object> localVariableMap;
+
private final DefaultDeallocatableRegistry deallocatableRegistry;
private final IWorkspaceFileFactory fileFactory;
- public Joblet(NodeControllerService nodeController, UUID jobId, int attempt, INCApplicationContext appCtx) {
+ public Joblet(NodeControllerService nodeController, JobId jobId, INCApplicationContext appCtx) {
this.nodeController = nodeController;
this.appCtx = appCtx;
this.jobId = jobId;
- this.attempt = attempt;
- stageletMap = new Hashtable<UUID, Stagelet>();
- envMap = new Hashtable<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
- counterMap = new Hashtable<String, Counter>();
+ partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
+ envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
+ taskMap = new HashMap<TaskAttemptId, Task>();
+ counterMap = new HashMap<String, Counter>();
+ localVariableMap = new HashMap<MultipartName, Object>();
deallocatableRegistry = new DefaultDeallocatableRegistry();
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
}
@Override
- public UUID getJobId() {
+ public JobId getJobId() {
return jobId;
}
- public IOperatorEnvironment getEnvironment(IOperatorDescriptor hod, int partition) {
- synchronized (envMap) {
- if (!envMap.containsKey(hod.getOperatorId())) {
- envMap.put(hod.getOperatorId(), new HashMap<Integer, IOperatorEnvironment>());
- }
+ public synchronized IOperatorEnvironment getEnvironment(OperatorDescriptorId opId, int partition) {
+ if (!envMap.containsKey(opId)) {
+ envMap.put(opId, new HashMap<Integer, IOperatorEnvironment>());
}
- Map<Integer, IOperatorEnvironment> opEnvMap = envMap.get(hod.getOperatorId());
+ Map<Integer, IOperatorEnvironment> opEnvMap = envMap.get(opId);
if (!opEnvMap.containsKey(partition)) {
- opEnvMap.put(partition, new OperatorEnvironmentImpl());
+ opEnvMap.put(partition, new OperatorEnvironmentImpl(nodeController.getId()));
}
return opEnvMap.get(partition);
}
+ public void addTask(Task task) {
+ taskMap.put(task.getTaskAttemptId(), task);
+ }
+
+ public Map<TaskAttemptId, Task> getTaskMap() {
+ return taskMap;
+ }
+
+ public synchronized Object lookupLocalVariable(MultipartName name) throws HyracksDataException {
+ if (!localVariableMap.containsKey(name)) {
+ throw new HyracksDataException("Unknown variable: " + name);
+ }
+ return localVariableMap.get(name);
+ }
+
+ public synchronized void setLocalVariable(MultipartName name, Object value) {
+ localVariableMap.put(name, value);
+ }
+
private static final class OperatorEnvironmentImpl implements IOperatorEnvironment {
+ private final String nodeId;
private final Map<String, Object> map;
- public OperatorEnvironmentImpl() {
+ public OperatorEnvironmentImpl(String nodeId) {
+ this.nodeId = nodeId;
map = new HashMap<String, Object>();
}
@@ -107,47 +137,41 @@
public void set(String name, Object value) {
map.put(name, value);
}
- }
- public void setStagelet(UUID stageId, Stagelet stagelet) {
- stageletMap.put(stageId, stagelet);
- }
-
- public Stagelet getStagelet(UUID stageId) throws Exception {
- return stageletMap.get(stageId);
+ public String toString() {
+ return super.toString() + "@" + nodeId;
+ }
}
public Executor getExecutor() {
return nodeController.getExecutor();
}
- public void notifyStageletComplete(UUID stageId, int attempt, StageletProfile stats) throws Exception {
- stageletMap.remove(stageId);
- nodeController.notifyStageComplete(jobId, stageId, attempt, stats);
+ public synchronized void notifyTaskComplete(Task task) throws Exception {
+ taskMap.remove(task);
+ TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId());
+ task.dumpProfile(taskProfile);
+ nodeController.notifyTaskComplete(jobId, task.getTaskAttemptId(), taskProfile);
}
- public void notifyStageletFailed(UUID stageId, int attempt) throws Exception {
- stageletMap.remove(stageId);
- nodeController.notifyStageFailed(jobId, stageId, attempt);
+ public synchronized void notifyTaskFailed(Task task, Exception exception) {
+ taskMap.remove(task);
+ nodeController.notifyTaskFailed(jobId, task.getTaskAttemptId(), exception);
}
public NodeControllerService getNodeController() {
return nodeController;
}
- public void dumpProfile(JobletProfile jProfile) {
- synchronized (counterMap) {
- Map<String, Long> counters = jProfile.getCounters();
- for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
- counters.put(e.getKey(), e.getValue().get());
- }
+ public synchronized void dumpProfile(JobletProfile jProfile) {
+ Map<String, Long> counters = jProfile.getCounters();
+ for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
+ counters.put(e.getKey(), e.getValue().get());
}
- synchronized (stageletMap) {
- for (Stagelet si : stageletMap.values()) {
- StageletProfile sProfile = new StageletProfile(si.getStageId());
- si.dumpProfile(sProfile);
- jProfile.getStageletProfiles().put(si.getStageId(), sProfile);
- }
+ for (Task task : taskMap.values()) {
+ TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId());
+ task.dumpProfile(taskProfile);
+ jProfile.getTaskProfiles().put(task.getTaskAttemptId(), taskProfile);
}
}
@@ -157,11 +181,6 @@
}
@Override
- public int getAttempt() {
- return attempt;
- }
-
- @Override
public ICounterContext getCounterContext() {
return this;
}
@@ -191,23 +210,38 @@
}
@Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
- return fileFactory.createWorkspaceFile(prefix);
- }
-
- public Map<UUID, Stagelet> getStageletMap() {
- return stageletMap;
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createManagedWorkspaceFile(prefix);
}
@Override
- public ICounter getCounter(String name, boolean create) {
- synchronized (counterMap) {
- Counter counter = counterMap.get(name);
- if (counter == null && create) {
- counter = new Counter(name);
- counterMap.put(name, counter);
- }
- return counter;
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createUnmanagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public synchronized ICounter getCounter(String name, boolean create) {
+ Counter counter = counterMap.get(name);
+ if (counter == null && create) {
+ counter = new Counter(name);
+ counterMap.put(name, counter);
+ }
+ return counter;
+ }
+
+ public synchronized void advertisePartitionRequest(TaskAttemptId taId, Collection<PartitionId> pids,
+ IPartitionCollector collector, PartitionState minState) throws Exception {
+ for (PartitionId pid : pids) {
+ partitionRequestMap.put(pid, collector);
+ PartitionRequest req = new PartitionRequest(pid, nodeController.getId(), taId, minState);
+ nodeController.getClusterController().registerPartitionRequest(req);
+ }
+ }
+
+ public synchronized void reportPartitionAvailability(PartitionChannel channel) throws HyracksException {
+ IPartitionCollector collector = partitionRequestMap.get(channel.getPartitionId());
+ if (collector != null) {
+ collector.addPartitions(Collections.singleton(channel));
}
}
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
index d8c978a..dde7abc 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NCDriver.java
@@ -18,7 +18,7 @@
import edu.uci.ics.dcache.client.DCacheClient;
import edu.uci.ics.dcache.client.DCacheClientConfig;
-import edu.uci.ics.hyracks.api.control.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
public class NCDriver {
public static void main(String args[]) throws Exception {
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index be48ae6..4eb1a03 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -19,7 +19,7 @@
import java.io.OutputStream;
import java.io.Serializable;
import java.net.InetAddress;
-import java.nio.ByteBuffer;
+import java.net.InetSocketAddress;
import java.rmi.registry.LocateRegistry;
import java.rmi.registry.Registry;
import java.text.MessageFormat;
@@ -28,11 +28,9 @@
import java.util.Hashtable;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.StringTokenizer;
import java.util.Timer;
import java.util.TimerTask;
-import java.util.UUID;
import java.util.concurrent.Executor;
import java.util.concurrent.Executors;
import java.util.logging.Level;
@@ -47,47 +45,51 @@
import org.apache.http.impl.client.DefaultHttpClient;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.control.IClusterController;
-import edu.uci.ics.hyracks.api.control.INodeController;
-import edu.uci.ics.hyracks.api.control.NCConfig;
-import edu.uci.ics.hyracks.api.control.NodeCapability;
-import edu.uci.ics.hyracks.api.control.NodeParameters;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.Direction;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+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.IEndpointDataWriterFactory;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+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.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.io.IODeviceHandle;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobPlan;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
+import edu.uci.ics.hyracks.control.common.base.IClusterController;
+import edu.uci.ics.hyracks.control.common.base.INodeController;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
+import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
+import edu.uci.ics.hyracks.control.common.controllers.NodeCapability;
+import edu.uci.ics.hyracks.control.common.controllers.NodeParameters;
+import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
+import edu.uci.ics.hyracks.control.common.job.TaskAttemptDescriptor;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.JobletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
-import edu.uci.ics.hyracks.control.nc.comm.ConnectionManager;
-import edu.uci.ics.hyracks.control.nc.comm.DemuxDataReceiveListenerFactory;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
+import edu.uci.ics.hyracks.control.nc.net.ConnectionManager;
+import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
+import edu.uci.ics.hyracks.control.nc.partitions.MaterializedPartitionWriter;
+import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
+import edu.uci.ics.hyracks.control.nc.partitions.PipelinedPartition;
+import edu.uci.ics.hyracks.control.nc.partitions.ReceiveSideMaterializingCollector;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
public class NodeControllerService extends AbstractRemoteService implements INodeController {
@@ -103,13 +105,15 @@
private final NodeCapability nodeCapability;
+ private final PartitionManager partitionManager;
+
private final ConnectionManager connectionManager;
private final Timer timer;
private IClusterController ccs;
- private final Map<UUID, Joblet> jobletMap;
+ private final Map<JobId, Joblet> jobletMap;
private final Executor executor;
@@ -129,13 +133,20 @@
}
nodeCapability = computeNodeCapability();
connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
- jobletMap = new Hashtable<UUID, Joblet>();
+ partitionManager = new PartitionManager(this);
+ connectionManager.setPartitionRequestListener(partitionManager);
+
+ jobletMap = new Hashtable<JobId, Joblet>();
timer = new Timer(true);
serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
NodeControllerService.class.getName()), id));
applications = new Hashtable<String, NCApplicationContext>();
}
+ public IHyracksRootContext getRootContext() {
+ return ctx;
+ }
+
private static List<IODeviceHandle> getDevices(String ioDevices) {
List<IODeviceHandle> devices = new ArrayList<IODeviceHandle>();
StringTokenizer tok = new StringTokenizer(ioDevices, ",");
@@ -152,7 +163,8 @@
connectionManager.start();
Registry registry = LocateRegistry.getRegistry(ncConfig.ccHost, ncConfig.ccPort);
IClusterController cc = (IClusterController) registry.lookup(IClusterController.class.getName());
- this.nodeParameters = cc.registerNode(this);
+ this.nodeParameters = cc.registerNode(new NodeRegistration(this, id, ncConfig, connectionManager
+ .getNetworkAddress()));
// Schedule heartbeat generator.
timer.schedule(new HeartbeatTask(cc), 0, nodeParameters.getHeartbeatPeriod());
@@ -168,6 +180,7 @@
@Override
public void stop() throws Exception {
LOGGER.log(Level.INFO, "Stopping NodeControllerService");
+ partitionManager.close();
connectionManager.stop();
LOGGER.log(Level.INFO, "Stopped NodeControllerService");
}
@@ -186,6 +199,14 @@
return connectionManager;
}
+ public PartitionManager getPartitionManager() {
+ return partitionManager;
+ }
+
+ public IClusterController getClusterController() {
+ return ccs;
+ }
+
private static NodeCapability computeNodeCapability() {
NodeCapability nc = new NodeCapability();
nc.setCPUCount(Runtime.getRuntime().availableProcessors());
@@ -210,14 +231,13 @@
}
@Override
- public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt,
- byte[] planBytes, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
- Map<OperatorDescriptorId, Integer> opNumPartitions) throws Exception {
+ public void startTasks(String appName, final JobId jobId, byte[] jagBytes,
+ List<TaskAttemptDescriptor> taskDescriptors,
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPoliciesMap, byte[] ctxVarBytes) throws Exception {
try {
- LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 1");
-
NCApplicationContext appCtx = applications.get(appName);
- final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
+ final JobActivityGraph plan = (JobActivityGraph) appCtx.deserialize(jagBytes);
+ Map<MultipartName, Object> ctxVarMap = (Map<MultipartName, Object>) appCtx.deserialize(ctxVarBytes);
IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@Override
@@ -231,156 +251,62 @@
}
};
- final Joblet joblet = getOrCreateLocalJoblet(jobId, attempt, appCtx);
+ final Joblet joblet = getOrCreateLocalJoblet(jobId, appCtx);
- Stagelet stagelet = new Stagelet(joblet, stageId, attempt, id);
- joblet.setStagelet(stageId, stagelet);
-
- final Map<PortInstanceId, Endpoint> portMap = new HashMap<PortInstanceId, Endpoint>();
- Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
-
- List<Endpoint> endpointList = new ArrayList<Endpoint>();
-
- for (ActivityNodeId hanId : tasks.keySet()) {
- IActivityNode han = plan.getActivityNodeMap().get(hanId);
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Initializing " + hanId + " -> " + han);
+ for (TaskAttemptDescriptor td : taskDescriptors) {
+ TaskAttemptId taId = td.getTaskAttemptId();
+ TaskId tid = taId.getTaskId();
+ IActivity han = plan.getActivityNodeMap().get(tid.getActivityId());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Initializing " + taId + " -> " + han);
}
- IOperatorDescriptor op = han.getOwner();
- List<IConnectorDescriptor> inputs = plan.getTaskInputs(hanId);
- for (int i : tasks.get(hanId)) {
- IOperatorNodePushable hon = han.createPushRuntime(stagelet, joblet.getEnvironment(op, i), rdp, i,
- opNumPartitions.get(op.getOperatorId()));
- OperatorRunnable or = new OperatorRunnable(stagelet, hon, inputs == null ? 0 : inputs.size(),
- executor);
- stagelet.setOperator(op.getOperatorId(), i, or);
- if (inputs != null) {
- for (int j = 0; j < inputs.size(); ++j) {
- IConnectorDescriptor conn = inputs.get(j);
- OperatorDescriptorId producerOpId = plan.getJobSpecification().getProducer(conn)
- .getOperatorId();
- OperatorDescriptorId consumerOpId = plan.getJobSpecification().getConsumer(conn)
- .getOperatorId();
- Endpoint endpoint = new Endpoint(connectionManager.getNetworkAddress(), i);
- endpointList.add(endpoint);
- DemuxDataReceiveListenerFactory drlf = new DemuxDataReceiveListenerFactory(stagelet, jobId,
- stageId);
- connectionManager.acceptConnection(endpoint.getEndpointId(), drlf);
- PortInstanceId piId = new PortInstanceId(op.getOperatorId(), Direction.INPUT, plan
- .getTaskInputMap().get(hanId).get(j), i);
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Created endpoint " + piId + " -> " + endpoint);
- }
- portMap.put(piId, endpoint);
- IFrameReader reader = createReader(stagelet, conn, drlf, i, plan, stagelet,
- opNumPartitions.get(producerOpId), opNumPartitions.get(consumerOpId));
- or.setFrameReader(j, reader);
+ final int partition = tid.getPartition();
+ Map<MultipartName, Object> inputGlobalVariables = createInputGlobalVariables(ctxVarMap, han);
+ Task task = new Task(joblet, taId, han.getClass().getName(), executor);
+ IOperatorEnvironment env = joblet.getEnvironment(tid.getActivityId().getOperatorDescriptorId(),
+ tid.getPartition());
+ IOperatorNodePushable operator = han.createPushRuntime(task, env, rdp, partition,
+ td.getPartitionCount());
+
+ List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
+
+ List<IConnectorDescriptor> inputs = plan.getActivityInputConnectorDescriptors(tid.getActivityId());
+ if (inputs != null) {
+ for (int i = 0; i < inputs.size(); ++i) {
+ IConnectorDescriptor conn = inputs.get(i);
+ IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("input: " + i + ": " + conn.getConnectorId());
}
+ RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+ IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn,
+ recordDesc, cPolicy);
+ collectors.add(collector);
}
- honMap.put(new OperatorInstanceId(op.getOperatorId(), i), or);
}
- }
+ List<IConnectorDescriptor> outputs = plan.getActivityOutputConnectorDescriptors(tid.getActivityId());
+ if (outputs != null) {
+ for (int i = 0; i < outputs.size(); ++i) {
+ final IConnectorDescriptor conn = outputs.get(i);
+ RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+ IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
- stagelet.setEndpointList(endpointList);
+ IPartitionWriterFactory pwFactory = createPartitionWriterFactory(cPolicy, jobId, conn,
+ partition, taId);
- return portMap;
- } catch (Exception e) {
- e.printStackTrace();
- throw e;
- }
- }
-
- private IFrameReader createReader(final IHyracksStageletContext stageletContext, final IConnectorDescriptor conn,
- IConnectionDemultiplexer demux, final int receiverIndex, JobPlan plan, final Stagelet stagelet,
- int nProducerCount, int nConsumerCount) throws HyracksDataException {
- final IFrameReader reader = conn.createReceiveSideReader(stageletContext, plan.getJobSpecification()
- .getConnectorRecordDescriptor(conn), demux, receiverIndex, nProducerCount, nConsumerCount);
-
- return plan.getJobFlags().contains(JobFlag.PROFILE_RUNTIME) ? new IFrameReader() {
- private ICounter openCounter = stageletContext.getCounterContext().getCounter(
- conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".open", true);
- private ICounter closeCounter = stageletContext.getCounterContext().getCounter(
- conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".close", true);
- private ICounter frameCounter = stageletContext.getCounterContext().getCounter(
- conn.getConnectorId().getId() + ".receiver." + receiverIndex + ".nextFrame", true);
-
- @Override
- public void open() throws HyracksDataException {
- reader.open();
- openCounter.update(1);
- }
-
- @Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- boolean status = reader.nextFrame(buffer);
- if (status) {
- frameCounter.update(1);
- }
- return status;
- }
-
- @Override
- public void close() throws HyracksDataException {
- reader.close();
- closeCounter.update(1);
- }
- } : reader;
- }
-
- @Override
- public void initializeJobletPhase2(String appName, UUID jobId, byte[] planBytes, UUID stageId,
- Map<ActivityNodeId, Set<Integer>> tasks, Map<OperatorDescriptorId, Integer> opNumPartitions,
- final Map<PortInstanceId, Endpoint> globalPortMap) throws Exception {
- try {
- LOGGER.log(Level.INFO, String.valueOf(jobId) + "[" + id + ":" + stageId + "]: Initializing Joblet Phase 2");
- ApplicationContext appCtx = applications.get(appName);
- final JobPlan plan = (JobPlan) appCtx.deserialize(planBytes);
-
- final Joblet ji = getLocalJoblet(jobId);
- final Stagelet stagelet = (Stagelet) ji.getStagelet(stageId);
- final Map<OperatorInstanceId, OperatorRunnable> honMap = stagelet.getOperatorMap();
-
- final JobSpecification spec = plan.getJobSpecification();
-
- for (ActivityNodeId hanId : tasks.keySet()) {
- IActivityNode han = plan.getActivityNodeMap().get(hanId);
- IOperatorDescriptor op = han.getOwner();
- List<IConnectorDescriptor> outputs = plan.getTaskOutputs(hanId);
- for (int i : tasks.get(hanId)) {
- OperatorRunnable or = honMap.get(new OperatorInstanceId(op.getOperatorId(), i));
- if (outputs != null) {
- for (int j = 0; j < outputs.size(); ++j) {
- final IConnectorDescriptor conn = outputs.get(j);
- OperatorDescriptorId producerOpId = plan.getJobSpecification().getProducer(conn)
- .getOperatorId();
- OperatorDescriptorId consumerOpId = plan.getJobSpecification().getConsumer(conn)
- .getOperatorId();
- final int senderIndex = i;
- IEndpointDataWriterFactory edwFactory = new IEndpointDataWriterFactory() {
- @Override
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException {
- PortInstanceId piId = new PortInstanceId(spec.getConsumer(conn).getOperatorId(),
- Direction.INPUT, spec.getConsumerInputIndex(conn), index);
- Endpoint ep = globalPortMap.get(piId);
- if (ep == null) {
- LOGGER.info("Got null Endpoint for " + piId);
- throw new NullPointerException();
- }
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Probed endpoint " + piId + " -> " + ep);
- }
- return createWriter(stagelet, connectionManager.connect(ep.getNetworkAddress(),
- ep.getEndpointId(), senderIndex), plan, conn, senderIndex, index, stagelet);
- }
- };
- or.setFrameWriter(j, conn.createSendSideWriter(stagelet, plan.getJobSpecification()
- .getConnectorRecordDescriptor(conn), edwFactory, i, opNumPartitions
- .get(producerOpId), opNumPartitions.get(consumerOpId)), spec
- .getConnectorRecordDescriptor(conn));
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("output: " + i + ": " + conn.getConnectorId());
}
+ IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
+ td.getPartitionCount(), td.getOutputPartitionCounts()[i]);
+ operator.setOutputFrameWriter(i, writer, recordDesc);
}
- stagelet.installRunnable(new OperatorInstanceId(op.getOperatorId(), i));
}
+
+ task.setTaskRuntime(collectors.toArray(new IPartitionCollector[collectors.size()]), operator);
+ joblet.addTask(task);
+
+ task.start();
}
} catch (Exception e) {
e.printStackTrace();
@@ -388,68 +314,55 @@
}
}
- private IFrameWriter createWriter(final IHyracksStageletContext stageletContext, final IFrameWriter writer,
- JobPlan plan, final IConnectorDescriptor conn, final int senderIndex, final int receiverIndex,
- final Stagelet stagelet) throws HyracksDataException {
- return plan.getJobFlags().contains(JobFlag.PROFILE_RUNTIME) ? new IFrameWriter() {
- private ICounter openCounter = stageletContext.getCounterContext().getCounter(
- conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex + ".open", true);
- private ICounter closeCounter = stageletContext.getCounterContext().getCounter(
- conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex + ".close", true);
- private ICounter frameCounter = stageletContext.getCounterContext()
- .getCounter(
- conn.getConnectorId().getId() + ".sender." + senderIndex + "." + receiverIndex
- + ".nextFrame", true);
-
- @Override
- public void open() throws HyracksDataException {
- writer.open();
- openCounter.update(1);
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- frameCounter.update(1);
- writer.nextFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- closeCounter.update(1);
- writer.close();
- }
-
- @Override
- public void flush() throws HyracksDataException {
- writer.flush();
- }
- } : writer;
+ private Map<MultipartName, Object> createInputGlobalVariables(Map<MultipartName, Object> ctxVarMap, IActivity han) {
+ Map<MultipartName, Object> gVars = new HashMap<MultipartName, Object>();
+ // for (MultipartName inVar : han.getInputVariables()) {
+ // gVars.put(inVar, ctxVarMap.get(inVar));
+ // }
+ return gVars;
}
- @Override
- public void commitJobletInitialization(UUID jobId, UUID stageId) throws Exception {
- final Joblet ji = getLocalJoblet(jobId);
- Stagelet si = (Stagelet) ji.getStagelet(stageId);
- for (Endpoint e : si.getEndpointList()) {
- connectionManager.unacceptConnection(e.getEndpointId());
+ private IPartitionCollector createPartitionCollector(TaskAttemptDescriptor td, final int partition, Task task,
+ int i, IConnectorDescriptor conn, RecordDescriptor recordDesc, IConnectorPolicy cPolicy)
+ throws HyracksDataException {
+ IPartitionCollector collector = conn.createPartitionCollector(task, recordDesc, partition,
+ td.getInputPartitionCounts()[i], td.getPartitionCount());
+ if (cPolicy.materializeOnReceiveSide()) {
+ return new ReceiveSideMaterializingCollector(ctx, partitionManager, collector, task.getTaskAttemptId(),
+ executor);
+ } else {
+ return collector;
}
- si.setEndpointList(null);
}
- private Joblet getLocalJoblet(UUID jobId) throws Exception {
+ private IPartitionWriterFactory createPartitionWriterFactory(IConnectorPolicy cPolicy, final JobId jobId,
+ final IConnectorDescriptor conn, final int senderIndex, final TaskAttemptId taId) {
+ if (cPolicy.materializeOnSendSide()) {
+ return new IPartitionWriterFactory() {
+ @Override
+ public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+ return new MaterializedPartitionWriter(ctx, partitionManager, new PartitionId(jobId,
+ conn.getConnectorId(), senderIndex, receiverIndex), taId, executor);
+ }
+ };
+ } else {
+ return new IPartitionWriterFactory() {
+ @Override
+ public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+ return new PipelinedPartition(partitionManager, new PartitionId(jobId, conn.getConnectorId(),
+ senderIndex, receiverIndex), taId);
+ }
+ };
+ }
+ }
+
+ private synchronized Joblet getOrCreateLocalJoblet(JobId jobId, INCApplicationContext appCtx) throws Exception {
Joblet ji = jobletMap.get(jobId);
- return ji;
- }
-
- private Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx) throws Exception {
- synchronized (jobletMap) {
- Joblet ji = jobletMap.get(jobId);
- if (ji == null || ji.getAttempt() != attempt) {
- ji = new Joblet(this, jobId, attempt, appCtx);
- jobletMap.put(jobId, ji);
- }
- return ji;
+ if (ji == null) {
+ ji = new Joblet(this, jobId, appCtx);
+ jobletMap.put(jobId, ji);
}
+ return ji;
}
public Executor getExecutor() {
@@ -457,43 +370,31 @@
}
@Override
- public void cleanUpJob(UUID jobId) throws Exception {
+ public void cleanUpJob(JobId jobId) throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Cleaning up after job: " + jobId);
}
Joblet joblet = jobletMap.remove(jobId);
if (joblet != null) {
+ partitionManager.unregisterPartitions(jobId);
joblet.close();
}
- connectionManager.dumpStats();
}
- @Override
- public void startStage(UUID jobId, UUID stageId) throws Exception {
- Joblet ji = jobletMap.get(jobId);
- if (ji != null) {
- Stagelet s = ji.getStagelet(stageId);
- if (s != null) {
- s.start();
- }
- }
- }
-
- public void notifyStageComplete(UUID jobId, UUID stageId, int attempt, StageletProfile stats) throws Exception {
+ public void notifyTaskComplete(JobId jobId, TaskAttemptId taskId, TaskProfile taskProfile) throws Exception {
try {
- ccs.notifyStageletComplete(jobId, stageId, attempt, id, stats);
+ ccs.notifyTaskComplete(jobId, taskId, id, taskProfile);
} catch (Exception e) {
e.printStackTrace();
throw e;
}
}
- public void notifyStageFailed(UUID jobId, UUID stageId, int attempt) throws Exception {
+ public void notifyTaskFailed(JobId jobId, TaskAttemptId taskId, Exception exception) {
try {
- ccs.notifyStageletFailure(jobId, stageId, attempt, id);
+ ccs.notifyTaskFailure(jobId, taskId, id, exception);
} catch (Exception e) {
e.printStackTrace();
- throw e;
}
}
@@ -538,7 +439,7 @@
synchronized (NodeControllerService.this) {
profiles = new ArrayList<JobProfile>();
for (Joblet ji : jobletMap.values()) {
- profiles.add(new JobProfile(ji.getJobId(), ji.getAttempt()));
+ profiles.add(new JobProfile(ji.getJobId()));
}
}
for (JobProfile jProfile : profiles) {
@@ -562,20 +463,20 @@
}
@Override
- public void abortJoblet(UUID jobId, int attempt) throws Exception {
+ public synchronized void abortTasks(JobId jobId, List<TaskAttemptId> tasks) throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Aborting Job: " + jobId + ":" + attempt);
+ LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks);
}
Joblet ji = jobletMap.get(jobId);
if (ji != null) {
- if (ji.getAttempt() == attempt) {
- jobletMap.remove(jobId);
+ Map<TaskAttemptId, Task> taskMap = ji.getTaskMap();
+ for (TaskAttemptId taId : tasks) {
+ Task task = taskMap.get(taId);
+ if (task != null) {
+ task.abort();
+ }
}
- for (Stagelet stagelet : ji.getStageletMap().values()) {
- stagelet.abort();
- stagelet.close();
- connectionManager.abortConnections(jobId, stagelet.getStageId());
- }
+ ji.close();
}
}
@@ -616,4 +517,14 @@
appCtx.deinitialize();
}
}
+
+ @Override
+ public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
+ Joblet ji = jobletMap.get(pid.getJobId());
+ if (ji != null) {
+ PartitionChannel channel = new PartitionChannel(pid, new NetworkInputChannel(ctx, connectionManager,
+ new InetSocketAddress(networkAddress.getIpAddress(), networkAddress.getPort()), pid, 1));
+ ji.reportPartitionAvailability(channel);
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
deleted file mode 100644
index 68b2cad..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * 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;
-
-import java.nio.ByteBuffer;
-import java.rmi.RemoteException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.Endpoint;
-import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.OperatorInstanceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.job.profiling.om.StageletProfile;
-import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
-import edu.uci.ics.hyracks.control.nc.runtime.OperatorRunnable;
-
-public class Stagelet implements IHyracksStageletContext, ICounterContext {
- private static final long serialVersionUID = 1L;
-
- private static final Logger LOGGER = Logger.getLogger(Stagelet.class.getName());
-
- private final Joblet joblet;
-
- private final UUID stageId;
-
- private final int attempt;
-
- private final Map<OperatorInstanceId, OperatorRunnable> honMap;
-
- private final Map<String, Counter> counterMap;
-
- private final IWorkspaceFileFactory fileFactory;
-
- private List<Endpoint> endpointList;
-
- private boolean started;
-
- private volatile boolean abort;
-
- private final Set<OperatorInstanceId> pendingOperators;
-
- private final DefaultDeallocatableRegistry deallocatableRegistry;
-
- public Stagelet(Joblet joblet, UUID stageId, int attempt, String nodeId) throws RemoteException {
- this.joblet = joblet;
- this.stageId = stageId;
- this.attempt = attempt;
- pendingOperators = new HashSet<OperatorInstanceId>();
- started = false;
- honMap = new HashMap<OperatorInstanceId, OperatorRunnable>();
- counterMap = new HashMap<String, Counter>();
- deallocatableRegistry = new DefaultDeallocatableRegistry();
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
- }
-
- public void setOperator(OperatorDescriptorId odId, int partition, OperatorRunnable hon) {
- honMap.put(new OperatorInstanceId(odId, partition), hon);
- }
-
- public Map<OperatorInstanceId, OperatorRunnable> getOperatorMap() {
- return honMap;
- }
-
- public void setEndpointList(List<Endpoint> endpointList) {
- this.endpointList = endpointList;
- }
-
- public List<Endpoint> getEndpointList() {
- return endpointList;
- }
-
- public synchronized void start() throws Exception {
- if (started) {
- throw new Exception("Joblet already started");
- }
- started = true;
- notifyAll();
- }
-
- public synchronized void abort() {
- this.abort = true;
- for (OperatorRunnable r : honMap.values()) {
- r.abort();
- }
- }
-
- public void installRunnable(final OperatorInstanceId opIId) {
- pendingOperators.add(opIId);
- final OperatorRunnable hon = honMap.get(opIId);
- joblet.getExecutor().execute(new Runnable() {
- @Override
- public void run() {
- try {
- waitUntilStarted();
- } catch (InterruptedException e) {
- e.printStackTrace();
- return;
- }
- if (abort) {
- return;
- }
- try {
- LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
- + opIId.getPartition() + "(" + hon + ")" + ": STARTED");
- hon.run();
- LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
- + opIId.getPartition() + "(" + hon + ")" + ": FINISHED");
- notifyOperatorCompletion(opIId);
- } catch (Exception e) {
- LOGGER.log(Level.INFO, joblet.getJobId() + ":" + stageId + ":" + opIId.getOperatorId() + ":"
- + opIId.getPartition() + "(" + hon + ")" + ": ABORTED");
- e.printStackTrace();
- notifyOperatorFailure(opIId);
- }
- }
- });
- }
-
- protected void notifyOperatorCompletion(OperatorInstanceId opIId) {
- boolean done = false;
- synchronized (pendingOperators) {
- pendingOperators.remove(opIId);
- done = pendingOperators.isEmpty();
- }
- if (done) {
- try {
- StageletProfile sProfile = new StageletProfile(stageId);
- dumpProfile(sProfile);
- close();
- joblet.notifyStageletComplete(stageId, attempt, sProfile);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
-
- protected synchronized void notifyOperatorFailure(OperatorInstanceId opIId) {
- abort();
- try {
- joblet.notifyStageletFailed(stageId, attempt);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
-
- private synchronized void waitUntilStarted() throws InterruptedException {
- while (!started && !abort) {
- wait();
- }
- }
-
- public synchronized void dumpProfile(StageletProfile sProfile) {
- Map<String, Long> dumpMap = sProfile.getCounters();
- for (Counter c : counterMap.values()) {
- dumpMap.put(c.getName(), c.get());
- }
- }
-
- @Override
- public IHyracksJobletContext getJobletContext() {
- return joblet;
- }
-
- @Override
- public UUID getStageId() {
- return stageId;
- }
-
- @Override
- public ICounterContext getCounterContext() {
- return this;
- }
-
- @Override
- public void registerDeallocatable(IDeallocatable deallocatable) {
- deallocatableRegistry.registerDeallocatable(deallocatable);
- }
-
- public void close() {
- deallocatableRegistry.close();
- }
-
- @Override
- public ByteBuffer allocateFrame() {
- return joblet.allocateFrame();
- }
-
- @Override
- public int getFrameSize() {
- return joblet.getFrameSize();
- }
-
- @Override
- public IIOManager getIOManager() {
- return joblet.getIOManager();
- }
-
- @Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
- return fileFactory.createWorkspaceFile(prefix);
- }
-
- @Override
- public synchronized ICounter getCounter(String name, boolean create) {
- Counter counter = counterMap.get(name);
- if (counter == null && create) {
- counter = new Counter(name);
- counterMap.put(name, counter);
- }
- return counter;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
new file mode 100644
index 0000000..e65f59e
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -0,0 +1,298 @@
+/*
+ * 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;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Semaphore;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+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.IIOManager;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.naming.MultipartName;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.api.workflow.variables.WorkflowVariableDescriptor;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
+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 Task implements IHyracksTaskContext, ICounterContext, Runnable {
+ private final Joblet joblet;
+
+ private final TaskAttemptId taskAttemptId;
+
+ private final String displayName;
+
+ private final Executor executor;
+
+ private final IWorkspaceFileFactory fileFactory;
+
+ private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+ private final Map<String, Counter> counterMap;
+
+ private final Map<MultipartName, Object> inputGlobalVariables;
+
+ private final Map<MultipartName, Object> outputVariables;
+
+ private final Map<MultipartName, WorkflowVariableDescriptor> outputVariableDescriptorMap;
+
+ private IPartitionCollector[] collectors;
+
+ private IOperatorNodePushable operator;
+
+ private volatile boolean aborted;
+
+ public Task(Joblet joblet, TaskAttemptId taskId, String displayName,
+ Executor executor) {
+ this.joblet = joblet;
+ this.taskAttemptId = taskId;
+ this.displayName = displayName;
+ this.executor = executor;
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
+ deallocatableRegistry = new DefaultDeallocatableRegistry();
+ counterMap = new HashMap<String, Counter>();
+// this.inputGlobalVariables = inputGlobalVariables;
+ inputGlobalVariables = Collections.emptyMap();
+ outputVariables = new HashMap<MultipartName, Object>();
+ outputVariableDescriptorMap = new HashMap<MultipartName, WorkflowVariableDescriptor>();
+// for (WorkflowVariableDescriptor wvd : outputVariableDescriptors) {
+// outputVariableDescriptorMap.put(wvd.getName(), wvd);
+// }
+ }
+
+ public void setTaskRuntime(IPartitionCollector[] collectors, IOperatorNodePushable operator) {
+ this.collectors = collectors;
+ this.operator = operator;
+ }
+
+ @Override
+ public ByteBuffer allocateFrame() {
+ return joblet.allocateFrame();
+ }
+
+ @Override
+ public int getFrameSize() {
+ return joblet.getFrameSize();
+ }
+
+ @Override
+ public IIOManager getIOManager() {
+ return joblet.getIOManager();
+ }
+
+ @Override
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createUnmanagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createManagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public void registerDeallocatable(IDeallocatable deallocatable) {
+ deallocatableRegistry.registerDeallocatable(deallocatable);
+ }
+
+ @Override
+ public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException {
+ MultipartName var = new MultipartName(producerActivity, partition, varName);
+ if (!inputGlobalVariables.containsKey(var)) {
+ throw new HyracksDataException("Unknown Variable: " + var);
+ }
+ return inputGlobalVariables.get(var);
+ }
+
+ @Override
+ public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException {
+ return joblet.lookupLocalVariable(new MultipartName(producerActivity, partition, varName));
+ }
+
+ @Override
+ public void setVariable(String name, Object value) {
+ outputVariables.put(new MultipartName(taskAttemptId.getTaskId().getActivityId(), taskAttemptId.getTaskId()
+ .getPartition(), name), value);
+ }
+
+ public void close() {
+ deallocatableRegistry.close();
+ }
+
+ @Override
+ public IHyracksJobletContext getJobletContext() {
+ return joblet;
+ }
+
+ @Override
+ public TaskAttemptId getTaskAttemptId() {
+ return taskAttemptId;
+ }
+
+ @Override
+ public ICounter getCounter(String name, boolean create) {
+ Counter counter = counterMap.get(name);
+ if (counter == null && create) {
+ counter = new Counter(name);
+ counterMap.put(name, counter);
+ }
+ return counter;
+ }
+
+ @Override
+ public ICounterContext getCounterContext() {
+ return this;
+ }
+
+ public synchronized void dumpProfile(TaskProfile tProfile) {
+ Map<String, Long> dumpMap = tProfile.getCounters();
+ for (Counter c : counterMap.values()) {
+ dumpMap.put(c.getName(), c.get());
+ }
+ }
+
+ public void start() throws HyracksException {
+ aborted = false;
+ joblet.getExecutor().execute(this);
+ }
+
+ public void abort() {
+ aborted = true;
+ for (IPartitionCollector c : collectors) {
+ c.abort();
+ }
+ }
+
+ @Override
+ public void run() {
+ Thread ct = Thread.currentThread();
+ String threadName = ct.getName();
+ try {
+ ct.setName(displayName + ": " + taskAttemptId);
+ operator.initialize();
+ try {
+ if (collectors.length > 0) {
+ final Semaphore sem = new Semaphore(collectors.length - 1);
+ for (int i = 1; i < collectors.length; ++i) {
+ final IPartitionCollector collector = collectors[i];
+ final IFrameWriter writer = operator.getInputFrameWriter(i);
+ sem.acquire();
+ executor.execute(new Runnable() {
+ public void run() {
+ try {
+ pushFrames(collector, writer);
+ } catch (HyracksDataException e) {
+ } finally {
+ sem.release();
+ }
+ }
+ });
+ }
+ try {
+ pushFrames(collectors[0], operator.getInputFrameWriter(0));
+ } finally {
+ sem.acquire(collectors.length - 1);
+ }
+ }
+ } finally {
+ operator.deinitialize();
+ }
+ Map<MultipartName, Object> outputGlobalVariables = new HashMap<MultipartName, Object>();
+ for (Map.Entry<MultipartName, Object> e : outputVariables.entrySet()) {
+ MultipartName varName = e.getKey();
+ WorkflowVariableDescriptor wvd = outputVariableDescriptorMap.get(varName);
+ if (wvd == null) {
+ throw new HyracksDataException("Unknown variable found: " + varName);
+ }
+ switch (wvd.getScope()) {
+ case LOCAL:
+ joblet.setLocalVariable(varName, e.getValue());
+ break;
+
+ case GLOBAL:
+ outputGlobalVariables.put(varName, e.getValue());
+ break;
+ }
+ }
+ joblet.notifyTaskComplete(this);
+ } catch (Exception e) {
+ e.printStackTrace();
+ joblet.notifyTaskFailed(this, e);
+ } finally {
+ ct.setName(threadName);
+ close();
+ }
+ }
+
+ private void pushFrames(IPartitionCollector collector, IFrameWriter writer) throws HyracksDataException {
+ if (aborted) {
+ return;
+ }
+ try {
+ collector.open();
+ try {
+ joblet.advertisePartitionRequest(taskAttemptId, collector.getRequiredPartitionIds(), collector,
+ PartitionState.STARTED);
+ IFrameReader reader = collector.getReader();
+ reader.open();
+ try {
+ writer.open();
+ try {
+ ByteBuffer buffer = allocateFrame();
+ while (reader.nextFrame(buffer)) {
+ if (aborted) {
+ return;
+ }
+ buffer.flip();
+ writer.nextFrame(buffer);
+ buffer.compact();
+ }
+ } finally {
+ writer.close();
+ }
+ } finally {
+ reader.close();
+ }
+ } finally {
+ collector.close();
+ }
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
deleted file mode 100644
index e49f435..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * 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.comm;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.SocketChannel;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-
-public class ConnectionEntry implements IConnectionEntry {
- private static final Logger LOGGER = Logger.getLogger(ConnectionEntry.class.getName());
-
- private SocketChannel socketChannel;
-
- private final ByteBuffer readBuffer;
-
- private final ByteBuffer writeBuffer;
-
- private IDataReceiveListener recvListener;
-
- private Object attachment;
-
- private final SelectionKey key;
-
- private UUID jobId;
-
- private UUID stageId;
-
- private boolean aborted;
-
- public ConnectionEntry(IHyracksRootContext ctx, SocketChannel socketChannel, SelectionKey key) {
- this.socketChannel = socketChannel;
- readBuffer = ctx.allocateFrame();
- readBuffer.clear();
- writeBuffer = ctx.allocateFrame();
- writeBuffer.clear();
- this.key = key;
- }
-
- public SocketChannel getSocketChannel() {
- return socketChannel;
- }
-
- public boolean dispatch(SelectionKey key) throws IOException {
- if (aborted) {
- recvListener.dataReceived(this);
- } else {
- if (key.isReadable()) {
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("Before read: " + readBuffer.position() + " " + readBuffer.limit());
- }
- int bytesRead = socketChannel.read(readBuffer);
- if (bytesRead < 0) {
- recvListener.eos(this);
- return true;
- }
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("After read: " + readBuffer.position() + " " + readBuffer.limit());
- }
- recvListener.dataReceived(this);
- } else if (key.isWritable()) {
- synchronized (this) {
- writeBuffer.flip();
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("Before write: " + writeBuffer.position() + " " + writeBuffer.limit());
- }
- int bytesWritten = socketChannel.write(writeBuffer);
- if (bytesWritten < 0) {
- return true;
- }
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("After write: " + writeBuffer.position() + " " + writeBuffer.limit());
- }
- if (writeBuffer.remaining() <= 0) {
- int ops = key.interestOps();
- key.interestOps(ops & ~SelectionKey.OP_WRITE);
- }
- writeBuffer.compact();
- notifyAll();
- }
- } else {
- LOGGER.warning("Spurious event triggered: " + key.readyOps());
- return true;
- }
- }
- return false;
- }
-
- @Override
- public ByteBuffer getReadBuffer() {
- return readBuffer;
- }
-
- @Override
- public synchronized void write(ByteBuffer buffer) {
- while (buffer.remaining() > 0) {
- while (writeBuffer.remaining() <= 0) {
- try {
- wait();
- } catch (InterruptedException e) {
- }
- }
- int oldLimit = buffer.limit();
- buffer.limit(Math.min(oldLimit, writeBuffer.remaining()));
- writeBuffer.put(buffer);
- buffer.limit(oldLimit);
- int ops = key.interestOps();
- key.interestOps(ops | SelectionKey.OP_WRITE);
- key.selector().wakeup();
- }
- }
-
- @Override
- public void setDataReceiveListener(IDataReceiveListener listener) {
- this.recvListener = listener;
- }
-
- @Override
- public void attach(Object attachment) {
- this.attachment = attachment;
- }
-
- @Override
- public Object getAttachment() {
- return attachment;
- }
-
- @Override
- public void close() {
- try {
- socketChannel.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- @Override
- public SelectionKey getSelectionKey() {
- return key;
- }
-
- @Override
- public UUID getJobId() {
- return jobId;
- }
-
- @Override
- public void setJobId(UUID jobId) {
- this.jobId = jobId;
- }
-
- @Override
- public UUID getStageId() {
- return stageId;
- }
-
- @Override
- public void setStageId(UUID stageId) {
- this.stageId = stageId;
- }
-
- @Override
- public void abort() {
- aborted = true;
- }
-
- @Override
- public boolean aborted() {
- return aborted;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
deleted file mode 100644
index 717144d..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
+++ /dev/null
@@ -1,399 +0,0 @@
-/*
- * 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.comm;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.ServerSocket;
-import java.nio.ByteBuffer;
-import java.nio.channels.AsynchronousCloseException;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.ServerSocketChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.FrameConstants;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.comm.NetworkAddress;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class ConnectionManager {
- private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
-
- private static final int INITIAL_MESSAGE_LEN = 20;
-
- private NetworkAddress networkAddress;
-
- private ServerSocketChannel serverSocketChannel;
-
- private final IHyracksRootContext ctx;
-
- private final Map<UUID, IDataReceiveListenerFactory> pendingConnectionReceivers;
-
- private final ConnectionListenerThread connectionListenerThread;
-
- private final DataListenerThread dataListenerThread;
-
- private final IDataReceiveListener initialDataReceiveListener;
-
- private final Set<IConnectionEntry> connections;
-
- private volatile boolean stopped;
-
- private ByteBuffer emptyFrame;
-
- public ConnectionManager(IHyracksRootContext ctx, InetAddress address) throws IOException {
- this.ctx = ctx;
- serverSocketChannel = ServerSocketChannel.open();
- ServerSocket serverSocket = serverSocketChannel.socket();
- serverSocket.bind(new InetSocketAddress(address, 0));
-
- networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Connection manager listening on " + serverSocket.getInetAddress() + ":"
- + serverSocket.getLocalPort());
- }
-
- pendingConnectionReceivers = new HashMap<UUID, IDataReceiveListenerFactory>();
- dataListenerThread = new DataListenerThread();
- connectionListenerThread = new ConnectionListenerThread();
- initialDataReceiveListener = new InitialDataReceiveListener();
- emptyFrame = ctx.allocateFrame();
- emptyFrame.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
- connections = new HashSet<IConnectionEntry>();
- }
-
- public synchronized void dumpStats() {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("Number of pendingConnectionReceivers: " + pendingConnectionReceivers.size());
- LOGGER.info("Number of selectable keys: " + dataListenerThread.selector.keys().size());
- }
- }
-
- public NetworkAddress getNetworkAddress() {
- return networkAddress;
- }
-
- public void start() {
- stopped = false;
- connectionListenerThread.start();
- dataListenerThread.start();
- }
-
- public void stop() {
- try {
- stopped = true;
- serverSocketChannel.close();
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
-
- public IFrameWriter connect(NetworkAddress address, UUID id, int senderId) throws HyracksDataException {
- try {
- SocketChannel channel = SocketChannel
- .open(new InetSocketAddress(address.getIpAddress(), address.getPort()));
- byte[] initialFrame = new byte[INITIAL_MESSAGE_LEN];
- ByteBuffer buffer = ByteBuffer.wrap(initialFrame);
- buffer.clear();
- buffer.putLong(id.getMostSignificantBits());
- buffer.putLong(id.getLeastSignificantBits());
- buffer.putInt(senderId);
- buffer.flip();
- int bytesWritten = 0;
- while (bytesWritten < INITIAL_MESSAGE_LEN) {
- int n = channel.write(buffer);
- if (n < 0) {
- throw new HyracksDataException("Stream closed prematurely");
- }
- bytesWritten += n;
- }
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Send Initial message: " + id + ":" + senderId);
- }
- buffer.clear();
- buffer.limit(FrameConstants.SIZE_LEN);
- int bytesRead = 0;
- while (bytesRead < FrameConstants.SIZE_LEN) {
- int n = channel.read(buffer);
- if (n < 0) {
- throw new HyracksDataException("Stream closed prematurely");
- }
- bytesRead += n;
- }
- buffer.flip();
- int frameLen = buffer.getInt();
- if (frameLen != FrameConstants.SIZE_LEN) {
- throw new IllegalStateException("Received illegal framelen = " + frameLen);
- }
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Got Ack message: " + id + ":" + senderId);
- }
- return new NetworkFrameWriter(channel);
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
-
- public synchronized void acceptConnection(UUID id, IDataReceiveListenerFactory receiver) {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.info("Connection manager accepting " + id);
- }
- pendingConnectionReceivers.put(id, receiver);
- }
-
- public synchronized void unacceptConnection(UUID id) {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.info("Connection manager unaccepting " + id);
- }
- pendingConnectionReceivers.remove(id);
- }
-
- public synchronized void abortConnections(UUID jobId, UUID stageId) {
- List<IConnectionEntry> abortConnections = new ArrayList<IConnectionEntry>();
- synchronized (this) {
- for (IConnectionEntry ce : connections) {
- if (ce.getJobId().equals(jobId) && ce.getStageId().equals(stageId)) {
- abortConnections.add(ce);
- }
- }
- }
- dataListenerThread.addPendingAbortConnections(abortConnections);
- }
-
- private final class NetworkFrameWriter implements IFrameWriter {
- private SocketChannel channel;
-
- NetworkFrameWriter(SocketChannel channel) {
- this.channel = channel;
- }
-
- @Override
- public void close() throws HyracksDataException {
- try {
- synchronized (emptyFrame) {
- emptyFrame.position(0);
- emptyFrame.limit(emptyFrame.capacity());
- channel.write(emptyFrame);
- }
- channel.close();
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- try {
- if (LOGGER.isLoggable(Level.FINER)) {
- int frameLen = buffer.getInt(buffer.position());
- LOGGER.finer("ConnectionManager.NetworkFrameWriter: frameLen = " + frameLen);
- }
- while (buffer.remaining() > 0) {
- channel.write(buffer);
- }
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
-
- @Override
- public void open() throws HyracksDataException {
- }
-
- @Override
- public void flush() {
- }
- }
-
- private final class ConnectionListenerThread extends Thread {
- public ConnectionListenerThread() {
- super("Hyracks Connection Listener Thread");
- }
-
- @Override
- public void run() {
- while (!stopped) {
- try {
- SocketChannel sc = serverSocketChannel.accept();
- dataListenerThread.addSocketChannel(sc);
- } catch (AsynchronousCloseException e) {
- // do nothing
- } catch (IOException e) {
- e.printStackTrace();
- }
- }
- }
- }
-
- private final class DataListenerThread extends Thread {
- private Selector selector;
-
- private List<SocketChannel> pendingNewSockets;
- private List<IConnectionEntry> pendingAbortConnections;
-
- public DataListenerThread() {
- super("Hyracks Data Listener Thread");
- try {
- selector = Selector.open();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- pendingNewSockets = new ArrayList<SocketChannel>();
- pendingAbortConnections = new ArrayList<IConnectionEntry>();
- }
-
- synchronized void addSocketChannel(SocketChannel sc) throws IOException {
- pendingNewSockets.add(sc);
- selector.wakeup();
- }
-
- synchronized void addPendingAbortConnections(List<IConnectionEntry> abortConnections) {
- pendingAbortConnections.addAll(abortConnections);
- selector.wakeup();
- }
-
- @Override
- public void run() {
- while (!stopped) {
- try {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Starting Select");
- }
- int n = selector.select();
- synchronized (this) {
- if (!pendingNewSockets.isEmpty()) {
- for (SocketChannel sc : pendingNewSockets) {
- sc.configureBlocking(false);
- SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
- ConnectionEntry entry = new ConnectionEntry(ctx, sc, scKey);
- entry.setDataReceiveListener(initialDataReceiveListener);
- scKey.attach(entry);
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Woke up selector");
- }
- }
- pendingNewSockets.clear();
- }
- if (!pendingAbortConnections.isEmpty()) {
- for (IConnectionEntry ce : pendingAbortConnections) {
- SelectionKey key = ce.getSelectionKey();
- ce.abort();
- ((ConnectionEntry) ce).dispatch(key);
- key.cancel();
- ce.close();
- synchronized (ConnectionManager.this) {
- connections.remove(ce);
- }
- }
- pendingAbortConnections.clear();
- }
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Selector: " + n);
- }
- if (n > 0) {
- for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
- SelectionKey key = i.next();
- i.remove();
- ConnectionEntry entry = (ConnectionEntry) key.attachment();
- boolean close = false;
- try {
- close = entry.dispatch(key);
- } catch (IOException e) {
- e.printStackTrace();
- close = true;
- }
- if (close) {
- key.cancel();
- entry.close();
- synchronized (ConnectionManager.this) {
- connections.remove(entry);
- }
- }
- }
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
-
- private class InitialDataReceiveListener implements IDataReceiveListener {
- @Override
- public void dataReceived(IConnectionEntry entry) throws IOException {
- ByteBuffer buffer = entry.getReadBuffer();
- buffer.flip();
- IDataReceiveListener newListener = null;
- if (buffer.remaining() >= INITIAL_MESSAGE_LEN) {
- long msb = buffer.getLong();
- long lsb = buffer.getLong();
- UUID endpointID = new UUID(msb, lsb);
- int senderId = buffer.getInt();
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Initial Frame received: " + endpointID + ":" + senderId);
- }
- IDataReceiveListenerFactory connectionReceiver;
- synchronized (ConnectionManager.this) {
- connectionReceiver = pendingConnectionReceivers.get(endpointID);
- if (connectionReceiver == null) {
- entry.close();
- return;
- }
- }
-
- newListener = connectionReceiver.getDataReceiveListener(endpointID, entry, senderId);
- entry.setDataReceiveListener(newListener);
- entry.setJobId(connectionReceiver.getJobId());
- entry.setStageId(connectionReceiver.getStageId());
- synchronized (ConnectionManager.this) {
- connections.add(entry);
- }
- byte[] ack = new byte[4];
- ByteBuffer ackBuffer = ByteBuffer.wrap(ack);
- ackBuffer.clear();
- ackBuffer.putInt(FrameConstants.SIZE_LEN);
- ackBuffer.flip();
- entry.write(ackBuffer);
- }
- buffer.compact();
- if (newListener != null && buffer.remaining() > 0) {
- newListener.dataReceived(entry);
- }
- }
-
- @Override
- public void eos(IConnectionEntry entry) {
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
deleted file mode 100644
index 32c8991..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * 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.comm;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.util.Arrays;
-import java.util.BitSet;
-import java.util.UUID;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListener;
-import edu.uci.ics.hyracks.api.comm.IDataReceiveListenerFactory;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class DemuxDataReceiveListenerFactory implements IDataReceiveListenerFactory, IConnectionDemultiplexer,
- IDataReceiveListener {
- private static final Logger LOGGER = Logger.getLogger(DemuxDataReceiveListenerFactory.class.getName());
-
- private final BitSet readyBits;
- private final int frameSize;
- private IConnectionEntry senders[];
- private int openSenderCount;
- private UUID jobId;
- private UUID stageId;
-
- public DemuxDataReceiveListenerFactory(IHyracksStageletContext ctx, UUID jobId, UUID stageId) {
- frameSize = ctx.getFrameSize();
- this.jobId = jobId;
- this.stageId = stageId;
- readyBits = new BitSet();
- senders = null;
- openSenderCount = 0;
- }
-
- @Override
- public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex) {
- entry.attach(senderIndex);
- addSender(senderIndex, entry);
- return this;
- }
-
- @Override
- public synchronized void dataReceived(IConnectionEntry entry) throws IOException {
- int senderIndex = (Integer) entry.getAttachment();
- ByteBuffer buffer = entry.getReadBuffer();
- buffer.flip();
- int dataLen = buffer.remaining();
- if (dataLen >= frameSize || entry.aborted()) {
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("NonDeterministicDataReceiveListener: frame received: sender = " + senderIndex);
- }
- SelectionKey key = entry.getSelectionKey();
- if (key.isValid()) {
- int ops = key.interestOps();
- key.interestOps(ops & ~SelectionKey.OP_READ);
- }
- readyBits.set(senderIndex);
- notifyAll();
- return;
- }
- buffer.compact();
- }
-
- @Override
- public void eos(IConnectionEntry entry) {
- }
-
- private synchronized void addSender(int senderIndex, IConnectionEntry entry) {
- readyBits.clear(senderIndex);
- if (senders == null) {
- senders = new IConnectionEntry[senderIndex + 1];
- } else if (senders.length <= senderIndex) {
- senders = Arrays.copyOf(senders, senderIndex + 1);
- }
- senders[senderIndex] = entry;
- ++openSenderCount;
- }
-
- @Override
- public synchronized IConnectionEntry findNextReadyEntry(int lastReadSender) {
- while (openSenderCount > 0 && readyBits.isEmpty()) {
- try {
- wait();
- } catch (InterruptedException e) {
- }
- }
- lastReadSender = readyBits.nextSetBit(lastReadSender);
- if (lastReadSender < 0) {
- lastReadSender = readyBits.nextSetBit(0);
- }
- return senders[lastReadSender];
- }
-
- @Override
- public synchronized void unreadyEntry(int index) {
- readyBits.clear(index);
- IConnectionEntry entry = senders[index];
- SelectionKey key = entry.getSelectionKey();
- if (key.isValid()) {
- int ops = key.interestOps();
- key.interestOps(ops | SelectionKey.OP_READ);
- key.selector().wakeup();
- }
- }
-
- @Override
- public synchronized int closeEntry(int index) throws HyracksDataException {
- IConnectionEntry entry = senders[index];
- SelectionKey key = entry.getSelectionKey();
- key.cancel();
- try {
- entry.close();
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- return --openSenderCount;
- }
-
- @Override
- public synchronized int getSenderCount() {
- return senders.length;
- }
-
- @Override
- public UUID getJobId() {
- return jobId;
- }
-
- @Override
- public UUID getStageId() {
- return stageId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
deleted file mode 100644
index bd6462d..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOFuture.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.io;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public final class IOFuture {
- private boolean complete;
-
- private Exception exception;
-
- public synchronized void reset() {
- complete = false;
- exception = null;
- }
-
- public synchronized void synchronize() throws HyracksDataException, InterruptedException {
- while (!complete) {
- try {
- wait();
- } catch (InterruptedException e) {
- throw e;
- }
- }
- if (exception != null) {
- throw new HyracksDataException(exception);
- }
- }
-
- public synchronized boolean isComplete() {
- return complete;
- }
-
- public synchronized void notifySuccess() {
- complete = true;
- notifyAll();
- }
-
- public synchronized void notifyFailure(Exception e) {
- complete = true;
- exception = e;
- notifyAll();
- }
-}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
similarity index 75%
rename from hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
rename to hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
index c6be844..16421ab 100644
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/ManagedWorkspaceFileFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/WorkspaceFileFactory.java
@@ -20,24 +20,29 @@
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
import edu.uci.ics.hyracks.api.resources.IDeallocatableRegistry;
-public final class ManagedWorkspaceFileFactory implements IWorkspaceFileFactory {
+public final class WorkspaceFileFactory implements IWorkspaceFileFactory {
private final IDeallocatableRegistry registry;
private final IOManager ioManager;
- public ManagedWorkspaceFileFactory(IDeallocatableRegistry registry, IOManager ioManager) {
+ public WorkspaceFileFactory(IDeallocatableRegistry registry, IOManager ioManager) {
this.registry = registry;
this.ioManager = ioManager;
}
@Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
final FileReference fRef = ioManager.createWorkspaceFile(prefix);
registry.registerDeallocatable(new IDeallocatable() {
@Override
public void deallocate() {
- fRef.getFile().delete();
+ fRef.delete();
}
});
return fRef;
}
+
+ @Override
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return ioManager.createWorkspaceFile(prefix);
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
new file mode 100644
index 0000000..b9eaf1f
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
@@ -0,0 +1,245 @@
+/*
+ * 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.ServerSocket;
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousCloseException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.Selector;
+import java.nio.channels.ServerSocketChannel;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+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.control.nc.partitions.IPartitionRequestListener;
+
+public class ConnectionManager {
+ private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
+
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IHyracksRootContext ctx;
+
+ private IPartitionRequestListener partitionRequestListener;
+
+ private final ServerSocketChannel serverChannel;
+
+ private volatile boolean stopped;
+
+ private final ConnectionListenerThread connectionListener;
+
+ private final DataListenerThread dataListener;
+
+ private final NetworkAddress networkAddress;
+
+ public ConnectionManager(IHyracksRootContext ctx, InetAddress inetAddress) throws IOException {
+ this.ctx = ctx;
+ serverChannel = ServerSocketChannel.open();
+ ServerSocket serverSocket = serverChannel.socket();
+ serverSocket.bind(new InetSocketAddress(inetAddress, 0));
+ stopped = false;
+ connectionListener = new ConnectionListenerThread();
+ dataListener = new DataListenerThread();
+ networkAddress = new NetworkAddress(serverSocket.getInetAddress(), serverSocket.getLocalPort());
+
+ }
+
+ public void setPartitionRequestListener(IPartitionRequestListener partitionRequestListener) {
+ this.partitionRequestListener = partitionRequestListener;
+ }
+
+ public void start() {
+ connectionListener.start();
+ dataListener.start();
+ }
+
+ public void stop() {
+ try {
+ stopped = true;
+ serverChannel.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ public void connect(INetworkChannel channel) throws IOException {
+ dataListener.addOutgoingConnection(channel);
+ }
+
+ private final class ConnectionListenerThread extends Thread {
+ public ConnectionListenerThread() {
+ super("Hyracks NC Connection Listener");
+ setDaemon(true);
+ }
+
+ @Override
+ public void run() {
+ while (!stopped) {
+ try {
+ SocketChannel sc = serverChannel.accept();
+ dataListener.addIncomingConnection(sc);
+ } catch (AsynchronousCloseException e) {
+ // do nothing
+ if (!stopped) {
+ e.printStackTrace();
+ }
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private final class DataListenerThread extends Thread {
+ private Selector selector;
+
+ private final List<SocketChannel> pendingIncomingConnections;
+ private final Set<SocketChannel> pendingNegotiations;
+ private final List<INetworkChannel> pendingOutgoingConnections;
+
+ public DataListenerThread() {
+ super("Hyracks Data Listener Thread");
+ setDaemon(true);
+ try {
+ selector = Selector.open();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ pendingIncomingConnections = new ArrayList<SocketChannel>();
+ pendingNegotiations = new HashSet<SocketChannel>();
+ pendingOutgoingConnections = new ArrayList<INetworkChannel>();
+ }
+
+ synchronized void addIncomingConnection(SocketChannel sc) throws IOException {
+ pendingIncomingConnections.add(sc);
+ selector.wakeup();
+ }
+
+ synchronized void addOutgoingConnection(INetworkChannel channel) throws IOException {
+ pendingOutgoingConnections.add(channel);
+ selector.wakeup();
+ }
+
+ @Override
+ public void run() {
+ while (!stopped) {
+ try {
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Starting Select");
+ }
+ int n = selector.select();
+ synchronized (this) {
+ if (!pendingIncomingConnections.isEmpty()) {
+ for (SocketChannel sc : pendingIncomingConnections) {
+ sc.configureBlocking(false);
+ SelectionKey scKey = sc.register(selector, SelectionKey.OP_READ);
+ ByteBuffer buffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+ scKey.attach(buffer);
+ pendingNegotiations.add(sc);
+ }
+ pendingIncomingConnections.clear();
+ }
+ if (!pendingOutgoingConnections.isEmpty()) {
+ for (INetworkChannel nc : pendingOutgoingConnections) {
+ SocketChannel sc = SocketChannel.open();
+ sc.configureBlocking(false);
+ SelectionKey scKey = sc.register(selector, 0);
+ scKey.attach(nc);
+ nc.setSelectionKey(scKey);
+ nc.notifyConnectionManagerRegistration();
+ }
+ pendingOutgoingConnections.clear();
+ }
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Selector: " + n);
+ }
+ if (n > 0) {
+ for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
+ SelectionKey key = i.next();
+ i.remove();
+ SocketChannel sc = (SocketChannel) key.channel();
+ if (pendingNegotiations.contains(sc)) {
+ if (key.isReadable()) {
+ ByteBuffer buffer = (ByteBuffer) key.attachment();
+ sc.read(buffer);
+ buffer.flip();
+ if (buffer.remaining() >= INITIAL_MESSAGE_SIZE) {
+ PartitionId pid = readInitialMessage(buffer);
+ pendingNegotiations.remove(sc);
+ key.interestOps(0);
+ NetworkOutputChannel channel = new NetworkOutputChannel(ctx, 5);
+ channel.setSelectionKey(key);
+ key.attach(channel);
+ try {
+ partitionRequestListener.registerPartitionRequest(pid, channel);
+ } catch (HyracksException e) {
+ key.cancel();
+ sc.close();
+ }
+ } else {
+ buffer.compact();
+ }
+ }
+ } else {
+ INetworkChannel channel = (INetworkChannel) key.attachment();
+ boolean close = false;
+ try {
+ close = channel.dispatchNetworkEvent();
+ } catch (IOException e) {
+ e.printStackTrace();
+ close = true;
+ }
+ if (close) {
+ key.cancel();
+ sc.close();
+ }
+ }
+ }
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+
+ private PartitionId readInitialMessage(ByteBuffer buffer) {
+ JobId jobId = new JobId(buffer.getLong());
+ ConnectorDescriptorId cdid = new ConnectorDescriptorId(buffer.getInt());
+ int senderIndex = buffer.getInt();
+ int receiverIndex = buffer.getInt();
+ return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
+ }
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
new file mode 100644
index 0000000..61cd91f
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
@@ -0,0 +1,19 @@
+package edu.uci.ics.hyracks.control.nc.net;
+
+import java.io.IOException;
+import java.net.SocketAddress;
+import java.nio.channels.SelectionKey;
+
+public interface INetworkChannel {
+ public boolean dispatchNetworkEvent() throws IOException;
+
+ public void setSelectionKey(SelectionKey key);
+
+ public SelectionKey getSelectionKey();
+
+ public SocketAddress getRemoteAddress();
+
+ public void abort();
+
+ public void notifyConnectionManagerRegistration() throws IOException;
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
new file mode 100644
index 0000000..58f7088
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -0,0 +1,235 @@
+/*
+ * 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.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class NetworkInputChannel implements IInputChannel, INetworkChannel {
+ private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+
+ private final ConnectionManager connectionManager;
+
+ private final SocketAddress remoteAddress;
+
+ private final PartitionId partitionId;
+
+ private final Queue<ByteBuffer> emptyQueue;
+
+ private final Queue<ByteBuffer> fullQueue;
+
+ private SocketChannel socketChannel;
+
+ private SelectionKey key;
+
+ private ByteBuffer currentBuffer;
+
+ private boolean eos;
+
+ private boolean aborted;
+
+ private IInputChannelMonitor monitor;
+
+ private Object attachment;
+
+ private ByteBuffer writeBuffer;
+
+ public NetworkInputChannel(IHyracksRootContext ctx, ConnectionManager connectionManager,
+ SocketAddress remoteAddress, PartitionId partitionId, int nBuffers) {
+ this.connectionManager = connectionManager;
+ this.remoteAddress = remoteAddress;
+ this.partitionId = partitionId;
+ this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ for (int i = 0; i < nBuffers; ++i) {
+ emptyQueue.add(ctx.allocateFrame());
+ }
+ fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ aborted = false;
+ eos = false;
+ }
+
+ @Override
+ public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException {
+ this.monitor = monitor;
+ }
+
+ @Override
+ public void setAttachment(Object attachment) {
+ this.attachment = attachment;
+ }
+
+ @Override
+ public Object getAttachment() {
+ return attachment;
+ }
+
+ @Override
+ public synchronized ByteBuffer getNextBuffer() {
+ return fullQueue.poll();
+ }
+
+ @Override
+ public synchronized void recycleBuffer(ByteBuffer buffer) {
+ buffer.clear();
+ emptyQueue.add(buffer);
+ if (!eos && !aborted) {
+ int ops = key.interestOps();
+ if ((ops & SelectionKey.OP_READ) == 0) {
+ key.interestOps(ops | SelectionKey.OP_READ);
+ key.selector().wakeup();
+ if (currentBuffer == null) {
+ currentBuffer = emptyQueue.poll();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ currentBuffer = emptyQueue.poll();
+ try {
+ connectionManager.connect(this);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+
+ }
+
+ @Override
+ public synchronized boolean dispatchNetworkEvent() throws IOException {
+ if (aborted) {
+ eos = true;
+ monitor.notifyEndOfStream(this);
+ return true;
+ }
+ if (key.isConnectable()) {
+ if (socketChannel.finishConnect()) {
+ key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT);
+ prepareForWrite();
+ }
+ } else if (key.isWritable()) {
+ socketChannel.write(writeBuffer);
+ if (writeBuffer.remaining() == 0) {
+ key.interestOps(SelectionKey.OP_READ);
+ }
+ } else if (key.isReadable()) {
+ if (LOGGER.isLoggable(Level.FINER)) {
+ LOGGER.finer("Before read: " + currentBuffer.position() + " " + currentBuffer.limit());
+ }
+ int bytesRead = socketChannel.read(currentBuffer);
+ if (bytesRead < 0) {
+ eos = true;
+ monitor.notifyEndOfStream(this);
+ return true;
+ }
+ if (LOGGER.isLoggable(Level.FINER)) {
+ LOGGER.finer("After read: " + currentBuffer.position() + " " + currentBuffer.limit());
+ }
+ currentBuffer.flip();
+ int dataLen = currentBuffer.remaining();
+ if (dataLen >= currentBuffer.capacity() || aborted()) {
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("NetworkInputChannel: frame received: sender = " + partitionId.getSenderIndex());
+ }
+ if (currentBuffer.getInt(FrameHelper.getTupleCountOffset(currentBuffer.capacity())) == 0) {
+ eos = true;
+ monitor.notifyEndOfStream(this);
+ return true;
+ }
+ fullQueue.add(currentBuffer);
+ currentBuffer = emptyQueue.poll();
+ if (currentBuffer == null && key.isValid()) {
+ int ops = key.interestOps();
+ key.interestOps(ops & ~SelectionKey.OP_READ);
+ }
+ monitor.notifyDataAvailability(this, 1);
+ return false;
+ }
+ currentBuffer.compact();
+ }
+ return false;
+ }
+
+ private void prepareForConnect() {
+ key.interestOps(SelectionKey.OP_CONNECT);
+ }
+
+ private void prepareForWrite() {
+ writeBuffer = ByteBuffer.allocate(ConnectionManager.INITIAL_MESSAGE_SIZE);
+ writeBuffer.putLong(partitionId.getJobId().getId());
+ writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
+ writeBuffer.putInt(partitionId.getSenderIndex());
+ writeBuffer.putInt(partitionId.getReceiverIndex());
+ writeBuffer.flip();
+
+ key.interestOps(SelectionKey.OP_WRITE);
+ }
+
+ @Override
+ public void setSelectionKey(SelectionKey key) {
+ this.key = key;
+ socketChannel = (SocketChannel) key.channel();
+ }
+
+ @Override
+ public SocketAddress getRemoteAddress() {
+ return remoteAddress;
+ }
+
+ @Override
+ public SelectionKey getSelectionKey() {
+ return key;
+ }
+
+ public PartitionId getPartitionId() {
+ return partitionId;
+ }
+
+ public void abort() {
+ aborted = true;
+ }
+
+ public boolean aborted() {
+ return aborted;
+ }
+
+ @Override
+ public void notifyConnectionManagerRegistration() throws IOException {
+ if (socketChannel.connect(remoteAddress)) {
+ prepareForWrite();
+ } else {
+ prepareForConnect();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
new file mode 100644
index 0000000..74542b3
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
@@ -0,0 +1,171 @@
+/*
+ * 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.SocketAddress;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NetworkOutputChannel implements INetworkChannel, IFrameWriter {
+ private final IHyracksRootContext ctx;
+
+ private final Queue<ByteBuffer> emptyQueue;
+
+ private final Queue<ByteBuffer> fullQueue;
+
+ private SelectionKey key;
+
+ private boolean aborted;
+
+ private boolean eos;
+
+ private boolean eosSent;
+
+ private ByteBuffer currentBuffer;
+
+ public NetworkOutputChannel(IHyracksRootContext ctx, int nBuffers) {
+ this.ctx = ctx;
+ emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ for (int i = 0; i < nBuffers; ++i) {
+ emptyQueue.add(ctx.allocateFrame());
+ }
+ fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ }
+
+ @Override
+ public synchronized boolean dispatchNetworkEvent() throws IOException {
+ if (aborted) {
+ eos = true;
+ return true;
+ } else if (key.isWritable()) {
+ while (true) {
+ if (currentBuffer == null) {
+ if (eosSent) {
+ return true;
+ }
+ currentBuffer = fullQueue.poll();
+ if (currentBuffer == null) {
+ if (eos) {
+ currentBuffer = emptyQueue.poll();
+ currentBuffer.clear();
+ currentBuffer.putInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()), 0);
+ eosSent = true;
+ } else {
+ key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
+ return false;
+ }
+ }
+ }
+ int bytesWritten = ((SocketChannel) key.channel()).write(currentBuffer);
+ if (bytesWritten < 0) {
+ eos = true;
+ return true;
+ }
+ if (currentBuffer.remaining() == 0) {
+ emptyQueue.add(currentBuffer);
+ notifyAll();
+ currentBuffer = null;
+ if (eosSent) {
+ return true;
+ }
+ } else {
+ return false;
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public void setSelectionKey(SelectionKey key) {
+ this.key = key;
+ }
+
+ @Override
+ public SelectionKey getSelectionKey() {
+ return key;
+ }
+
+ @Override
+ public SocketAddress getRemoteAddress() {
+ return ((SocketChannel) key.channel()).socket().getRemoteSocketAddress();
+ }
+
+ @Override
+ public synchronized void abort() {
+ aborted = true;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ currentBuffer = null;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ByteBuffer destBuffer = null;
+ synchronized (this) {
+ if (aborted) {
+ throw new HyracksDataException("Connection has been aborted");
+ }
+ while (true) {
+ destBuffer = emptyQueue.poll();
+ if (destBuffer != null) {
+ break;
+ }
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ buffer.position(0);
+ buffer.limit(destBuffer.capacity());
+ destBuffer.clear();
+ destBuffer.put(buffer);
+ destBuffer.flip();
+ synchronized (this) {
+ fullQueue.add(destBuffer);
+ }
+ key.interestOps(SelectionKey.OP_WRITE);
+ key.selector().wakeup();
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+
+ }
+
+ @Override
+ public synchronized void close() throws HyracksDataException {
+ eos = true;
+ key.interestOps(SelectionKey.OP_WRITE);
+ key.selector().wakeup();
+ }
+
+ @Override
+ public void notifyConnectionManagerRegistration() throws IOException {
+ }
+}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
similarity index 67%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
copy to hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
index 45c589a..20ed49c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IEndpointDataWriterFactory.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
@@ -12,11 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.dataflow;
+package edu.uci.ics.hyracks.control.nc.partitions;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface IPartitionRequestListener {
+ public void registerPartitionRequest(PartitionId partitionId, IFrameWriter writer) throws HyracksException;
}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
new file mode 100644
index 0000000..97f82d9
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
@@ -0,0 +1,93 @@
+/*
+ * 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.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class MaterializedPartition implements IPartition {
+ private final IHyracksRootContext ctx;
+
+ private final FileReference partitionFile;
+
+ private final Executor executor;
+
+ private final IOManager ioManager;
+
+ public MaterializedPartition(IHyracksRootContext ctx, FileReference partitionFile, Executor executor,
+ IOManager ioManager) {
+ this.ctx = ctx;
+ this.partitionFile = partitionFile;
+ this.executor = executor;
+ this.ioManager = ioManager;
+ }
+
+ @Override
+ public void deallocate() {
+ partitionFile.delete();
+ }
+
+ @Override
+ public void writeTo(final IFrameWriter writer) {
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ FileHandle fh = ioManager.open(partitionFile, IIOManager.FileReadWriteMode.READ_ONLY,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ try {
+ writer.open();
+ try {
+ long offset = 0;
+ ByteBuffer buffer = ctx.allocateFrame();
+ while (true) {
+ buffer.clear();
+ long size = ioManager.syncRead(fh, offset, buffer);
+ if (size < 0) {
+ break;
+ } else if (size < buffer.capacity()) {
+ throw new HyracksDataException("Premature end of file");
+ }
+ offset += size;
+ buffer.flip();
+ writer.nextFrame(buffer);
+ }
+ } finally {
+ writer.close();
+ }
+ } finally {
+ ioManager.close(fh);
+ }
+ } catch (HyracksDataException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ });
+ }
+
+ @Override
+ public boolean isReusable() {
+ return true;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
new file mode 100644
index 0000000..971f2b6
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -0,0 +1,132 @@
+/*
+ * 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.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Queue;
+
+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.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class MaterializedPartitionInputChannel implements IInputChannel {
+ private final Queue<ByteBuffer> emptyQueue;
+
+ private final Queue<ByteBuffer> fullQueue;
+
+ private final PartitionId pid;
+
+ private final PartitionManager manager;
+
+ private final FrameWriter writer;
+
+ private IInputChannelMonitor monitor;
+
+ private Object attachment;
+
+ public MaterializedPartitionInputChannel(IHyracksRootContext ctx, int nBuffers, PartitionId pid,
+ PartitionManager manager) {
+ this.emptyQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ for (int i = 0; i < nBuffers; ++i) {
+ emptyQueue.add(ctx.allocateFrame());
+ }
+ fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
+ this.pid = pid;
+ this.manager = manager;
+ writer = new FrameWriter();
+ }
+
+ @Override
+ public void registerMonitor(IInputChannelMonitor monitor) throws HyracksException {
+ this.monitor = monitor;
+ }
+
+ @Override
+ public void setAttachment(Object attachment) {
+ this.attachment = attachment;
+ }
+
+ @Override
+ public Object getAttachment() {
+ return attachment;
+ }
+
+ @Override
+ public ByteBuffer getNextBuffer() {
+ return fullQueue.poll();
+ }
+
+ @Override
+ public void recycleBuffer(ByteBuffer buffer) {
+ buffer.clear();
+ synchronized (this) {
+ emptyQueue.add(buffer);
+ notifyAll();
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ MaterializedPartition partition = (MaterializedPartition) manager.getPartition(pid);
+ partition.writeTo(writer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+
+ }
+
+ private class FrameWriter implements IFrameWriter {
+ @Override
+ public void open() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ synchronized (MaterializedPartitionInputChannel.this) {
+ while (emptyQueue.isEmpty()) {
+ try {
+ MaterializedPartitionInputChannel.this.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ ByteBuffer destFrame = emptyQueue.poll();
+ buffer.position(0);
+ buffer.limit(buffer.capacity());
+ destFrame.clear();
+ destFrame.put(buffer);
+ fullQueue.add(destFrame);
+ monitor.notifyDataAvailability(MaterializedPartitionInputChannel.this, 1);
+ }
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ monitor.notifyEndOfStream(MaterializedPartitionInputChannel.this);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
new file mode 100644
index 0000000..db2e405
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionWriter.java
@@ -0,0 +1,81 @@
+/*
+ * 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.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileHandle;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+
+public class MaterializedPartitionWriter implements IFrameWriter {
+ protected final IHyracksRootContext ctx;
+
+ protected final PartitionManager manager;
+
+ protected final PartitionId pid;
+
+ protected final TaskAttemptId taId;
+
+ protected final Executor executor;
+
+ private FileReference fRef;
+
+ private FileHandle handle;
+
+ private long size;
+
+ public MaterializedPartitionWriter(IHyracksRootContext ctx, PartitionManager manager, PartitionId pid,
+ TaskAttemptId taId, Executor executor) {
+ this.ctx = ctx;
+ this.manager = manager;
+ this.pid = pid;
+ this.taId = taId;
+ this.executor = executor;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(pid.toString());
+ handle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ size = 0;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ size += ctx.getIOManager().syncWrite(handle, size, buffer);
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ ctx.getIOManager().close(handle);
+ manager.registerPartition(pid, taId,
+ new MaterializedPartition(ctx, fRef, executor, (IOManager) ctx.getIOManager()),
+ PartitionState.COMMITTED);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
new file mode 100644
index 0000000..baa50a1
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -0,0 +1,117 @@
+/*
+ * 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.partitions;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+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.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+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.resources.DefaultDeallocatableRegistry;
+
+public class PartitionManager implements IPartitionRequestListener {
+ private final NodeControllerService ncs;
+
+ private final Map<PartitionId, List<IPartition>> partitionMap;
+
+ private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+ private final IWorkspaceFileFactory fileFactory;
+
+ public PartitionManager(NodeControllerService ncs) {
+ this.ncs = ncs;
+ partitionMap = new HashMap<PartitionId, List<IPartition>>();
+ deallocatableRegistry = new DefaultDeallocatableRegistry();
+ fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+ }
+
+ public void registerPartition(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
+ throws HyracksDataException {
+ synchronized (this) {
+ List<IPartition> pList = partitionMap.get(pid);
+ if (pList == null) {
+ pList = new ArrayList<IPartition>();
+ partitionMap.put(pid, pList);
+ }
+ pList.add(partition);
+ }
+ updatePartitionState(pid, taId, partition, state);
+ }
+
+ public void updatePartitionState(PartitionId pid, TaskAttemptId taId, IPartition partition, PartitionState state)
+ throws HyracksDataException {
+ PartitionDescriptor desc = new PartitionDescriptor(pid, ncs.getId(), taId, partition.isReusable());
+ desc.setState(state);
+ try {
+ ncs.getClusterController().registerPartitionProvider(desc);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public synchronized IPartition getPartition(PartitionId pid) {
+ return partitionMap.get(pid).get(0);
+ }
+
+ public synchronized void unregisterPartitions(JobId jobId) {
+ for (Iterator<Map.Entry<PartitionId, List<IPartition>>> i = partitionMap.entrySet().iterator(); i.hasNext();) {
+ Map.Entry<PartitionId, List<IPartition>> e = i.next();
+ PartitionId pid = e.getKey();
+ if (jobId.equals(pid.getJobId())) {
+ for (IPartition p : e.getValue()) {
+ p.deallocate();
+ }
+ i.remove();
+ }
+ }
+ }
+
+ @Override
+ public synchronized void registerPartitionRequest(PartitionId partitionId, IFrameWriter writer)
+ throws HyracksException {
+ List<IPartition> pList = partitionMap.get(partitionId);
+ if (pList != null && !pList.isEmpty()) {
+ IPartition partition = pList.get(0);
+ partition.writeTo(writer);
+ if (!partition.isReusable()) {
+ partitionMap.remove(partitionId);
+ }
+ } else {
+ throw new HyracksException("Request for unknown partition " + partitionId);
+ }
+ }
+
+ public IWorkspaceFileFactory getFileFactory() {
+ return fileFactory;
+ }
+
+ public void close() {
+ deallocatableRegistry.close();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
new file mode 100644
index 0000000..1a9fb53
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.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.control.nc.partitions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.common.job.PartitionState;
+
+public class PipelinedPartition implements IFrameWriter, IPartition {
+ private final PartitionManager manager;
+
+ private final PartitionId pid;
+
+ private final TaskAttemptId taId;
+
+ private IFrameWriter delegate;
+
+ public PipelinedPartition(PartitionManager manager, PartitionId pid, TaskAttemptId taId) {
+ this.manager = manager;
+ this.pid = pid;
+ this.taId = taId;
+ }
+
+ @Override
+ public boolean isReusable() {
+ return false;
+ }
+
+ @Override
+ public void deallocate() {
+ // do nothing
+ }
+
+ @Override
+ public synchronized void writeTo(IFrameWriter writer) {
+ delegate = writer;
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void open() throws HyracksDataException {
+ manager.registerPartition(pid, taId, this, PartitionState.STARTED);
+ while (delegate == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ delegate.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ delegate.nextFrame(buffer);
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ delegate.flush();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ manager.updatePartitionState(pid, taId, this, PartitionState.COMMITTED);
+ delegate.close();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
new file mode 100644
index 0000000..c108bef
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/ReceiveSideMaterializingCollector.java
@@ -0,0 +1,160 @@
+/*
+ * 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.partitions;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class ReceiveSideMaterializingCollector implements IPartitionCollector {
+ private final IHyracksRootContext ctx;
+
+ private PartitionManager manager;
+
+ private final IPartitionCollector delegate;
+
+ private final TaskAttemptId taId;
+
+ private final Executor executor;
+
+ public ReceiveSideMaterializingCollector(IHyracksRootContext ctx, PartitionManager manager,
+ IPartitionCollector collector, TaskAttemptId taId, Executor executor) {
+ this.ctx = ctx;
+ this.manager = manager;
+ this.delegate = collector;
+ this.taId = taId;
+ this.executor = executor;
+ }
+
+ @Override
+ public JobId getJobId() {
+ return delegate.getJobId();
+ }
+
+ @Override
+ public ConnectorDescriptorId getConnectorId() {
+ return delegate.getConnectorId();
+ }
+
+ @Override
+ public int getReceiverIndex() {
+ return delegate.getReceiverIndex();
+ }
+
+ @Override
+ public void open() throws HyracksException {
+ delegate.open();
+ }
+
+ @Override
+ public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+ for (final PartitionChannel pc : partitions) {
+ PartitionWriter writer = new PartitionWriter(pc);
+ executor.execute(writer);
+ }
+ }
+
+ private class PartitionWriter implements Runnable, IInputChannelMonitor {
+ private PartitionChannel pc;
+
+ private int nAvailableFrames;
+
+ private boolean eos;
+
+ public PartitionWriter(PartitionChannel pc) {
+ this.pc = pc;
+ nAvailableFrames = 0;
+ eos = false;
+ }
+
+ @Override
+ public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+ nAvailableFrames += nFrames;
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyEndOfStream(IInputChannel channel) {
+ eos = true;
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void run() {
+ PartitionId pid = pc.getPartitionId();
+ MaterializedPartitionWriter mpw = new MaterializedPartitionWriter(ctx, manager, pid, taId, executor);
+ IInputChannel channel = pc.getInputChannel();
+ try {
+ channel.registerMonitor(this);
+ channel.open();
+ mpw.open();
+ while (true) {
+ if (nAvailableFrames > 0) {
+ ByteBuffer buffer = channel.getNextBuffer();
+ --nAvailableFrames;
+ mpw.nextFrame(buffer);
+ channel.recycleBuffer(buffer);
+ } else if (eos) {
+ break;
+ } else {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ mpw.close();
+ channel.close();
+ delegate.addPartitions(Collections.singleton(new PartitionChannel(pid,
+ new MaterializedPartitionInputChannel(ctx, 5, pid, manager))));
+ } catch (HyracksException e) {
+ }
+ }
+ }
+
+ @Override
+ public IFrameReader getReader() throws HyracksException {
+ return delegate.getReader();
+ }
+
+ @Override
+ public void close() throws HyracksException {
+ delegate.close();
+ }
+
+ @Override
+ public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+ return delegate.getRequiredPartitionIds();
+ }
+
+ @Override
+ public void abort() {
+ delegate.abort();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
new file mode 100644
index 0000000..96fcf75
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.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.nc.profiling;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+
+public class ConnectorReceiverProfilingFrameReader implements IFrameReader {
+ private final IFrameReader reader;
+ private final ICounter openCounter;
+ private final ICounter closeCounter;
+ private final ICounter frameCounter;
+
+ public ConnectorReceiverProfilingFrameReader(IHyracksTaskContext ctx, IFrameReader reader,
+ ConnectorDescriptorId cdId, int receiverIndex) {
+ this.reader = reader;
+ this.openCounter = ctx.getCounterContext().getCounter(cdId + ".receiver." + receiverIndex + ".open", true);
+ this.closeCounter = ctx.getCounterContext().getCounter(cdId + ".receiver." + receiverIndex + ".close", true);
+ this.frameCounter = ctx.getCounterContext()
+ .getCounter(cdId + ".receiver." + receiverIndex + ".nextFrame", true);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ reader.open();
+ openCounter.update(1);
+ }
+
+ @Override
+ public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ boolean status = reader.nextFrame(buffer);
+ if (status) {
+ frameCounter.update(1);
+ }
+ return status;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ reader.close();
+ closeCounter.update(1);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java
new file mode 100644
index 0000000..c64e187
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorSenderProfilingFrameWriter.java
@@ -0,0 +1,64 @@
+/*
+ * 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.profiling;
+
+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.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounter;
+
+public class ConnectorSenderProfilingFrameWriter implements IFrameWriter {
+ private final IFrameWriter writer;
+ private final ICounter openCounter;
+ private final ICounter closeCounter;
+ private final ICounter frameCounter;
+
+ public ConnectorSenderProfilingFrameWriter(IHyracksTaskContext ctx, IFrameWriter writer,
+ ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
+ this.writer = writer;
+ this.openCounter = ctx.getCounterContext().getCounter(
+ cdId + ".sender." + senderIndex + "." + receiverIndex + ".open", true);
+ this.closeCounter = ctx.getCounterContext().getCounter(
+ cdId + ".sender." + senderIndex + "." + receiverIndex + ".close", true);
+ this.frameCounter = ctx.getCounterContext().getCounter(
+ cdId + ".sender." + senderIndex + "." + receiverIndex + ".nextFrame", true);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ openCounter.update(1);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ frameCounter.update(1);
+ writer.nextFrame(buffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ closeCounter.update(1);
+ writer.close();
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+ writer.flush();
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
deleted file mode 100644
index 8edd992..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.runtime;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.Executor;
-import java.util.concurrent.Semaphore;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class OperatorRunnable implements Runnable {
- private final IHyracksStageletContext ctx;
- private final IOperatorNodePushable opNode;
- private final int nInputs;
- private final Executor executor;
- private IFrameReader[] readers;
- private volatile boolean abort;
-
- public OperatorRunnable(IHyracksStageletContext ctx, IOperatorNodePushable opNode, int nInputs, Executor executor) {
- this.ctx = ctx;
- this.opNode = opNode;
- this.nInputs = nInputs;
- this.executor = executor;
- readers = new IFrameReader[nInputs];
- }
-
- public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
- opNode.setOutputFrameWriter(index, writer, recordDesc);
- }
-
- public void setFrameReader(int inputIdx, IFrameReader reader) {
- this.readers[inputIdx] = reader;
- }
-
- public void abort() {
- abort = true;
- }
-
- @Override
- public void run() {
- try {
- opNode.initialize();
- if (nInputs > 0) {
- final Semaphore sem = new Semaphore(nInputs - 1);
- for (int i = 1; i < nInputs; ++i) {
- final IFrameReader reader = readers[i];
- final IFrameWriter writer = opNode.getInputFrameWriter(i);
- sem.acquire();
- executor.execute(new Runnable() {
- public void run() {
- try {
- pushFrames(reader, writer);
- } catch (HyracksDataException e) {
- } finally {
- sem.release();
- }
- }
- });
- }
- try {
- pushFrames(readers[0], opNode.getInputFrameWriter(0));
- } finally {
- sem.acquire(nInputs - 1);
- }
- }
- opNode.deinitialize();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- private void pushFrames(IFrameReader reader, IFrameWriter writer) throws HyracksDataException {
- ByteBuffer buffer = ctx.allocateFrame();
- writer.open();
- reader.open();
- while (reader.nextFrame(buffer)) {
- if (abort) {
- break;
- }
- buffer.flip();
- writer.nextFrame(buffer);
- buffer.compact();
- }
- reader.close();
- writer.close();
- }
-
- @Override
- public String toString() {
- return "OperatorRunnable[" + opNode.getDisplayName() + "]";
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
deleted file mode 100644
index 49d3db7..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicFrameReader.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * 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;
-
-import java.nio.ByteBuffer;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public class NonDeterministicFrameReader implements IFrameReader {
- private static final Logger LOGGER = Logger.getLogger(NonDeterministicFrameReader.class.getName());
-
- private final IHyracksStageletContext ctx;
- private final IConnectionDemultiplexer demux;
- private int lastReadSender;
- private boolean eos;
-
- public NonDeterministicFrameReader(IHyracksStageletContext ctx, IConnectionDemultiplexer demux) {
- this.ctx = ctx;
- this.demux = demux;
- }
-
- @Override
- public void open() throws HyracksDataException {
- lastReadSender = 0;
- eos = false;
- }
-
- @Override
- public void close() throws HyracksDataException {
- }
-
- @Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (eos) {
- return false;
- }
- while (true) {
- IConnectionEntry entry = demux.findNextReadyEntry(lastReadSender);
- if (entry.aborted()) {
- eos = true;
- return false;
- }
- lastReadSender = (Integer) entry.getAttachment();
- ByteBuffer netBuffer = entry.getReadBuffer();
- int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()));
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("Frame Tuple Count: " + tupleCount);
- }
- if (tupleCount == 0) {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Empty Frame received: Closing " + lastReadSender);
- }
- int openEntries = demux.closeEntry(lastReadSender);
- if (openEntries == 0) {
- eos = true;
- return false;
- }
- netBuffer.clear();
- demux.unreadyEntry(lastReadSender);
- } else {
- buffer.clear();
- buffer.put(netBuffer);
- netBuffer.clear();
- demux.unreadyEntry(lastReadSender);
- return true;
- }
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
deleted file mode 100644
index 7be9b08..0000000
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergeFrameReader.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * 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;
-
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-import java.util.PriorityQueue;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IConnectionEntry;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileHandle;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.io.IIOManager;
-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.io.FrameTuplePairComparator;
-
-public class SortMergeFrameReader implements IFrameReader {
- private static final Logger LOGGER = Logger.getLogger(SortMergeFrameReader.class.getName());
-
- private final IHyracksStageletContext ctx;
- private final IConnectionDemultiplexer demux;
- private final FrameTuplePairComparator tpc;
- private final FrameTupleAppender appender;
- private final RecordDescriptor recordDescriptor;
- private Run[] runs;
- private ByteBuffer[] frames;
- private PriorityQueue<Integer> pQueue;
- private int lastReadSender;
- private boolean first;
-
- public SortMergeFrameReader(IHyracksStageletContext ctx, IConnectionDemultiplexer demux, int[] sortFields,
- IBinaryComparator[] comparators, RecordDescriptor recordDescriptor) {
- this.ctx = ctx;
- this.demux = demux;
- tpc = new FrameTuplePairComparator(sortFields, sortFields, comparators);
- appender = new FrameTupleAppender(ctx.getFrameSize());
- this.recordDescriptor = recordDescriptor;
- }
-
- @Override
- public void open() throws HyracksDataException {
- int nSenders = demux.getSenderCount();
- runs = new Run[nSenders];
- frames = new ByteBuffer[nSenders];
- for (int i = 0; i < runs.length; ++i) {
- runs[i] = new Run(i);
- frames[i] = ctx.allocateFrame();
- }
- pQueue = new PriorityQueue<Integer>(nSenders, new Comparator<Integer>() {
- @Override
- public int compare(Integer o1, Integer o2) {
- int i1 = o1.intValue();
- int i2 = o2.intValue();
- Run r1 = runs[i1];
- Run r2 = runs[i2];
-
- int c = tpc.compare(r1.accessor, r1.tIndex, r2.accessor, r2.tIndex);
- return c == 0 ? (i1 < i2 ? -1 : 1) : c;
- }
- });
- lastReadSender = 0;
- first = true;
- }
-
- @Override
- public void close() throws HyracksDataException {
- for (Run r : runs) {
- r.close();
- }
- }
-
- @Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- buffer.clear();
- buffer.position(buffer.capacity());
- appender.reset(buffer, true);
- if (first) {
- for (int i = 0; i < runs.length; ++i) {
- if (runs[i].next()) {
- pQueue.add(Integer.valueOf(i));
- }
- }
- }
- first = false;
- while (true) {
- if (pQueue.isEmpty()) {
- return appender.getTupleCount() > 0;
- }
- Integer top = pQueue.peek();
- Run run = runs[top.intValue()];
- if (!appender.append(run.accessor, run.tIndex)) {
- return true;
- }
- pQueue.remove();
- if (run.next()) {
- pQueue.add(top);
- }
- }
- }
-
- private class Run {
- private final int runId;
- private final FileReference fRef;
- private final FileHandle fHandle;
- private final ByteBuffer frame;
- private final FrameTupleAccessor accessor;
- private int tIndex;
- private long readFP;
- private long writeFP;
- private boolean eof;
-
- public Run(int runId) throws HyracksDataException {
- this.runId = runId;
- fRef = ctx.createWorkspaceFile(SortMergeFrameReader.class.getName());
- fHandle = ctx.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
- frame = ctx.allocateFrame();
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- readFP = 0;
- writeFP = 0;
- eof = false;
- }
-
- public void close() throws HyracksDataException {
- ctx.getIOManager().close(fHandle);
- }
-
- private void write(ByteBuffer frame) throws HyracksDataException {
- int sz = ctx.getIOManager().syncWrite(fHandle, writeFP, frame);
- writeFP += sz;
- }
-
- private boolean next() throws HyracksDataException {
- ++tIndex;
- while (readFP == 0 || tIndex >= accessor.getTupleCount()) {
- if (!read(frame)) {
- return false;
- }
- accessor.reset(frame);
- tIndex = 0;
- }
- return true;
- }
-
- private boolean read(ByteBuffer frame) throws HyracksDataException {
- frame.clear();
- while (!eof && readFP >= writeFP) {
- spoolRuns(runId);
- }
- if (eof && readFP >= writeFP) {
- return false;
- }
- int sz = ctx.getIOManager().syncRead(fHandle, readFP, frame);
- readFP += sz;
- return true;
- }
-
- private void eof() {
- eof = true;
- }
- }
-
- private void spoolRuns(int interestingRun) throws HyracksDataException {
- while (true) {
- IConnectionEntry entry = demux.findNextReadyEntry(lastReadSender);
- lastReadSender = (Integer) entry.getAttachment();
- ByteBuffer netBuffer = entry.getReadBuffer();
- int tupleCount = netBuffer.getInt(FrameHelper.getTupleCountOffset(ctx.getFrameSize()));
- if (LOGGER.isLoggable(Level.FINER)) {
- LOGGER.finer("Frame Tuple Count: " + tupleCount);
- }
- if (tupleCount == 0) {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Empty Frame received: Closing " + lastReadSender);
- }
- int openEntries = demux.closeEntry(lastReadSender);
- runs[lastReadSender].eof();
- netBuffer.clear();
- demux.unreadyEntry(lastReadSender);
- if (openEntries == 0) {
- return;
- }
- } else {
- runs[lastReadSender].write(netBuffer);
- netBuffer.clear();
- demux.unreadyEntry(lastReadSender);
- }
- if (lastReadSender == interestingRun) {
- return;
- }
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index d2a3c53..f4862d8 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -17,7 +17,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -33,7 +33,7 @@
private final FrameDeserializer frameDeserializer;
- public FrameDeserializingDataReader(IHyracksStageletContext ctx, IFrameReader frameReader,
+ public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
RecordDescriptor recordDescriptor) {
buffer = ctx.allocateFrame();
this.frameReader = frameReader;
diff --git a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 9edd3bf..7c7979b 100644
--- a/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -19,7 +19,7 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -39,8 +39,7 @@
private boolean open;
- public SerializingDataWriter(IHyracksStageletContext ctx, RecordDescriptor recordDescriptor,
- IFrameWriter frameWriter) {
+ public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter) {
buffer = ctx.allocateFrame();
tb = new ArrayTupleBuilder(recordDescriptor.getFields().length);
this.recordDescriptor = recordDescriptor;
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
index fd549b3..14c034a 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
@@ -34,7 +34,7 @@
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.util.ReflectionUtils;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -367,7 +367,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
JobConf conf = getJobConf();
@@ -398,7 +398,7 @@
(Class<? extends Writable>) oldReader.createKey().getClass(),
(Class<? extends Writable>) oldReader.createValue().getClass());
}
- return createSelfReadingMapper(ctx, env, recordDescriptor, partition);
+ return createSelfReadingMapper(ctx, recordDescriptor, partition);
} else {
return new DeserializedOperatorNodePushable(ctx, new MapperOperator(partition), recordDescProvider
.getInputRecordDescriptor(this.odId, 0));
@@ -408,7 +408,7 @@
}
}
- private IOperatorNodePushable createSelfReadingMapper(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+ private IOperatorNodePushable createSelfReadingMapper(final IHyracksTaskContext ctx,
final RecordDescriptor recordDescriptor, final int partition) {
return new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 915d09b..52f0c8a 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -32,7 +32,7 @@
import org.apache.hadoop.util.ReflectionUtils;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -143,7 +143,7 @@
@SuppressWarnings("deprecation")
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
return new AbstractUnaryOutputSourceOperatorNodePushable() {
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
index 5857f36..0af0596 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
@@ -35,7 +35,7 @@
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.ReflectionUtils;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IDataReader;
import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -348,7 +348,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
try {
if (this.comparatorFactory == null) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
index c4d623c..45d49e2 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/AvgAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -39,7 +39,7 @@
}
@Override
- public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+ public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException {
if (this.outField < 0)
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
index 90edf4a..1cc3340 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/ConcatAggregatorDescriptorFactory.java
@@ -21,7 +21,7 @@
import java.io.IOException;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -61,7 +61,7 @@
* onto the output frame for indexing the aggregation result from the buffer.
*/
@Override
- public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+ public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
if (this.outField < 0)
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
index c5f0a42..74ac53a 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/CountAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -36,7 +36,7 @@
}
@Override
- public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+ public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
if (this.outField < 0) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
index e324e1c..7f6928b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IAggregatorDescriptorFactory.java
@@ -16,7 +16,7 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -32,6 +32,6 @@
* @return
* @throws HyracksDataException
*/
- IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+ IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, int[] keyFields) throws HyracksDataException;
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
index e07b123..770d36b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/IntSumAggregatorDescriptorFactory.java
@@ -18,7 +18,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -39,7 +39,7 @@
}
@Override
- public IAggregatorDescriptor createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDescriptor,
+ public IAggregatorDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, final int[] keyFields) throws HyracksDataException {
if (this.outField < 0) {
@@ -111,7 +111,7 @@
@Override
public void reset() {
-
+
}
};
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
index 58d0da7..053b9e2 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorDescriptorFactory.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.dataflow.std.aggregators;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -29,7 +29,7 @@
}
@Override
- public IAggregatorDescriptor createAggregator(final IHyracksStageletContext ctx,
+ public IAggregatorDescriptor createAggregator(final IHyracksTaskContext ctx,
final RecordDescriptor inRecordDescriptor, final RecordDescriptor outRecordDescriptor, final int[] keyFields)
throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
index eb9c12d..8237110 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
@@ -17,7 +17,7 @@
import java.io.DataOutput;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -35,7 +35,7 @@
}
@Override
- public IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
+ public IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
RecordDescriptor outRecordDescriptor) throws HyracksDataException {
final IFieldValueResultingAggregator aggregators[] = new IFieldValueResultingAggregator[aFactories.length];
for (int i = 0; i < aFactories.length; ++i) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
index 4976230..11a02b3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractActivityNode.java
@@ -14,22 +14,20 @@
*/
package edu.uci.ics.hyracks.dataflow.std.base;
-import java.util.UUID;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-
-public abstract class AbstractActivityNode implements IActivityNode {
+public abstract class AbstractActivityNode implements IActivity {
private static final long serialVersionUID = 1L;
- protected final ActivityNodeId id;
+ protected final ActivityId id;
- public AbstractActivityNode() {
- this.id = new ActivityNodeId(getOwner().getOperatorId(), UUID.randomUUID());
+ public AbstractActivityNode(ActivityId id) {
+ this.id = id;
}
@Override
- public ActivityNodeId getActivityNodeId() {
+ public ActivityId getActivityId() {
return id;
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index e4613dd..f3535f8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
@@ -14,16 +14,14 @@
*/
package edu.uci.ics.hyracks.dataflow.std.base;
-import java.util.UUID;
-
import org.json.JSONException;
import org.json.JSONObject;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public abstract class AbstractConnectorDescriptor implements IConnectorDescriptor {
@@ -31,7 +29,7 @@
protected final ConnectorDescriptorId id;
public AbstractConnectorDescriptor(JobSpecification spec) {
- this.id = new ConnectorDescriptorId(UUID.randomUUID());
+ this.id = spec.createConnectorDescriptor();
spec.getConnectorMap().put(id, this);
}
@@ -44,14 +42,14 @@
JSONObject jconn = new JSONObject();
jconn.put("type", "connector");
- jconn.put("id", getConnectorId().getId().toString());
+ jconn.put("id", getConnectorId().getId());
jconn.put("java-class", getClass().getName());
return jconn;
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx) {
// do nothing
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
new file mode 100644
index 0000000..aa550ae
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.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.dataflow.std.base;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public AbstractMToNConnectorDescriptor(JobSpecification spec) {
+ super(spec);
+ }
+
+ @Override
+ public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+ BitSet targetBitmap) {
+ targetBitmap.clear();
+ targetBitmap.set(0, nConsumerPartitions);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
index 4453412..15fc75d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractOperatorDescriptor.java
@@ -14,17 +14,15 @@
*/
package edu.uci.ics.hyracks.dataflow.std.base;
-import java.util.UUID;
-
import org.json.JSONException;
import org.json.JSONObject;
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public abstract class AbstractOperatorDescriptor implements IOperatorDescriptor {
@@ -41,7 +39,7 @@
protected final int outputArity;
public AbstractOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
- odId = new OperatorDescriptorId(UUID.randomUUID());
+ odId = spec.createOperatorDescriptorId();
this.inputArity = inputArity;
this.outputArity = outputArity;
recordDescriptors = new RecordDescriptor[outputArity];
@@ -69,7 +67,7 @@
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx) {
// do nothing
}
@@ -78,7 +76,7 @@
public JSONObject toJSON() throws JSONException {
JSONObject jop = new JSONObject();
jop.put("type", "operator");
- jop.put("id", getOperatorId().getId().toString());
+ jop.put("id", getOperatorId().getId());
jop.put("java-class", getClass().getName());
jop.put("in-arity", getInputArity());
jop.put("out-arity", getOutputArity());
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
index 018b7b7..c4c3326 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -14,38 +14,29 @@
*/
package edu.uci.ics.hyracks.dataflow.std.base;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements
- IActivityNode {
+public abstract class AbstractSingleActivityOperatorDescriptor extends AbstractOperatorDescriptor implements IActivity {
private static final long serialVersionUID = 1L;
- protected final ActivityNodeId activityNodeId;
+ protected final ActivityId activityNodeId;
public AbstractSingleActivityOperatorDescriptor(JobSpecification spec, int inputArity, int outputArity) {
super(spec, inputArity, outputArity);
- activityNodeId = new ActivityNodeId(odId, UUID.randomUUID());
+ activityNodeId = new ActivityId(odId, 0);
}
@Override
- public ActivityNodeId getActivityNodeId() {
+ public ActivityId getActivityId() {
return activityNodeId;
}
@Override
- public final IOperatorDescriptor getOwner() {
- return this;
- }
-
- @Override
- public final void contributeTaskGraph(IActivityGraphBuilder builder) {
- builder.addTask(this);
+ public final void contributeActivities(IActivityGraphBuilder builder) {
+ builder.addActivity(this);
for (int i = 0; i < getInputArity(); ++i) {
builder.addSourceEdge(i, this, i);
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
new file mode 100644
index 0000000..b2dad8f
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/AbstractPartitionCollector.java
@@ -0,0 +1,49 @@
+/*
+ * 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.collectors;
+
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public abstract class AbstractPartitionCollector implements IPartitionCollector {
+ protected final IHyracksTaskContext ctx;
+
+ protected final ConnectorDescriptorId connectorId;
+
+ protected final int receiverIndex;
+
+ public AbstractPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex) {
+ this.ctx = ctx;
+ this.connectorId = connectorId;
+ this.receiverIndex = receiverIndex;
+ }
+
+ @Override
+ public JobId getJobId() {
+ return ctx.getJobletContext().getJobId();
+ }
+
+ @Override
+ public ConnectorDescriptorId getConnectorId() {
+ return connectorId;
+ }
+
+ @Override
+ public int getReceiverIndex() {
+ return receiverIndex;
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
new file mode 100644
index 0000000..5ff528e
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicPartitionCollector.java
@@ -0,0 +1,197 @@
+/*
+ * 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.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class NonDeterministicPartitionCollector extends AbstractPartitionCollector {
+ private final FrameReader reader;
+
+ private final BitSet expectedPartitions;
+
+ private final int nSenderPartitions;
+
+ private final IInputChannel[] channels;
+
+ private final BitSet frameAvailability;
+
+ private final int[] availableFrameCounts;
+
+ private final BitSet eosSenders;
+
+ private BitSet closedSenders;
+
+ private int lastReadSender;
+
+ public NonDeterministicPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId,
+ int receiverIndex, int nSenderPartitions, BitSet expectedPartitions) {
+ super(ctx, connectorId, receiverIndex);
+ this.expectedPartitions = expectedPartitions;
+ this.nSenderPartitions = nSenderPartitions;
+ reader = new FrameReader();
+ channels = new IInputChannel[nSenderPartitions];
+ eosSenders = new BitSet(nSenderPartitions);
+ closedSenders = new BitSet(nSenderPartitions);
+ closedSenders.or(expectedPartitions);
+ closedSenders.flip(0, nSenderPartitions);
+ frameAvailability = new BitSet(nSenderPartitions);
+ availableFrameCounts = new int[nSenderPartitions];
+ }
+
+ @Override
+ public void open() throws HyracksException {
+ lastReadSender = 0;
+ }
+
+ @Override
+ public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+ for (PartitionChannel pc : partitions) {
+ PartitionId pid = pc.getPartitionId();
+ IInputChannel channel = pc.getInputChannel();
+ channel.setAttachment(pid);
+ channel.registerMonitor(reader);
+ synchronized (this) {
+ channels[pid.getSenderIndex()] = channel;
+ }
+ channel.open();
+ }
+ }
+
+ @Override
+ public IFrameReader getReader() throws HyracksException {
+ return reader;
+ }
+
+ @Override
+ public void close() throws HyracksException {
+ }
+
+ private final class FrameReader implements IFrameReader, IInputChannelMonitor {
+ @Override
+ public void open() throws HyracksDataException {
+ }
+
+ @Override
+ public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ findNextSender();
+ if (lastReadSender >= 0) {
+ ByteBuffer srcFrame = channels[lastReadSender].getNextBuffer();
+ FrameUtils.copy(srcFrame, buffer);
+ channels[lastReadSender].recycleBuffer(srcFrame);
+ return true;
+ }
+ return false;
+ }
+
+ private void findNextSender() throws HyracksDataException {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ while (true) {
+ switch (lastReadSender) {
+ default:
+ lastReadSender = frameAvailability.nextSetBit(lastReadSender + 1);
+ if (lastReadSender >= 0) {
+ break;
+ }
+ case 0:
+ lastReadSender = frameAvailability.nextSetBit(0);
+ }
+ if (lastReadSender >= 0) {
+ assert availableFrameCounts[lastReadSender] > 0;
+ if (--availableFrameCounts[lastReadSender] == 0) {
+ frameAvailability.clear(lastReadSender);
+ }
+ return;
+ }
+ for (int i = eosSenders.nextSetBit(0); i >= 0; i = eosSenders.nextSetBit(i)) {
+ channels[i].close();
+ eosSenders.clear(i);
+ closedSenders.set(i);
+ }
+ int nextClosedBitIndex = closedSenders.nextClearBit(0);
+ if (nextClosedBitIndex < 0 || nextClosedBitIndex >= nSenderPartitions) {
+ lastReadSender = -1;
+ return;
+ }
+ try {
+ NonDeterministicPartitionCollector.this.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ for (int i = closedSenders.nextClearBit(0); i >= 0 && i < nSenderPartitions; i = closedSenders
+ .nextClearBit(i + 1)) {
+ if (channels[i] != null) {
+ channels[i].close();
+ channels[i] = null;
+ }
+ }
+ }
+ }
+
+ @Override
+ public void notifyDataAvailability(IInputChannel channel, int nFrames) {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ PartitionId pid = (PartitionId) channel.getAttachment();
+ int senderIndex = pid.getSenderIndex();
+ availableFrameCounts[senderIndex] += nFrames;
+ frameAvailability.set(senderIndex);
+ NonDeterministicPartitionCollector.this.notifyAll();
+ }
+ }
+
+ @Override
+ public void notifyEndOfStream(IInputChannel channel) {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ PartitionId pid = (PartitionId) channel.getAttachment();
+ int senderIndex = pid.getSenderIndex();
+ eosSenders.set(senderIndex);
+ NonDeterministicPartitionCollector.this.notifyAll();
+ }
+ }
+ }
+
+ @Override
+ public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+ Collection<PartitionId> c = new ArrayList<PartitionId>(expectedPartitions.cardinality());
+ for (int i = expectedPartitions.nextSetBit(0); i >= 0; i = expectedPartitions.nextSetBit(i + 1)) {
+ c.add(new PartitionId(getJobId(), getConnectorId(), i, getReceiverIndex()));
+ }
+ return c;
+ }
+
+ @Override
+ public void abort() {
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java
new file mode 100644
index 0000000..8faee7b
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergePartitionCollector.java
@@ -0,0 +1,263 @@
+/*
+ * 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.collectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.PartitionChannel;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+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.partitions.PartitionId;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+
+public class SortMergePartitionCollector extends AbstractPartitionCollector {
+ private final int[] sortFields;
+
+ private final IBinaryComparator[] comparators;
+
+ private final RecordDescriptor recordDescriptor;
+
+ private final int maxConcurrentMerges;
+
+ private final IInputChannel[] channels;
+
+ private final int nSenders;
+
+ private final boolean stable;
+
+ private final FrameReader frameReader;
+
+ private final PartitionBatchManager pbm;
+
+ public SortMergePartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex,
+ int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDescriptor,
+ int maxConcurrentMerges, int nSenders, boolean stable) {
+ super(ctx, connectorId, receiverIndex);
+ this.sortFields = sortFields;
+ this.comparators = comparators;
+ this.recordDescriptor = recordDescriptor;
+ this.maxConcurrentMerges = maxConcurrentMerges;
+ channels = new IInputChannel[nSenders];
+ this.nSenders = nSenders;
+ this.stable = stable;
+ this.frameReader = new FrameReader();
+ pbm = new NonDeterministicPartitionBatchManager();
+ }
+
+ @Override
+ public void open() throws HyracksException {
+ }
+
+ @Override
+ public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+ for (PartitionChannel pc : partitions) {
+ PartitionId pid = pc.getPartitionId();
+ IInputChannel channel = pc.getInputChannel();
+ InputChannelFrameReader channelReader = new InputChannelFrameReader(channel);
+ channel.registerMonitor(channelReader);
+ channel.setAttachment(channelReader);
+ int senderIndex = pid.getSenderIndex();
+ synchronized (this) {
+ channels[senderIndex] = channel;
+ }
+ pbm.addPartition(senderIndex);
+ channel.open();
+ }
+ }
+
+ @Override
+ public IFrameReader getReader() throws HyracksException {
+ return frameReader;
+ }
+
+ @Override
+ public void close() throws HyracksException {
+
+ }
+
+ @Override
+ public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+ Collection<PartitionId> requiredPartitionIds = new ArrayList<PartitionId>();
+ for (int i = 0; i < nSenders; ++i) {
+ requiredPartitionIds.add(new PartitionId(getJobId(), getConnectorId(), i, receiverIndex));
+ }
+ return requiredPartitionIds;
+ }
+
+ @Override
+ public void abort() {
+
+ }
+
+ private abstract class PartitionBatchManager {
+ protected abstract void addPartition(int index);
+
+ protected abstract void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException;
+ }
+
+ private class NonDeterministicPartitionBatchManager extends PartitionBatchManager {
+ private List<IFrameReader> partitions;
+
+ private List<IFrameReader> batch;
+
+ private int requiredSize;
+
+ public NonDeterministicPartitionBatchManager() {
+ partitions = new ArrayList<IFrameReader>();
+ }
+
+ @Override
+ protected void addPartition(int index) {
+ synchronized (SortMergePartitionCollector.this) {
+ if (batch != null && batch.size() < requiredSize) {
+ batch.add((IFrameReader) channels[index].getAttachment());
+ if (batch.size() == requiredSize) {
+ SortMergePartitionCollector.this.notifyAll();
+ }
+ } else {
+ partitions.add((IFrameReader) channels[index].getAttachment());
+ }
+ }
+ }
+
+ @Override
+ protected void getNextBatch(List<IFrameReader> batch, int size) throws HyracksDataException {
+ synchronized (SortMergePartitionCollector.this) {
+ if (partitions.size() <= size) {
+ batch.addAll(partitions);
+ partitions.clear();
+ } else if (partitions.size() > size) {
+ List<IFrameReader> sublist = partitions.subList(0, size);
+ batch.addAll(sublist);
+ sublist.clear();
+ }
+ if (batch.size() == size) {
+ return;
+ }
+ this.batch = batch;
+ this.requiredSize = size;
+ while (batch.size() < size) {
+ try {
+ SortMergePartitionCollector.this.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ this.batch = null;
+ }
+ }
+ }
+
+ private static class InputChannelFrameReader implements IFrameReader, IInputChannelMonitor {
+ private final IInputChannel channel;
+
+ private int availableFrames;
+
+ private boolean eos;
+
+ public InputChannelFrameReader(IInputChannel channel) {
+ this.channel = channel;
+ availableFrames = 0;
+ eos = false;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ }
+
+ @Override
+ public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ synchronized (this) {
+ while (!eos && availableFrames <= 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ if (availableFrames <=0 && eos) {
+ return false;
+ }
+ --availableFrames;
+ }
+ ByteBuffer srcBuffer = channel.getNextBuffer();
+ FrameUtils.copy(srcBuffer, buffer);
+ channel.recycleBuffer(srcBuffer);
+ return true;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+
+ }
+
+ @Override
+ public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+ availableFrames += nFrames;
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyEndOfStream(IInputChannel channel) {
+ eos = true;
+ notifyAll();
+ }
+ }
+
+ private class FrameReader implements IFrameReader {
+ private RunMergingFrameReader merger;
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (maxConcurrentMerges >= nSenders) {
+ List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+ for (int i = 0; i < nSenders; ++i) {
+ inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+ }
+ List<IFrameReader> batch = new ArrayList<IFrameReader>();
+ pbm.getNextBatch(batch, nSenders);
+ merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames,
+ sortFields, comparators, recordDescriptor);
+ } else {
+ // multi level merge.
+ throw new HyracksDataException("Not yet supported");
+ }
+ merger.open();
+ }
+
+ @Override
+ public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ buffer.position(buffer.capacity());
+ buffer.limit(buffer.capacity());
+ return merger.nextFrame(buffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ merger.close();
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
deleted file mode 100644
index b814867..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningConnectorDescriptor.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.connectors;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
-
-public class MToNHashPartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
- private static final long serialVersionUID = 1L;
- private ITuplePartitionComputerFactory tpcf;
-
- public MToNHashPartitioningConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf) {
- super(spec);
- this.tpcf = tpcf;
- }
-
- @Override
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
- tpcf.createPartitioner());
- return hashWriter;
- }
-
- @Override
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- return new NonDeterministicFrameReader(ctx, demux);
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
deleted file mode 100644
index 36f8d5b..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNHashPartitioningMergingConnectorDescriptor.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * 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.connectors;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
-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.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.SortMergeFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
-
-public class MToNHashPartitioningMergingConnectorDescriptor extends AbstractConnectorDescriptor {
- private static final long serialVersionUID = 1L;
-
- private final ITuplePartitionComputerFactory tpcf;
- private final int[] sortFields;
- private final IBinaryComparatorFactory[] comparatorFactories;
-
- public MToNHashPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
- int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
- super(spec);
- this.tpcf = tpcf;
- this.sortFields = sortFields;
- this.comparatorFactories = comparatorFactories;
- }
-
- @Override
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- final HashDataWriter hashWriter = new HashDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
- tpcf.createPartitioner());
- return hashWriter;
- }
-
- @Override
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- return new SortMergeFrameReader(ctx, demux, sortFields, comparators, recordDesc);
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..12935af
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.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.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
+
+public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+ private ITuplePartitionComputerFactory tpcf;
+
+ public MToNPartitioningConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf) {
+ super(spec);
+ this.tpcf = tpcf;
+ }
+
+ @Override
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ throws HyracksDataException {
+ final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+ recordDesc, tpcf.createPartitioner());
+ return hashWriter;
+ }
+
+ @Override
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ BitSet expectedPartitions = new BitSet(nProducerPartitions);
+ expectedPartitions.set(0, nProducerPartitions);
+ return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+ expectedPartitions);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
new file mode 100644
index 0000000..54979f0
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -0,0 +1,71 @@
+/*
+ * 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.connectors;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.SortMergePartitionCollector;
+
+public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ITuplePartitionComputerFactory tpcf;
+ private final int[] sortFields;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+ private final boolean stable;
+
+ public MToNPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+ int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+ this(spec, tpcf, sortFields, comparatorFactories, false);
+ }
+
+ public MToNPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+ int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, boolean stable) {
+ super(spec);
+ this.tpcf = tpcf;
+ this.sortFields = sortFields;
+ this.comparatorFactories = comparatorFactories;
+ this.stable = stable;
+ }
+
+ @Override
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ throws HyracksDataException {
+ final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory,
+ recordDesc, tpcf.createPartitioner());
+ return hashWriter;
+ }
+
+ @Override
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ return new SortMergePartitionCollector(ctx, getConnectorId(), index, sortFields, comparators, recordDesc,
+ nProducerPartitions, nProducerPartitions, stable);
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
deleted file mode 100644
index e108396..0000000
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.connectors;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-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.std.base.AbstractConnectorDescriptor;
-
-public class MToNRangePartitioningConnectorDescriptor extends AbstractConnectorDescriptor {
- private static final long serialVersionUID = 1L;
-
- private class RangeDataWriter implements IFrameWriter {
- private final IFrameWriter[] epWriters;
- private final FrameTupleAppender[] appenders;
- private final FrameTupleAccessor tupleAccessor;
-
- public RangeDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount, IFrameWriter[] epWriters,
- FrameTupleAppender[] appenders, RecordDescriptor recordDescriptor) {
- this.epWriters = epWriters;
- this.appenders = appenders;
- tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- }
-
- @Override
- public void close() throws HyracksDataException {
- for (int i = 0; i < epWriters.length; ++i) {
- if (appenders[i].getTupleCount() > 0) {
- flushFrame(appenders[i].getBuffer(), epWriters[i]);
- }
- epWriters[i].close();
- }
- }
-
- private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
- frameWriter.nextFrame(buffer);
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (true) {
- throw new UnsupportedOperationException();
- }
- tupleAccessor.reset(buffer);
- int slotLength = tupleAccessor.getFieldSlotsLength();
- int tupleCount = tupleAccessor.getTupleCount();
- for (int i = 0; i < tupleCount; ++i) {
- int startOffset = tupleAccessor.getTupleStartOffset(i);
- int fStart = tupleAccessor.getFieldStartOffset(i, partitioningField);
- int fEnd = tupleAccessor.getFieldEndOffset(i, partitioningField);
- int h = 0;
- FrameTupleAppender appender = appenders[h];
- if (!appender.append(tupleAccessor, i)) {
- ByteBuffer appenderBuffer = appender.getBuffer();
- flushFrame(appenderBuffer, epWriters[h]);
- appender.reset(appenderBuffer, true);
- }
- }
- }
-
- @Override
- public void open() throws HyracksDataException {
- for (int i = 0; i < epWriters.length; ++i) {
- epWriters[i].open();
- appenders[i].reset(appenders[i].getBuffer(), true);
- }
- }
-
- @Override
- public void flush() throws HyracksDataException {
- for (int i = 0; i < appenders.length; ++i) {
- FrameTupleAppender appender = appenders[i];
- if (appender.getTupleCount() > 0) {
- ByteBuffer buffer = appender.getBuffer();
- IFrameWriter frameWriter = epWriters[i];
- flushFrame(buffer, frameWriter);
- epWriters[i].flush();
- appender.reset(buffer, true);
- }
- }
- }
- }
-
- private final int partitioningField;
- private final Object[] splitVector;
-
- public MToNRangePartitioningConnectorDescriptor(JobSpecification spec, int partitioningField, Object[] splitVector) {
- super(spec);
- this.partitioningField = partitioningField;
- this.splitVector = splitVector;
- }
-
- @Override
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
- final FrameTupleAppender[] appenders = new FrameTupleAppender[nConsumerPartitions];
- for (int i = 0; i < nConsumerPartitions; ++i) {
- try {
- epWriters[i] = edwFactory.createFrameWriter(i);
- appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
- appenders[i].reset(ctx.allocateFrame(), true);
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
- final RangeDataWriter rangeWriter = new RangeDataWriter(ctx, nConsumerPartitions, epWriters, appenders,
- recordDesc);
- return rangeWriter;
- }
-
- @Override
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- return new NonDeterministicFrameReader(ctx, demux);
- }
-}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
index 37f68b7..5297a99 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNReplicatingConnectorDescriptor.java
@@ -15,19 +15,19 @@
package edu.uci.ics.hyracks.dataflow.std.connectors;
import java.nio.ByteBuffer;
+import java.util.BitSet;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+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.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
-public class MToNReplicatingConnectorDescriptor extends AbstractConnectorDescriptor {
+public class MToNReplicatingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
public MToNReplicatingConnectorDescriptor(JobSpecification spec) {
super(spec);
}
@@ -35,8 +35,8 @@
private static final long serialVersionUID = 1L;
@Override
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
throws HyracksDataException {
final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
for (int i = 0; i < nConsumerPartitions; ++i) {
@@ -78,9 +78,11 @@
}
@Override
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- return new NonDeterministicFrameReader(ctx, demux);
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ BitSet expectedPartitions = new BitSet(nProducerPartitions);
+ expectedPartitions.set(0, nProducerPartitions);
+ return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+ expectedPartitions);
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index 6ee9ab37..67130c9 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -14,22 +14,23 @@
*/
package edu.uci.ics.hyracks.dataflow.std.connectors;
+import java.util.BitSet;
+
import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.constraints.IConstraintExpressionAcceptor;
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
-import edu.uci.ics.hyracks.api.constraints.expressions.RelationalExpression;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.NonDeterministicFrameReader;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicPartitionCollector;
public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
private static final long serialVersionUID = 1L;
@@ -39,28 +40,36 @@
}
@Override
- public IFrameWriter createSendSideWriter(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IEndpointDataWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
throws HyracksDataException {
return edwFactory.createFrameWriter(index);
}
@Override
- public IFrameReader createReceiveSideReader(IHyracksStageletContext ctx, RecordDescriptor recordDesc,
- IConnectionDemultiplexer demux, int index, int nProducerPartitions, int nConsumerPartitions)
- throws HyracksDataException {
- return new NonDeterministicFrameReader(ctx, demux);
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ BitSet expectedPartitions = new BitSet(nProducerPartitions);
+ expectedPartitions.set(index);
+ return new NonDeterministicPartitionCollector(ctx, getConnectorId(), index, nProducerPartitions,
+ expectedPartitions);
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan,
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx) {
JobSpecification jobSpec = plan.getJobSpecification();
IOperatorDescriptor consumer = jobSpec.getConsumer(this);
IOperatorDescriptor producer = jobSpec.getProducer(this);
- constraintAcceptor.addConstraintExpression(new RelationalExpression(new PartitionCountExpression(consumer
- .getOperatorId()), new PartitionCountExpression(producer.getOperatorId()),
- RelationalExpression.Operator.EQUAL));
+ constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(consumer.getOperatorId()),
+ new PartitionCountExpression(producer.getOperatorId())));
+ }
+
+ @Override
+ public void indicateTargetPartitions(int nProducerPartitions, int nConsumerPartitions, int producerIndex,
+ BitSet targetBitmap) {
+ targetBitmap.clear();
+ targetBitmap.set(producerIndex);
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
similarity index 78%
rename from hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
rename to hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 44dd84c..7054535 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/HashDataWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -18,30 +18,29 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.IEndpointDataWriterFactory;
+import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
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.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-public class HashDataWriter implements IFrameWriter {
+public class PartitionDataWriter implements IFrameWriter {
private final int consumerPartitionCount;
- private final IFrameWriter[] epWriters;
+ private final IFrameWriter[] pWriters;
private final FrameTupleAppender[] appenders;
private final FrameTupleAccessor tupleAccessor;
private final ITuplePartitionComputer tpc;
- public HashDataWriter(IHyracksStageletContext ctx, int consumerPartitionCount,
- IEndpointDataWriterFactory edwFactory, RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc)
- throws HyracksDataException {
+ public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
+ RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
this.consumerPartitionCount = consumerPartitionCount;
- epWriters = new IFrameWriter[consumerPartitionCount];
+ pWriters = new IFrameWriter[consumerPartitionCount];
appenders = new FrameTupleAppender[consumerPartitionCount];
for (int i = 0; i < consumerPartitionCount; ++i) {
try {
- epWriters[i] = edwFactory.createFrameWriter(i);
+ pWriters[i] = pwFactory.createFrameWriter(i);
appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
appenders[i].reset(ctx.allocateFrame(), true);
} catch (IOException e) {
@@ -54,11 +53,11 @@
@Override
public void close() throws HyracksDataException {
- for (int i = 0; i < epWriters.length; ++i) {
+ for (int i = 0; i < pWriters.length; ++i) {
if (appenders[i].getTupleCount() > 0) {
- flushFrame(appenders[i].getBuffer(), epWriters[i]);
+ flushFrame(appenders[i].getBuffer(), pWriters[i]);
}
- epWriters[i].close();
+ pWriters[i].close();
}
}
@@ -70,8 +69,8 @@
@Override
public void open() throws HyracksDataException {
- for (int i = 0; i < epWriters.length; ++i) {
- epWriters[i].open();
+ for (int i = 0; i < pWriters.length; ++i) {
+ pWriters[i].open();
appenders[i].reset(appenders[i].getBuffer(), true);
}
}
@@ -85,7 +84,7 @@
FrameTupleAppender appender = appenders[h];
if (!appender.append(tupleAccessor, i)) {
ByteBuffer appenderBuffer = appender.getBuffer();
- flushFrame(appenderBuffer, epWriters[h]);
+ flushFrame(appenderBuffer, pWriters[h]);
appender.reset(appenderBuffer, true);
if (!appender.append(tupleAccessor, i)) {
throw new IllegalStateException();
@@ -100,9 +99,9 @@
FrameTupleAppender appender = appenders[i];
if (appender.getTupleCount() > 0) {
ByteBuffer buffer = appender.getBuffer();
- IFrameWriter frameWriter = epWriters[i];
+ IFrameWriter frameWriter = pWriters[i];
flushFrame(buffer, frameWriter);
- epWriters[i].flush();
+ pWriters[i].flush();
appender.reset(buffer, true);
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
index b2265dd..03438d3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractDeserializedFileScanOperatorDescriptor.java
@@ -16,7 +16,7 @@
import java.io.File;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -100,7 +100,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new DeserializedFileScanOperator(partition), null);
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
index 8e4199b..ae1a722 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.dataflow.std.file;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -85,7 +85,7 @@
protected abstract IRecordWriter createRecordWriter(FileSplit fileSplit, int index) throws Exception;
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 954d64a..0096576 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -23,7 +23,7 @@
import java.util.Arrays;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -42,7 +42,7 @@
}
@Override
- public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
+ public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
return new ITupleParser() {
@Override
public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
index 7cbb2ed..1d580b1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FileScanOperatorDescriptor.java
@@ -19,7 +19,7 @@
import java.io.FileNotFoundException;
import java.io.InputStream;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -45,7 +45,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final FileSplit split = fileSplitProvider.getFileSplits()[partition];
final ITupleParser tp = tupleParserFactory.createTupleParser(ctx);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
index 1628984..21e197d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/FrameFileWriterOperatorDescriptor.java
@@ -20,7 +20,7 @@
import java.io.OutputStream;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.exceptions.HyracksDataException;
@@ -40,7 +40,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
final FileSplit[] splits = fileSplitProvider.getFileSplits();
return new AbstractUnaryInputSinkOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
index a11dae0..eb051f5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/ITupleParserFactory.java
@@ -16,8 +16,8 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
public interface ITupleParserFactory extends Serializable {
- public ITupleParser createTupleParser(IHyracksStageletContext ctx);
+ public ITupleParser createTupleParser(IHyracksTaskContext ctx);
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index d599fee..cd320dd1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -20,7 +20,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -61,7 +61,7 @@
* @see edu.uci.ics.hyracks.api.dataflow.IActivityNode#createPushRuntime(edu.uci.ics.hyracks.api.context.IHyracksContext, edu.uci.ics.hyracks.api.job.IOperatorEnvironment, edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider, int, int)
*/
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
throws HyracksDataException {
// Output files
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
index 65f7f08..73de7f5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalGroupOperatorDescriptor.java
@@ -23,9 +23,9 @@
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -101,32 +101,30 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- AggregateActivity aggregateAct = new AggregateActivity();
- MergeActivity mergeAct = new MergeActivity();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ AggregateActivity aggregateAct = new AggregateActivity(new ActivityId(getOperatorId(), 0));
+ MergeActivity mergeAct = new MergeActivity(new ActivityId(odId, 1));
- builder.addTask(aggregateAct);
+ builder.addActivity(aggregateAct);
builder.addSourceEdge(0, aggregateAct, 0);
- builder.addTask(mergeAct);
+ builder.addActivity(mergeAct);
builder.addTargetEdge(0, mergeAct, 0);
builder.addBlockingEdge(aggregateAct, mergeAct);
}
private class AggregateActivity extends AbstractActivityNode {
-
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return ExternalGroupOperatorDescriptor.this;
+ public AggregateActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) throws HyracksDataException {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+ throws HyracksDataException {
final ISpillableTable gTable = spillableTableFactory.buildSpillableTable(ctx, keyFields,
comparatorFactories, firstNormalizerFactory, aggregatorFactory,
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0), recordDescriptors[0],
@@ -191,7 +189,7 @@
private void flushFramesToRun() throws HyracksDataException {
FileReference runFile;
try {
- runFile = ctx.getJobletContext().createWorkspaceFile(
+ runFile = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalGroupOperatorDescriptor.class.getSimpleName());
} catch (IOException e) {
throw new HyracksDataException(e);
@@ -215,18 +213,16 @@
}
private class MergeActivity extends AbstractActivityNode {
-
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return ExternalGroupOperatorDescriptor.this;
+ public MergeActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) throws HyracksDataException {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+ throws HyracksDataException {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -325,7 +321,7 @@
runNumber = runs.size();
} else {
runNumber = framesLimit - 2;
- newRun = ctx.getJobletContext().createWorkspaceFile(
+ newRun = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalGroupOperatorDescriptor.class.getSimpleName());
writer = new RunFileWriter(newRun, ctx.getIOManager());
writer.open();
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
index 07a5b81..5733c11 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/GroupingHashTable.java
@@ -20,7 +20,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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.ISerializerDeserializer;
@@ -60,7 +60,7 @@
}
private static final int INIT_ACCUMULATORS_SIZE = 8;
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final FrameTupleAppender appender;
private final List<ByteBuffer> buffers;
private final Link[] table;
@@ -79,7 +79,7 @@
private final FrameTupleAccessor storedKeysAccessor;
- GroupingHashTable(IHyracksStageletContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
+ GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
ITuplePartitionComputerFactory tpcf, IAccumulatingAggregatorFactory aggregatorFactory,
RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize) {
this.ctx = ctx;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
index 50d467c..cf954f6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashGroupOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -57,12 +57,12 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- HashBuildActivity ha = new HashBuildActivity();
- builder.addTask(ha);
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, 0));
+ builder.addActivity(ha);
- OutputActivity oa = new OutputActivity();
- builder.addTask(oa);
+ OutputActivity oa = new OutputActivity(new ActivityId(odId, 1));
+ builder.addActivity(oa);
builder.addSourceEdge(0, ha, 0);
builder.addTargetEdge(0, oa, 0);
@@ -72,10 +72,13 @@
private class HashBuildActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public HashBuildActivity(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
return new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -107,18 +110,17 @@
}
};
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return HashGroupOperatorDescriptor.this;
- }
}
private class OutputActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public OutputActivity(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
@@ -131,10 +133,5 @@
}
};
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return HashGroupOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
index 3cc2fad..af5d16b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableGroupingTableFactory.java
@@ -19,7 +19,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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.INormalizedKeyComputer;
@@ -52,7 +52,7 @@
}
@Override
- public ISpillableTable buildSpillableTable(final IHyracksStageletContext ctx, final int[] keyFields,
+ public ISpillableTable buildSpillableTable(final IHyracksTaskContext ctx, final int[] keyFields,
final IBinaryComparatorFactory[] comparatorFactories,
final INormalizedKeyComputerFactory firstKeyNormalizerFactory,
final IAggregatorDescriptorFactory aggregateDescriptorFactory, final RecordDescriptor inRecordDescriptor,
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
index 8eff82f..978f671 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/IAccumulatingAggregatorFactory.java
@@ -16,12 +16,11 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
public interface IAccumulatingAggregatorFactory extends Serializable {
- IAccumulatingAggregator createAggregator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc,
+ IAccumulatingAggregator createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc,
RecordDescriptor outRecordDescriptor) throws HyracksDataException;
-
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
index 07f0ac0..0ff1d1d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTableFactory.java
@@ -16,7 +16,7 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -27,7 +27,7 @@
* @author jarodwen
*/
public interface ISpillableTableFactory extends Serializable {
- ISpillableTable buildSpillableTable(IHyracksStageletContext ctx, int[] keyFields,
+ ISpillableTable buildSpillableTable(IHyracksTaskContext ctx, int[] keyFields,
IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory normalizedKeyComputerFactory,
IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor, int framesLimit) throws HyracksDataException;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
index ea86d19..b5568e6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupOperatorDescriptor.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -48,7 +48,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
index 8817113..60a1dc3 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/PreclusteredGroupWriter.java
@@ -17,7 +17,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -37,7 +37,7 @@
private final FrameTupleAppender appender;
private boolean first;
- public PreclusteredGroupWriter(IHyracksStageletContext ctx, int[] groupFields, IBinaryComparator[] comparators,
+ public PreclusteredGroupWriter(IHyracksTaskContext ctx, int[] groupFields, IBinaryComparator[] comparators,
IAccumulatingAggregator aggregator, RecordDescriptor inRecordDesc, IFrameWriter writer) {
this.groupFields = groupFields;
this.comparators = comparators;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
index 1a69952..c60c0af 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -97,20 +97,20 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- HashPartitionActivityNode part0 = new HashPartitionActivityNode(RELATION0, keys0, 0);
- HashPartitionActivityNode part1 = new HashPartitionActivityNode(RELATION1, keys1, 1);
- JoinActivityNode join = new JoinActivityNode();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ HashPartitionActivityNode rpart = new HashPartitionActivityNode(new ActivityId(odId, 0), RELATION0, keys0, 0);
+ HashPartitionActivityNode spart = new HashPartitionActivityNode(new ActivityId(odId, 1), RELATION1, keys1, 1);
+ JoinActivityNode join = new JoinActivityNode(new ActivityId(odId, 2));
- builder.addTask(part0);
- builder.addSourceEdge(0, part0, 0);
+ builder.addActivity(rpart);
+ builder.addSourceEdge(0, rpart, 0);
- builder.addTask(part1);
- builder.addSourceEdge(1, part1, 0);
+ builder.addActivity(spart);
+ builder.addSourceEdge(1, spart, 0);
- builder.addTask(join);
- builder.addBlockingEdge(part0, part1);
- builder.addBlockingEdge(part1, join);
+ builder.addActivity(join);
+ builder.addBlockingEdge(rpart, spart);
+ builder.addBlockingEdge(spart, join);
builder.addTargetEdge(0, join, 0);
}
@@ -125,16 +125,16 @@
private int operatorInputIndex;
private int keys[];
- public HashPartitionActivityNode(String partitionsKey, int keys[], int operatorInputIndex) {
+ public HashPartitionActivityNode(ActivityId id, String partitionsKey, int keys[], int operatorInputIndex) {
+ super(id);
this.partitionsKey = partitionsKey;
this.keys = keys;
this.operatorInputIndex = operatorInputIndex;
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
- final int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -175,7 +175,7 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = fWriters[i];
if (writer == null) {
- FileReference file = ctx.getJobletContext().createWorkspaceFile(partitionsKey);
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(partitionsKey);
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
fWriters[i] = writer;
@@ -221,20 +221,18 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return GraceHashJoinOperatorDescriptor.this;
- }
}
private class JoinActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public JoinActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, final IRecordDescriptorProvider recordDescProvider, int partition,
- final int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ final IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -314,10 +312,5 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return GraceHashJoinOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 516bb88..7c9134f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -116,14 +116,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(BUILDRELATION);
- PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(PROBERELATION);
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(new ActivityId(odId, 0), BUILDRELATION);
+ PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(new ActivityId(odId, 1), PROBERELATION);
- builder.addTask(phase1);
+ builder.addActivity(phase1);
builder.addSourceEdge(1, phase1, 0);
- builder.addTask(phase2);
+ builder.addActivity(phase2);
builder.addSourceEdge(0, phase2, 0);
builder.addBlockingEdge(phase1, phase2);
@@ -135,16 +135,14 @@
private static final long serialVersionUID = 1L;
private String relationName;
- public BuildAndPartitionActivityNode(String relationName) {
- super();
+ public BuildAndPartitionActivityNode(ActivityId id, String relationName) {
+ super(id);
this.relationName = relationName;
-
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- final int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -310,7 +308,7 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = fWriters[i];
if (writer == null) {
- FileReference file = ctx.getJobletContext().createWorkspaceFile(relationName);
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(relationName);
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
fWriters[i] = writer;
@@ -320,26 +318,20 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return HybridHashJoinOperatorDescriptor.this;
- }
}
private class PartitionAndJoinActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
private String relationName;
- public PartitionAndJoinActivityNode(String relationName) {
- super();
+ public PartitionAndJoinActivityNode(ActivityId id, String relationName) {
+ super(id);
this.relationName = relationName;
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- final int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, final int nPartitions) {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -524,7 +516,7 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = probeWriters[i];
if (writer == null) {
- FileReference file = ctx.createWorkspaceFile(relationName);
+ FileReference file = ctx.createManagedWorkspaceFile(relationName);
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
probeWriters[i] = writer;
@@ -534,10 +526,5 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return HybridHashJoinOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index d1d528c..fad12c1 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -21,7 +21,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.ITuplePartitionComputer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -43,7 +43,7 @@
private final boolean isLeftOuter;
private final ArrayTupleBuilder nullTupleBuild;
- public InMemoryHashJoin(IHyracksStageletContext ctx, int tableSize, FrameTupleAccessor accessor0,
+ public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
FrameTuplePairComparator comparator, boolean isLeftOuter, INullWriter[] nullWriters1)
throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 13af25d..ce8be1c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -82,14 +82,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- HashBuildActivityNode hba = new HashBuildActivityNode();
- HashProbeActivityNode hpa = new HashProbeActivityNode();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ HashBuildActivityNode hba = new HashBuildActivityNode(new ActivityId(odId, 0));
+ HashProbeActivityNode hpa = new HashProbeActivityNode(new ActivityId(odId, 1));
- builder.addTask(hba);
+ builder.addActivity(hba);
builder.addSourceEdge(1, hba, 0);
- builder.addTask(hpa);
+ builder.addActivity(hpa);
builder.addSourceEdge(0, hpa, 0);
builder.addTargetEdge(0, hpa, 0);
@@ -100,10 +100,13 @@
private class HashBuildActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public HashBuildActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -149,20 +152,18 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return InMemoryHashJoinOperatorDescriptor.this;
- }
}
private class HashProbeActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public HashProbeActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private InMemoryHashJoin joiner;
@@ -191,10 +192,5 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return InMemoryHashJoinOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index dd985d3..7e84229 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -19,7 +19,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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;
@@ -38,12 +38,12 @@
private final ByteBuffer innerBuffer;
private final List<ByteBuffer> outBuffers;
private final int memSize;
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private RunFileReader runFileReader;
private int currentMemSize = 0;
private final RunFileWriter runFileWriter;
- public NestedLoopJoin(IHyracksStageletContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
+ public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
ITuplePairComparator comparators, int memSize) throws HyracksDataException {
this.accessorInner = accessor1;
this.accessorOuter = accessor0;
@@ -56,7 +56,7 @@
this.memSize = memSize;
this.ctx = ctx;
- FileReference file = ctx.getJobletContext().createWorkspaceFile(
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
this.getClass().getSimpleName() + this.toString());
runFileWriter = new RunFileWriter(file, ctx.getIOManager());
runFileWriter.open();
@@ -163,4 +163,4 @@
}
return 0;
}
-}
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index efee8ff..9e69e4b 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -17,9 +17,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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.ITuplePairComparator;
@@ -51,14 +51,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- JoinCacheActivityNode jc = new JoinCacheActivityNode();
- NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ JoinCacheActivityNode jc = new JoinCacheActivityNode(new ActivityId(getOperatorId(), 0));
+ NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode(new ActivityId(getOperatorId(), 1));
- builder.addTask(jc);
+ builder.addActivity(jc);
builder.addSourceEdge(1, jc, 0);
- builder.addTask(nlj);
+ builder.addActivity(nlj);
builder.addSourceEdge(0, nlj, 0);
builder.addTargetEdge(0, nlj, 0);
@@ -68,10 +68,13 @@
private class JoinCacheActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public JoinCacheActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator();
@@ -105,20 +108,18 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return NestedLoopJoinOperatorDescriptor.this;
- }
}
private class NestedLoopJoinActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public NestedLoopJoinActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private NestedLoopJoin joiner;
@@ -148,10 +149,5 @@
};
return op;
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return NestedLoopJoinOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
index 37b5165..a142c94 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.dataflow.std.map;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -69,7 +69,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
index 4ec6393..48e53b5 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorDescriptor.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.dataflow.std.misc;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -41,7 +41,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
return new ConstantTupleSourceOperatorNodePushable(ctx, fieldSlots, tupleData, tupleSize);
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 8784d4a..34a8e5e 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -17,20 +17,20 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
public class ConstantTupleSourceOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
- private IHyracksStageletContext ctx;
+ private IHyracksTaskContext ctx;
private int[] fieldSlots;
private byte[] tupleData;
private int tupleSize;
- public ConstantTupleSourceOperatorNodePushable(IHyracksStageletContext ctx, int[] fieldSlots, byte[] tupleData,
+ public ConstantTupleSourceOperatorNodePushable(IHyracksTaskContext ctx, int[] fieldSlots, byte[] tupleData,
int tupleSize) {
super();
this.fieldSlots = fieldSlots;
@@ -46,6 +46,7 @@
appender.reset(writeBuffer, true);
if (fieldSlots != null && tupleData != null && tupleSize > 0)
appender.append(fieldSlots, tupleData, 0, tupleSize);
+ writer.open();
FrameUtils.flushFrame(writeBuffer, writer);
writer.close();
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index 23c6385..e8a6eae 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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;
@@ -43,14 +43,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- MaterializerActivityNode ma = new MaterializerActivityNode();
- ReaderActivityNode ra = new ReaderActivityNode();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ MaterializerActivityNode ma = new MaterializerActivityNode(new ActivityId(odId, 0));
+ ReaderActivityNode ra = new ReaderActivityNode(new ActivityId(odId, 1));
- builder.addTask(ma);
+ builder.addActivity(ma);
builder.addSourceEdge(0, ma, 0);
- builder.addTask(ra);
+ builder.addActivity(ra);
builder.addTargetEdge(0, ra, 0);
builder.addBlockingEdge(ma, ra);
@@ -59,16 +59,19 @@
private final class MaterializerActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public MaterializerActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new AbstractUnaryInputSinkOperatorNodePushable() {
private RunFileWriter out;
@Override
public void open() throws HyracksDataException {
- FileReference file = ctx.getJobletContext().createWorkspaceFile(
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
MaterializingOperatorDescriptor.class.getSimpleName());
out = new RunFileWriter(file, ctx.getIOManager());
out.open();
@@ -90,20 +93,18 @@
}
};
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return MaterializingOperatorDescriptor.this;
- }
}
private final class ReaderActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public ReaderActivityNode(ActivityId id) {
+ super(id);
+ }
+
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
public void initialize() throws HyracksDataException {
@@ -127,10 +128,5 @@
}
};
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return MaterializingOperatorDescriptor.this;
- }
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
index 4bc4a7f..5a4ea37 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/NullSinkOperatorDescriptor.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.exceptions.HyracksDataException;
@@ -33,7 +33,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new AbstractUnaryInputSinkOperatorNodePushable() {
@Override
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
index e44e8a9..2177409 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.dataflow.std.misc;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -58,7 +58,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index f82a7ef..ff39cc8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -3,7 +3,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -25,7 +25,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
throws HyracksDataException {
return new AbstractUnaryInputOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
index 5b1acd5..8a3addb 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -17,11 +17,10 @@
import java.util.ArrayList;
import java.util.List;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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;
@@ -37,18 +36,17 @@
private class CollectActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
+ public CollectActivity(ActivityId id) {
+ super(id);
+ }
+
@Override
- public ActivityNodeId getActivityNodeId() {
+ public ActivityId getActivityId() {
return id;
}
@Override
- public IOperatorDescriptor getOwner() {
- return SplitVectorOperatorDescriptor.this;
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
private ArrayList<Object[]> buffer;
@@ -82,13 +80,12 @@
private class SplitActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return SplitVectorOperatorDescriptor.this;
+ public SplitActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, final IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, final IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOpenableDataWriterOperator op = new IOpenableDataWriterOperator() {
private IOpenableDataWriter<Object[]> writer;
@@ -139,14 +136,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- CollectActivity ca = new CollectActivity();
- SplitActivity sa = new SplitActivity();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ CollectActivity ca = new CollectActivity(new ActivityId(odId, 0));
+ SplitActivity sa = new SplitActivity(new ActivityId(odId, 1));
- builder.addTask(ca);
+ builder.addActivity(ca);
builder.addSourceEdge(0, ca, 0);
- builder.addTask(sa);
+ builder.addActivity(sa);
builder.addTargetEdge(0, sa, 0);
builder.addBlockingEdge(ca, sa);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index fd1d96e..3bde4c4 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -17,10 +17,12 @@
import java.nio.ByteBuffer;
import java.util.List;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -28,7 +30,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -64,14 +65,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- SortActivity sa = new SortActivity();
- MergeActivity ma = new MergeActivity();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ SortActivity sa = new SortActivity(new ActivityId(odId, 0));
+ MergeActivity ma = new MergeActivity(new ActivityId(odId, 1));
- builder.addTask(sa);
+ builder.addActivity(sa);
builder.addSourceEdge(0, sa, 0);
- builder.addTask(ma);
+ builder.addActivity(ma);
builder.addTargetEdge(0, ma, 0);
builder.addBlockingEdge(sa, ma);
@@ -80,15 +81,13 @@
private class SortActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return ExternalSortOperatorDescriptor.this;
+ public SortActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final ExternalSortRunGenerator runGen = new ExternalSortRunGenerator(ctx, sortFields,
firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0], framesLimit);
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -121,22 +120,24 @@
private class MergeActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return ExternalSortOperatorDescriptor.this;
+ public MergeActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
public void initialize() throws HyracksDataException {
- List<RunFileReader> runs = (List<RunFileReader>) env.get(RUNS);
+ List<IFrameReader> runs = (List<IFrameReader>) env.get(RUNS);
FrameSorter frameSorter = (FrameSorter) env.get(FRAMESORTER);
+ IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
- comparatorFactories, recordDescriptors[0], framesLimit, writer);
+ comparators, recordDescriptors[0], framesLimit, writer);
merger.process();
env.set(FRAMESORTER, null);
env.set(RUNS, null);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 50b968a..eee6f49 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -18,28 +18,28 @@
import java.util.LinkedList;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
public class ExternalSortRunGenerator implements IFrameWriter {
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final FrameSorter frameSorter;
- private final List<RunFileReader> runs;
+ private final List<IFrameReader> runs;
private final int maxSortFrames;
- public ExternalSortRunGenerator(IHyracksStageletContext ctx, int[] sortFields,
+ public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDesc, int framesLimit) {
this.ctx = ctx;
frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc);
- runs = new LinkedList<RunFileReader>();
+ runs = new LinkedList<IFrameReader>();
maxSortFrames = framesLimit - 1;
}
@@ -70,7 +70,8 @@
private void flushFramesToRun() throws HyracksDataException {
frameSorter.sortFrames();
- FileReference file = ctx.getJobletContext().createWorkspaceFile(ExternalSortRunGenerator.class.getSimpleName());
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+ ExternalSortRunGenerator.class.getSimpleName());
RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
try {
@@ -90,7 +91,7 @@
return frameSorter;
}
- public List<RunFileReader> getRuns() {
+ public List<IFrameReader> getRuns() {
return runs;
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 19f3b7c..8b8429c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -16,29 +16,24 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.Comparator;
import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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.RecordDescriptor;
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.FrameTupleAccessor;
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.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
-import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
public class ExternalSortRunMerger {
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final FrameSorter frameSorter;
- private final List<RunFileReader> runs;
+ private final List<IFrameReader> runs;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
private final RecordDescriptor recordDesc;
@@ -48,26 +43,21 @@
private ByteBuffer outFrame;
private FrameTupleAppender outFrameAppender;
- public ExternalSortRunMerger(IHyracksStageletContext ctx, FrameSorter frameSorter, List<RunFileReader> runs,
- int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDesc,
- int framesLimit, IFrameWriter writer) {
+ public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<IFrameReader> runs,
+ int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc, int framesLimit,
+ IFrameWriter writer) {
this.ctx = ctx;
this.frameSorter = frameSorter;
this.runs = runs;
this.sortFields = sortFields;
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
+ this.comparators = comparators;
this.recordDesc = recordDesc;
this.framesLimit = framesLimit;
this.writer = writer;
}
- public void process(boolean doFinalPass) throws HyracksDataException {
- if (doFinalPass) {
- writer.open();
- }
+ public void process() throws HyracksDataException {
+ writer.open();
try {
if (runs.size() <= 0) {
if (frameSorter != null && frameSorter.getFrameCount() > 0) {
@@ -81,86 +71,48 @@
for (int i = 0; i < framesLimit - 1; ++i) {
inFrames.add(ctx.allocateFrame());
}
- int passCount = 0;
while (runs.size() > 0) {
- passCount++;
try {
- doPass(runs, passCount, doFinalPass);
+ doPass(runs);
} catch (Exception e) {
throw new HyracksDataException(e);
}
}
}
} finally {
- if (doFinalPass) {
- writer.close();
- }
+ writer.close();
}
}
- public void process() throws HyracksDataException {
- process(true);
- }
-
// creates a new run from runs that can fit in memory.
- private void doPass(List<RunFileReader> runs, int passCount, boolean doFinalPass) throws HyracksDataException {
+ private void doPass(List<IFrameReader> runs) throws HyracksDataException {
FileReference newRun = null;
IFrameWriter writer = this.writer;
boolean finalPass = false;
if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
- if (!doFinalPass) {
- return;
- }
finalPass = true;
for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
inFrames.remove(i);
}
} else {
- newRun = ctx.createWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+ newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
writer = new RunFileWriter(newRun, ctx.getIOManager());
writer.open();
}
try {
- RunFileReader[] runCursors = new RunFileReader[inFrames.size()];
- FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
- Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
- ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc,
- inFrames.size(), comparator);
- int[] tupleIndexes = new int[inFrames.size()];
+ IFrameReader[] runCursors = new RunFileReader[inFrames.size()];
for (int i = 0; i < inFrames.size(); i++) {
- tupleIndexes[i] = 0;
- int runIndex = topTuples.peek().getRunid();
- runCursors[runIndex] = runs.get(runIndex);
- runCursors[runIndex].open();
- if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
- tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
- tupleAccessors[runIndex].reset(inFrames.get(runIndex));
- setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
- } else {
- closeRun(runIndex, runCursors, tupleAccessors);
- }
+ runCursors[i] = runs.get(i);
}
-
- while (!topTuples.areRunsExhausted()) {
- ReferenceEntry top = topTuples.peek();
- int runIndex = top.getRunid();
- FrameTupleAccessor fta = top.getAccessor();
- int tupleIndex = top.getTupleIndex();
-
- if (!outFrameAppender.append(fta, tupleIndex)) {
+ RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
+ comparators, recordDesc);
+ merger.open();
+ try {
+ while (merger.nextFrame(outFrame)) {
FrameUtils.flushFrame(outFrame, writer);
- outFrameAppender.reset(outFrame, true);
- if (!outFrameAppender.append(fta, tupleIndex)) {
- throw new IllegalStateException();
- }
}
-
- ++tupleIndexes[runIndex];
- setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
- }
- if (outFrameAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- outFrameAppender.reset(outFrame, true);
+ } finally {
+ merger.close();
}
runs.subList(0, inFrames.size()).clear();
if (!finalPass) {
@@ -172,66 +124,4 @@
}
}
}
-
- private void setNextTopTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
- FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
- boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
- if (exists) {
- topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
- } else {
- topTuples.pop();
- closeRun(runIndex, runCursors, tupleAccessors);
- }
- }
-
- private boolean hasNextTuple(int runIndex, int[] tupleIndexes, RunFileReader[] runCursors,
- FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
- if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
- return false;
- } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
- ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
- if (runCursors[runIndex].nextFrame(buf)) {
- tupleIndexes[runIndex] = 0;
- return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
- } else {
- return false;
- }
- } else {
- return true;
- }
- }
-
- private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor)
- throws HyracksDataException {
- runCursors[index].close();
- runCursors[index] = null;
- tupleAccessor[index] = null;
- }
-
- private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
- return new Comparator<ReferenceEntry>() {
- public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
- FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
- FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
- int j1 = tp1.getTupleIndex();
- int j2 = tp2.getTupleIndex();
- byte[] b1 = fta1.getBuffer().array();
- byte[] b2 = fta2.getBuffer().array();
- for (int f = 0; f < sortFields.length; ++f) {
- int fIdx = sortFields[f];
- int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
- + fta1.getFieldStartOffset(j1, fIdx);
- int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
- int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
- + fta2.getFieldStartOffset(j2, fIdx);
- int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
- int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
- if (c != 0) {
- return c;
- }
- }
- return 0;
- }
- };
- }
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
index 710ef38..8cf782e 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorter.java
@@ -121,7 +121,6 @@
}
public void flushFrames(IFrameWriter writer) throws HyracksDataException {
- writer.open();
appender.reset(outFrame, true);
for (int ptr = 0; ptr < tupleCount; ++ptr) {
int i = tPointers[ptr * 4];
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 3d41e18..d3015b7 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -16,9 +16,9 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
@@ -56,14 +56,14 @@
}
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- SortActivity sa = new SortActivity();
- MergeActivity ma = new MergeActivity();
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ SortActivity sa = new SortActivity(new ActivityId(odId, 0));
+ MergeActivity ma = new MergeActivity(new ActivityId(odId, 1));
- builder.addTask(sa);
+ builder.addActivity(sa);
builder.addSourceEdge(0, sa, 0);
- builder.addTask(ma);
+ builder.addActivity(ma);
builder.addTargetEdge(0, ma, 0);
builder.addBlockingEdge(sa, ma);
@@ -72,15 +72,13 @@
private class SortActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return InMemorySortOperatorDescriptor.this;
+ public SortActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final FrameSorter frameSorter = new FrameSorter(ctx, sortFields, firstKeyNormalizerFactory,
comparatorFactories, recordDescriptors[0]);
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
@@ -111,15 +109,13 @@
private class MergeActivity extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- @Override
- public IOperatorDescriptor getOwner() {
- return InMemorySortOperatorDescriptor.this;
+ public MergeActivity(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx,
- final IOperatorEnvironment env, IRecordDescriptorProvider recordDescProvider, int partition,
- int nPartitions) {
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
public void initialize() throws HyracksDataException {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
new file mode 100644
index 0000000..e791582
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -0,0 +1,178 @@
+/*
+ * 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.sort;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+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.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
+import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
+
+public class RunMergingFrameReader implements IFrameReader {
+ private final IHyracksTaskContext ctx;
+
+ private final IFrameReader[] runCursors;
+
+ private final List<ByteBuffer> inFrames;
+
+ private final int[] sortFields;
+
+ private final IBinaryComparator[] comparators;
+
+ private final RecordDescriptor recordDesc;
+
+ private final FrameTupleAppender outFrameAppender;
+
+ private ReferencedPriorityQueue topTuples;
+
+ private int[] tupleIndexes;
+
+ private FrameTupleAccessor[] tupleAccessors;
+
+ public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
+ int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDesc) {
+ this.ctx = ctx;
+ this.runCursors = runCursors;
+ this.inFrames = inFrames;
+ this.sortFields = sortFields;
+ this.comparators = comparators;
+ this.recordDesc = recordDesc;
+ outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ tupleAccessors = new FrameTupleAccessor[inFrames.size()];
+ Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
+ topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, inFrames.size(), comparator);
+ tupleIndexes = new int[inFrames.size()];
+ for (int i = 0; i < inFrames.size(); i++) {
+ tupleIndexes[i] = 0;
+ int runIndex = topTuples.peek().getRunid();
+ runCursors[runIndex].open();
+ if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
+ tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
+ tupleAccessors[runIndex].reset(inFrames.get(runIndex));
+ setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+ } else {
+ closeRun(runIndex, runCursors, tupleAccessors);
+ topTuples.pop();
+ }
+ }
+ }
+
+ @Override
+ public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ outFrameAppender.reset(buffer, true);
+ while (!topTuples.areRunsExhausted()) {
+ ReferenceEntry top = topTuples.peek();
+ int runIndex = top.getRunid();
+ FrameTupleAccessor fta = top.getAccessor();
+ int tupleIndex = top.getTupleIndex();
+
+ if (!outFrameAppender.append(fta, tupleIndex)) {
+ return true;
+ }
+
+ ++tupleIndexes[runIndex];
+ setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+ }
+
+ if (outFrameAppender.getTupleCount() > 0) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ for (int i = 0; i < inFrames.size(); ++i) {
+ closeRun(i, runCursors, tupleAccessors);
+ }
+ }
+
+ private void setNextTopTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+ FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
+ boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+ if (exists) {
+ topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
+ } else {
+ topTuples.pop();
+ closeRun(runIndex, runCursors, tupleAccessors);
+ }
+ }
+
+ private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
+ FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+ if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+ return false;
+ } else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
+ ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
+ if (runCursors[runIndex].nextFrame(buf)) {
+ tupleIndexes[runIndex] = 0;
+ return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+ } else {
+ return false;
+ }
+ } else {
+ return true;
+ }
+ }
+
+ private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+ throws HyracksDataException {
+ if (runCursors[index] != null) {
+ runCursors[index].close();
+ runCursors[index] = null;
+ tupleAccessors[index] = null;
+ }
+ }
+
+ private Comparator<ReferenceEntry> createEntryComparator(final IBinaryComparator[] comparators) {
+ return new Comparator<ReferenceEntry>() {
+ public int compare(ReferenceEntry tp1, ReferenceEntry tp2) {
+ FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
+ FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+ int j1 = tp1.getTupleIndex();
+ int j2 = tp2.getTupleIndex();
+ byte[] b1 = fta1.getBuffer().array();
+ byte[] b2 = fta2.getBuffer().array();
+ for (int f = 0; f < sortFields.length; ++f) {
+ int fIdx = sortFields[f];
+ int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
+ + fta1.getFieldStartOffset(j1, fIdx);
+ int l1 = fta1.getFieldEndOffset(j1, fIdx) - fta1.getFieldStartOffset(j1, fIdx);
+ int s2 = fta2.getTupleStartOffset(j2) + fta2.getFieldSlotsLength()
+ + fta2.getFieldStartOffset(j2, fIdx);
+ int l2 = fta2.getFieldEndOffset(j2, fIdx) - fta2.getFieldStartOffset(j2, fIdx);
+ int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+ if (c != 0) {
+ return c;
+ }
+ }
+ return 0;
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 9e8cf00..0acafee 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -3,7 +3,7 @@
import java.util.ArrayList;
import java.util.List;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
/**
* An entry in the table is: #elements, #no-empty elements; fIndex, tIndex;
@@ -17,14 +17,14 @@
private IntSerDeBuffer[] headers;
private List<IntSerDeBuffer> contents = new ArrayList<IntSerDeBuffer>();
private List<Integer> frameCurrentIndex = new ArrayList<Integer>();
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private int frameCapacity = 0;
private int currentLargestFrameIndex = 0;
private int tupleCount = 0;
private int headerFrameCount = 0;
private TuplePointer tempTuplePointer = new TuplePointer();
- public SerializableHashTable(int tableSize, final IHyracksStageletContext ctx) {
+ public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) {
this.ctx = ctx;
int frameSize = ctx.getFrameSize();
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index eccb945..f5ecde6 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -17,9 +17,9 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
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;
@@ -39,9 +39,9 @@
private static final long serialVersionUID = 1L;
@Override
- public void contributeTaskGraph(IActivityGraphBuilder builder) {
- UnionActivityNode uba = new UnionActivityNode();
- builder.addTask(uba);
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ UnionActivityNode uba = new UnionActivityNode(new ActivityId(getOperatorId(), 0));
+ builder.addActivity(uba);
for (int i = 0; i < inputArity; ++i) {
builder.addSourceEdge(i, uba, i);
}
@@ -51,22 +51,17 @@
private class UnionActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public UnionActivityNode() {
+ public UnionActivityNode(ActivityId id) {
+ super(id);
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
throws HyracksDataException {
RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
return new UnionOperator(ctx, inRecordDesc);
}
-
- @Override
- public IOperatorDescriptor getOwner() {
- return UnionAllOperatorDescriptor.this;
- }
-
}
private class UnionOperator extends AbstractUnaryOutputOperatorNodePushable {
@@ -74,7 +69,7 @@
private int nClosed;
- public UnionOperator(IHyracksStageletContext ctx, RecordDescriptor inRecordDesc) {
+ public UnionOperator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc) {
nOpened = 0;
nClosed = 0;
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index 6660a60..b2c6feb 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -17,7 +17,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.FrameDeserializer;
@@ -26,13 +26,13 @@
import edu.uci.ics.hyracks.dataflow.std.base.IOpenableDataWriterOperator;
public final class DeserializedOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable {
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final IOpenableDataWriterOperator delegate;
private final FrameDeserializer deserializer;
- public DeserializedOperatorNodePushable(IHyracksStageletContext ctx, IOpenableDataWriterOperator delegate,
+ public DeserializedOperatorNodePushable(IHyracksTaskContext ctx, IOpenableDataWriterOperator delegate,
RecordDescriptor inRecordDesc) {
this.ctx = ctx;
this.delegate = delegate;
diff --git a/hyracks-documentation/.classpath b/hyracks-documentation/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-documentation/.classpath
+++ b/hyracks-documentation/.classpath
@@ -1,6 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
- <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
</classpath>
diff --git a/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs b/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
index 75a3d16..f362c73 100644
--- a/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-documentation/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:30 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/btree-example/btreeapp/.classpath b/hyracks-examples/btree-example/btreeapp/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-examples/btree-example/btreeapp/.classpath
+++ b/hyracks-examples/btree-example/btreeapp/.classpath
@@ -1,6 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
- <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
</classpath>
diff --git a/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs b/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
index 75a3d16..f362c73 100644
--- a/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-examples/btree-example/btreeapp/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:30 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 7aeaf77..6f371c0 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -15,8 +15,6 @@
package edu.uci.ics.hyracks.examples.btree.client;
-import java.util.UUID;
-
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -30,6 +28,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -37,7 +36,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
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.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
@@ -92,7 +91,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
@@ -202,7 +201,7 @@
// ops
IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
- IConnectorDescriptor hashConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
// connect the ops
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 1533714..f7942eb 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -14,8 +14,6 @@
*/
package edu.uci.ics.hyracks.examples.btree.client;
-import java.util.UUID;
-
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -29,13 +27,14 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
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.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
@@ -89,7 +88,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
@@ -167,7 +166,7 @@
// ops
IBinaryHashFunctionFactory[] hashFactories = new IBinaryHashFunctionFactory[1];
hashFactories[0] = UTF8StringBinaryHashFunctionFactory.INSTANCE;
- IConnectorDescriptor hashConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 }, hashFactories));
spec.connect(hashConn, dataGen, 0, sorter, 0);
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
index c991a89..cae9660 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexEnlistFilesExample.java
@@ -14,8 +14,6 @@
*/
package edu.uci.ics.hyracks.examples.btree.client;
-import java.util.UUID;
-
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -26,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -74,7 +73,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 7e4ae9d..fbf7f9d 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.examples.btree.client;
import java.io.DataOutput;
-import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -28,6 +27,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
@@ -81,7 +81,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index b061811..47fecad 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -14,8 +14,6 @@
*/
package edu.uci.ics.hyracks.examples.btree.client;
-import java.util.UUID;
-
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -26,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -85,7 +84,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index 5e5eaeb..5eec60c 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.examples.btree.client;
import java.io.DataOutput;
-import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -28,6 +27,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.TypeTrait;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
@@ -84,7 +84,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index 0c72503..357760a 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -20,7 +20,7 @@
import java.util.HashSet;
import java.util.Random;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -59,7 +59,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
final ByteBuffer outputFrame = ctx.allocateFrame();
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
index 9fd7df4..4a452a8 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/RuntimeContext.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.examples.btree.helper;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
@@ -58,7 +58,7 @@
return treeIndexRegistry;
}
- public static RuntimeContext get(IHyracksStageletContext ctx) {
+ public static RuntimeContext get(IHyracksTaskContext ctx) {
return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
}
}
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
index e1e47af..4c6363b 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/StorageManagerInterface.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.examples.btree.helper;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -29,12 +29,12 @@
}
@Override
- public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+ public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
return RuntimeContext.get(ctx).getBufferCache();
}
@Override
- public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+ public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
return RuntimeContext.get(ctx).getFileMapManager();
}
}
\ No newline at end of file
diff --git a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
index 6a6ddf8..f8a7bf1 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/TreeIndexRegistryProvider.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.examples.btree.helper;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
@@ -29,7 +29,7 @@
}
@Override
- public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
+ public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
return RuntimeContext.get(ctx).getTreeIndexRegistry();
}
}
\ No newline at end of file
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index f13a701..99204d8 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -19,16 +19,12 @@
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
-import java.util.UUID;
import org.junit.Test;
-import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
@@ -40,25 +36,19 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.SerializingDataWriter;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.test.support.TestJobletContext;
-import edu.uci.ics.hyracks.test.support.TestNCApplicationContext;
-import edu.uci.ics.hyracks.test.support.TestRootContext;
-import edu.uci.ics.hyracks.test.support.TestStageletContext;
+import edu.uci.ics.hyracks.test.support.TestUtils;
public class SerializationDeserializationTest {
private static final String DBLP_FILE = "data/dblp.txt";
private static class SerDeserRunner {
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private static final int FRAME_SIZE = 32768;
private RecordDescriptor rDes;
private List<ByteBuffer> buffers;
public SerDeserRunner(RecordDescriptor rDes) throws HyracksException {
- IHyracksRootContext rootCtx = new TestRootContext(FRAME_SIZE);
- INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx, null);
- IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
- ctx = new TestStageletContext(jobletCtx, UUID.randomUUID());
+ ctx = TestUtils.create(FRAME_SIZE);
this.rDes = rDes;
buffers = new ArrayList<ByteBuffer>();
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 2bb1d80..237ab52 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -15,18 +15,18 @@
package edu.uci.ics.hyracks.tests.integration;
import java.util.EnumSet;
-import java.util.UUID;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import edu.uci.ics.hyracks.api.client.HyracksLocalConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.control.CCConfig;
-import edu.uci.ics.hyracks.api.control.NCConfig;
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 abstract class AbstractIntegrationTest {
@@ -42,7 +42,7 @@
public static void init() throws Exception {
CCConfig ccConfig = new CCConfig();
ccConfig.port = 39001;
- ccConfig.profileDumpPeriod = 1000;
+ ccConfig.profileDumpPeriod = 10000;
cc = new ClusterControllerService(ccConfig);
cc.start();
@@ -74,8 +74,8 @@
}
protected void runTest(JobSpecification spec) throws Exception {
- UUID jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
- System.err.println(spec.toJSON());
+ JobId jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ System.err.println(spec.toJSON().toString(2));
hcc.start(jobId);
System.err.print(jobId);
cc.waitForCompletion(jobId);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index 8468b54..0a0df60 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -37,7 +37,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -95,7 +95,7 @@
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -103,7 +103,7 @@
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, sorter, 0, group, 0);
- IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn3, group, 0, sorter2, 0);
@@ -162,7 +162,7 @@
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -170,7 +170,7 @@
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, sorter, 0, group, 0);
- IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn3, group, 0, sorter2, 0);
@@ -229,7 +229,7 @@
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, sorter, 0);
@@ -237,7 +237,7 @@
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, sorter, 0, group, 0);
- IConnectorDescriptor conn3 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn3 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn3, group, 0, sorter2, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index f0974d9..d343232 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -33,7 +33,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -96,7 +96,7 @@
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, ordScanner, 0, printer, 0);
@@ -130,7 +130,7 @@
PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { IntegerBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, ordScanner, 0, printer, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index b0c33ae..1b6ff14 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -31,7 +31,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningMergingConnectorDescriptor;
+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.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -75,7 +75,7 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new MToNHashPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
new int[] { 1 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }),
new int[] { 1 }, new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE }),
sorter, 0, printer, 0);
@@ -116,7 +116,7 @@
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
- spec.connect(new MToNHashPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
+ spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
new int[] { 1, 0 }, new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE,
UTF8StringBinaryHashFunctionFactory.INSTANCE }), new int[] { 1, 0 },
new IBinaryComparatorFactory[] { UTF8StringBinaryComparatorFactory.INSTANCE,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index a8db325..d417fda 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -38,7 +38,7 @@
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
@@ -621,12 +621,12 @@
: new NullSinkOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -700,12 +700,12 @@
: new NullSinkOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -779,12 +779,12 @@
: new NullSinkOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -857,12 +857,12 @@
: new NullSinkOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
@@ -941,12 +941,12 @@
: new NullSinkOperatorDescriptor(spec);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
- IConnectorDescriptor ordPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordPartConn, ordScanner, 0, ordMat, 0);
- IConnectorDescriptor custPartConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custPartConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custPartConn, custScanner, 0, custMat, 0);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
index b107282..2f66bfd 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/spillable/ExternalAggregateTest.java
@@ -44,7 +44,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.IntSumAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -144,7 +144,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -188,7 +188,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -234,7 +234,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -276,7 +276,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -325,7 +325,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE, }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -367,7 +367,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index b02b0ee..68e33cb 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.examples.text.client;
import java.io.File;
-import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -30,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.IntegerBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.hash.IntegerBinaryHashFunctionFactory;
@@ -51,7 +51,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -122,7 +122,7 @@
System.out.print(i + "\t" + (System.currentTimeMillis() - start));
start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
System.out.println("\t" + (System.currentTimeMillis() - start));
@@ -221,7 +221,7 @@
createPartitionConstraint(spec, grouper, outSplits);
// Connect scanner with the grouper
- IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
// IntegerBinaryHashFunctionFactory.INSTANCE,
IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -235,7 +235,7 @@
createPartitionConstraint(spec, sorter, inSplits);
// Connect scan operator with the sorter
- IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
// IntegerBinaryHashFunctionFactory.INSTANCE,
IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -265,7 +265,7 @@
createPartitionConstraint(spec, grouper, outSplits);
// Connect scanner with the grouper
- IConnectorDescriptor scanConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
// IntegerBinaryHashFunctionFactory.INSTANCE,
IntegerBinaryHashFunctionFactory.INSTANCE }));
@@ -292,11 +292,11 @@
createPartitionConstraint(spec, grouper, outSplits);
// Connect scanner with the grouper
- IConnectorDescriptor defaultGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
// IntegerBinaryHashFunctionFactory.INSTANCE,
IntegerBinaryHashFunctionFactory.INSTANCE }));
- spec.connect(defaultGroupConn, fileScanner, 0, grouper, 0);
+ spec.connect(scanGroupConnDef, fileScanner, 0, grouper, 0);
}
IFileSplitProvider outSplitProvider = new ConstantFileSplitProvider(outSplits);
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index a0f24c4..51d3661 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.examples.text.client;
import java.io.File;
-import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -30,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -40,7 +40,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
@@ -95,7 +95,7 @@
options.algo, options.htSize, options.sbSize, options.format);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job);
+ JobId jobId = hcc.createJob(options.app, job);
hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
@@ -141,7 +141,7 @@
new IFieldValueResultingAggregatorFactory[] { new CountAggregatorFactory() }),
groupResultDesc, htSize);
createPartitionConstraint(spec, gBy, outSplits);
- IConnectorDescriptor scanGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanGroupConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(scanGroupConn, wordScanner, 0, gBy, 0);
@@ -153,7 +153,7 @@
new UTF8StringNormalizedKeyComputerFactory(), cfs, wordDesc);
createPartitionConstraint(spec, sorter, outSplits);
- IConnectorDescriptor scanSortConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanSortConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(scanSortConn, wordScanner, 0, sorter, 0);
diff --git a/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index 4d7cc1f..249e3bb 100644
--- a/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -9,7 +9,7 @@
import java.util.Arrays;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -23,7 +23,7 @@
private static final long serialVersionUID = 1L;
@Override
- public ITupleParser createTupleParser(final IHyracksStageletContext ctx) {
+ public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
return new ITupleParser() {
@Override
public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
diff --git a/hyracks-examples/tpch-example/tpchapp/.classpath b/hyracks-examples/tpch-example/tpchapp/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-examples/tpch-example/tpchapp/.classpath
+++ b/hyracks-examples/tpch-example/tpchapp/.classpath
@@ -1,6 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
- <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
</classpath>
diff --git a/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs b/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
index b61b886..f362c73 100644
--- a/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-examples/tpch-example/tpchapp/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:35 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index b1a1224..09d0e5a 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -16,7 +16,6 @@
import java.io.File;
import java.util.EnumSet;
-import java.util.UUID;
import org.kohsuke.args4j.CmdLineParser;
import org.kohsuke.args4j.Option;
@@ -32,6 +31,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.io.FileReference;
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.dataflow.common.data.comparators.UTF8StringBinaryComparatorFactory;
import edu.uci.ics.hyracks.dataflow.common.data.hash.UTF8StringBinaryHashFunctionFactory;
@@ -43,7 +43,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.CountAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.IFieldValueResultingAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -93,7 +93,7 @@
options.numJoinPartitions);
long start = System.currentTimeMillis();
- UUID jobId = hcc.createJob(options.app, job,
+ JobId jobId = hcc.createJob(options.app, job,
options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
hcc.start(jobId);
hcc.waitForCompletion(jobId);
@@ -184,17 +184,17 @@
FrameFileWriterOperatorDescriptor writer = new FrameFileWriterOperatorDescriptor(spec, outSplitProvider);
createPartitionConstraint(spec, writer, resultSplits);
- IConnectorDescriptor ordJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 1 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(ordJoinConn, ordScanner, 0, join, 1);
- IConnectorDescriptor custJoinConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor custJoinConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 0 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(custJoinConn, custScanner, 0, join, 0);
- IConnectorDescriptor joinGroupConn = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor joinGroupConn = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(new int[] { 6 },
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(joinGroupConn, join, 0, gby, 0);
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index a363221..9e76a3e 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -4,10 +4,10 @@
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
-import java.util.UUID;
import edu.uci.ics.hyracks.api.client.HyracksRMIConnection;
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.job.JobStatus;
import edu.uci.ics.hyracks.hadoop.compat.util.ConfigurationConstants;
@@ -15,77 +15,73 @@
public class HyracksClient {
- private static HyracksRMIConnection connection;
- private static final String jobProfilingKey = "jobProfilingKey";
- Set<String> systemLibs;
+ private static HyracksRMIConnection connection;
+ private static final String jobProfilingKey = "jobProfilingKey";
+ Set<String> systemLibs;
- public HyracksClient(Properties clusterProperties) throws Exception {
- initialize(clusterProperties);
- }
+ public HyracksClient(Properties clusterProperties) throws Exception {
+ initialize(clusterProperties);
+ }
- private void initialize(Properties properties) throws Exception {
- String clusterController = (String) properties
- .get(ConfigurationConstants.clusterControllerHost);
- connection = new HyracksRMIConnection(clusterController, 1099);
- systemLibs = new HashSet<String>();
- for (String systemLib : ConfigurationConstants.systemLibs) {
- String systemLibPath = properties.getProperty(systemLib);
- if (systemLibPath != null) {
- systemLibs.add(systemLibPath);
- }
- }
- }
+ private void initialize(Properties properties) throws Exception {
+ String clusterController = (String) properties.get(ConfigurationConstants.clusterControllerHost);
+ connection = new HyracksRMIConnection(clusterController, 1099);
+ systemLibs = new HashSet<String>();
+ for (String systemLib : ConfigurationConstants.systemLibs) {
+ String systemLibPath = properties.getProperty(systemLib);
+ if (systemLibPath != null) {
+ systemLibs.add(systemLibPath);
+ }
+ }
+ }
- public HyracksClient(String clusterConf, char delimiter) throws Exception {
- Properties properties = Utilities.getProperties(clusterConf, delimiter);
- initialize(properties);
- }
+ public HyracksClient(String clusterConf, char delimiter) throws Exception {
+ Properties properties = Utilities.getProperties(clusterConf, delimiter);
+ initialize(properties);
+ }
- private Set<String> getRequiredLibs(Set<String> userLibs) {
- Set<String> requiredLibs = new HashSet<String>();
- for (String systemLib : systemLibs) {
- requiredLibs.add(systemLib);
- }
- for (String userLib : userLibs) {
- requiredLibs.add(userLib);
- }
- return requiredLibs;
- }
+ private Set<String> getRequiredLibs(Set<String> userLibs) {
+ Set<String> requiredLibs = new HashSet<String>();
+ for (String systemLib : systemLibs) {
+ requiredLibs.add(systemLib);
+ }
+ for (String userLib : userLibs) {
+ requiredLibs.add(userLib);
+ }
+ return requiredLibs;
+ }
- public JobStatus getJobStatus(UUID jobId) throws Exception {
- return connection.getJobStatus(jobId);
- }
+ public JobStatus getJobStatus(JobId jobId) throws Exception {
+ return connection.getJobStatus(jobId);
+ }
- private void createApplication(String applicationName, Set<String> userLibs)
- throws Exception {
- connection.createApplication(applicationName, Utilities
- .getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
- }
+ private void createApplication(String applicationName, Set<String> userLibs) throws Exception {
+ connection.createApplication(applicationName,
+ Utilities.getHyracksArchive(applicationName, getRequiredLibs(userLibs)));
+ }
- public HyracksRunningJob submitJob(String applicationName,
- JobSpecification spec) throws Exception {
- String jobProfilingVal = System.getenv(jobProfilingKey);
- boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
- UUID jobId;
- if (doProfiling) {
- System.out.println("PROFILING");
- jobId = connection.createJob(applicationName, spec, EnumSet
- .of(JobFlag.PROFILE_RUNTIME));
- } else {
- jobId = connection.createJob(applicationName, spec);
- }
- connection.start(jobId);
- HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
- return runningJob;
- }
+ public HyracksRunningJob submitJob(String applicationName, JobSpecification spec) throws Exception {
+ String jobProfilingVal = System.getenv(jobProfilingKey);
+ boolean doProfiling = ("true".equalsIgnoreCase(jobProfilingVal));
+ JobId jobId;
+ if (doProfiling) {
+ System.out.println("PROFILING");
+ jobId = connection.createJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ } else {
+ jobId = connection.createJob(applicationName, spec);
+ }
+ connection.start(jobId);
+ HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
+ return runningJob;
+ }
- public HyracksRunningJob submitJob(String applicationName,
- JobSpecification spec, Set<String> userLibs) throws Exception {
- createApplication(applicationName, userLibs);
- return submitJob(applicationName, spec);
- }
+ public HyracksRunningJob submitJob(String applicationName, JobSpecification spec, Set<String> userLibs)
+ throws Exception {
+ createApplication(applicationName, userLibs);
+ return submitJob(applicationName, spec);
+ }
- public void waitForCompleton(UUID jobId) throws Exception {
- connection.waitForCompletion(jobId);
- }
+ public void waitForCompleton(JobId jobId) throws Exception {
+ connection.waitForCompletion(jobId);
+ }
}
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
index 8470e12..7c650ec 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksRunningJob.java
@@ -1,7 +1,6 @@
package edu.uci.ics.hyracks.hadoop.compat.client;
import java.io.IOException;
-import java.util.UUID;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.JobID;
@@ -9,19 +8,20 @@
import org.apache.hadoop.mapred.TaskAttemptID;
import org.apache.hadoop.mapred.TaskCompletionEvent;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public class HyracksRunningJob implements RunningJob {
- UUID jobId;
+ JobId jobId;
JobSpecification spec;
HyracksClient hyracksClient;
- public UUID getJobId() {
+ public JobId getJobId() {
return jobId;
}
- public void setJobId(UUID jobId) {
+ public void setJobId(JobId jobId) {
this.jobId = jobId;
}
@@ -33,7 +33,7 @@
this.spec = spec;
}
- public HyracksRunningJob(UUID jobId, JobSpecification jobSpec, HyracksClient hyracksClient) {
+ public HyracksRunningJob(JobId jobId, JobSpecification jobSpec, HyracksClient hyracksClient) {
this.spec = jobSpec;
this.jobId = jobId;
this.hyracksClient = hyracksClient;
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
index 37f4d34..bf09bb0 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/driver/CompatibilityLayer.java
@@ -6,15 +6,16 @@
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
-import java.util.UUID;
-import java.util.Map.Entry;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.kohsuke.args4j.CmdLineParser;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.hadoop.compat.client.HyracksClient;
import edu.uci.ics.hyracks.hadoop.compat.client.HyracksRunningJob;
import edu.uci.ics.hyracks.hadoop.compat.util.CompatibilityConfig;
@@ -22,187 +23,168 @@
import edu.uci.ics.hyracks.hadoop.compat.util.DCacheHandler;
import edu.uci.ics.hyracks.hadoop.compat.util.HadoopAdapter;
import edu.uci.ics.hyracks.hadoop.compat.util.Utilities;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
public class CompatibilityLayer {
- HyracksClient hyracksClient;
- DCacheHandler dCacheHander = null;
- Properties clusterConf;
- HadoopAdapter hadoopAdapter;
+ HyracksClient hyracksClient;
+ DCacheHandler dCacheHander = null;
+ Properties clusterConf;
+ HadoopAdapter hadoopAdapter;
- private static char configurationFileDelimiter = '=';
- private static final String dacheKeyPrefix = "dcache.key";
+ private static char configurationFileDelimiter = '=';
+ private static final String dacheKeyPrefix = "dcache.key";
- public CompatibilityLayer(CompatibilityConfig clConfig) throws Exception {
- initialize(clConfig);
- }
+ public CompatibilityLayer(CompatibilityConfig clConfig) throws Exception {
+ initialize(clConfig);
+ }
- private void initialize(CompatibilityConfig clConfig) throws Exception {
- clusterConf = Utilities.getProperties(clConfig.clusterConf,
- configurationFileDelimiter);
- hadoopAdapter = new HadoopAdapter(clusterConf
- .getProperty(ConfigurationConstants.namenodeURL));
- hyracksClient = new HyracksClient(clusterConf);
- dCacheHander = new DCacheHandler(clusterConf
- .getProperty(ConfigurationConstants.dcacheServerConfiguration));
- }
+ private void initialize(CompatibilityConfig clConfig) throws Exception {
+ clusterConf = Utilities.getProperties(clConfig.clusterConf, configurationFileDelimiter);
+ hadoopAdapter = new HadoopAdapter(clusterConf.getProperty(ConfigurationConstants.namenodeURL));
+ hyracksClient = new HyracksClient(clusterConf);
+ dCacheHander = new DCacheHandler(clusterConf.getProperty(ConfigurationConstants.dcacheServerConfiguration));
+ }
- public HyracksRunningJob submitJob(JobConf conf,Set<String> userLibs) throws Exception {
- List<JobConf> jobConfs = new ArrayList<JobConf>();
- jobConfs.add(conf);
- String applicationName = conf.getJobName() + System.currentTimeMillis();
- JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
- HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
- applicationName, spec, userLibs);
- return hyracksRunningJob;
- }
-
- public HyracksRunningJob submitJobs(String applicationName,
- String[] jobFiles, Set<String> userLibs) throws Exception {
- List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
- populateDCache(jobFiles[0]);
- JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
- HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
- applicationName, spec, userLibs);
- return hyracksRunningJob;
- }
+ public HyracksRunningJob submitJob(JobConf conf, Set<String> userLibs) throws Exception {
+ List<JobConf> jobConfs = new ArrayList<JobConf>();
+ jobConfs.add(conf);
+ String applicationName = conf.getJobName() + System.currentTimeMillis();
+ JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+ HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec, userLibs);
+ return hyracksRunningJob;
+ }
- public HyracksRunningJob submitJobs(String applicationName,
- String[] jobFiles) throws Exception {
- List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
- populateDCache(jobFiles[0]);
- JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
- HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(
- applicationName, spec);
- return hyracksRunningJob;
- }
+ public HyracksRunningJob submitJobs(String applicationName, String[] jobFiles, Set<String> userLibs)
+ throws Exception {
+ List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
+ populateDCache(jobFiles[0]);
+ JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+ HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec, userLibs);
+ return hyracksRunningJob;
+ }
- private void populateDCache(String jobFile) throws IOException {
- Map<String, String> dcacheTasks = preparePreLaunchDCacheTasks(jobFile);
- String tempDir = "/tmp";
- if (dcacheTasks.size() > 0) {
- for (String key : dcacheTasks.keySet()) {
- String destPath = tempDir + "/" + key
- + System.currentTimeMillis();
- hadoopAdapter.getHDFSClient().copyToLocalFile(
- new Path(dcacheTasks.get(key)), new Path(destPath));
- System.out.println(" source :" + dcacheTasks.get(key));
- System.out.println(" dest :" + destPath);
- System.out.println(" key :" + key);
- System.out.println(" value :" + destPath);
- dCacheHander.put(key, destPath);
- }
- }
- }
+ public HyracksRunningJob submitJobs(String applicationName, String[] jobFiles) throws Exception {
+ List<JobConf> jobConfs = constructHadoopJobConfs(jobFiles);
+ populateDCache(jobFiles[0]);
+ JobSpecification spec = hadoopAdapter.getJobSpecification(jobConfs);
+ HyracksRunningJob hyracksRunningJob = hyracksClient.submitJob(applicationName, spec);
+ return hyracksRunningJob;
+ }
- private String getApplicationNameForHadoopJob(JobConf jobConf) {
- String jar = jobConf.getJar();
- if (jar != null) {
- return jar.substring(jar.lastIndexOf("/") >= 0 ? jar
- .lastIndexOf("/") + 1 : 0);
- } else {
- return "" + System.currentTimeMillis();
- }
- }
+ private void populateDCache(String jobFile) throws IOException {
+ Map<String, String> dcacheTasks = preparePreLaunchDCacheTasks(jobFile);
+ String tempDir = "/tmp";
+ if (dcacheTasks.size() > 0) {
+ for (String key : dcacheTasks.keySet()) {
+ String destPath = tempDir + "/" + key + System.currentTimeMillis();
+ hadoopAdapter.getHDFSClient().copyToLocalFile(new Path(dcacheTasks.get(key)), new Path(destPath));
+ System.out.println(" source :" + dcacheTasks.get(key));
+ System.out.println(" dest :" + destPath);
+ System.out.println(" key :" + key);
+ System.out.println(" value :" + destPath);
+ dCacheHander.put(key, destPath);
+ }
+ }
+ }
- private Map<String, String> initializeCustomProperties(
- Properties properties, String prefix) {
- Map<String, String> foundProperties = new HashMap<String, String>();
- Set<Entry<Object, Object>> entrySet = properties.entrySet();
- for (Entry entry : entrySet) {
- String key = (String) entry.getKey();
- String value = (String) entry.getValue();
- if ((key.startsWith(prefix))) {
- String actualKey = key.substring(prefix.length() + 1); // "cut off '<prefix>.' from the beginning"
- foundProperties.put(actualKey, value);
- }
- }
- return foundProperties;
- }
+ private String getApplicationNameForHadoopJob(JobConf jobConf) {
+ String jar = jobConf.getJar();
+ if (jar != null) {
+ return jar.substring(jar.lastIndexOf("/") >= 0 ? jar.lastIndexOf("/") + 1 : 0);
+ } else {
+ return "" + System.currentTimeMillis();
+ }
+ }
- public Map<String, String> preparePreLaunchDCacheTasks(String jobFile) {
- Properties jobProperties = Utilities.getProperties(jobFile, ',');
- Map<String, String> dcacheTasks = new HashMap<String, String>();
- Map<String, String> dcacheKeys = initializeCustomProperties(
- jobProperties, dacheKeyPrefix);
- for (String key : dcacheKeys.keySet()) {
- String sourcePath = dcacheKeys.get(key);
- if (sourcePath != null) {
- dcacheTasks.put(key, sourcePath);
- }
- }
- return dcacheTasks;
- }
+ private Map<String, String> initializeCustomProperties(Properties properties, String prefix) {
+ Map<String, String> foundProperties = new HashMap<String, String>();
+ Set<Entry<Object, Object>> entrySet = properties.entrySet();
+ for (Entry entry : entrySet) {
+ String key = (String) entry.getKey();
+ String value = (String) entry.getValue();
+ if ((key.startsWith(prefix))) {
+ String actualKey = key.substring(prefix.length() + 1); // "cut off '<prefix>.' from the beginning"
+ foundProperties.put(actualKey, value);
+ }
+ }
+ return foundProperties;
+ }
- public void waitForCompletion(UUID jobId) throws Exception {
- hyracksClient.waitForCompleton(jobId);
- }
+ public Map<String, String> preparePreLaunchDCacheTasks(String jobFile) {
+ Properties jobProperties = Utilities.getProperties(jobFile, ',');
+ Map<String, String> dcacheTasks = new HashMap<String, String>();
+ Map<String, String> dcacheKeys = initializeCustomProperties(jobProperties, dacheKeyPrefix);
+ for (String key : dcacheKeys.keySet()) {
+ String sourcePath = dcacheKeys.get(key);
+ if (sourcePath != null) {
+ dcacheTasks.put(key, sourcePath);
+ }
+ }
+ return dcacheTasks;
+ }
- private List<JobConf> constructHadoopJobConfs(String[] jobFiles)
- throws Exception {
- List<JobConf> jobConfs = new ArrayList<JobConf>();
- for (String jobFile : jobFiles) {
- jobConfs.add(constructHadoopJobConf(jobFile));
- }
- return jobConfs;
- }
+ public void waitForCompletion(JobId jobId) throws Exception {
+ hyracksClient.waitForCompleton(jobId);
+ }
- private JobConf constructHadoopJobConf(String jobFile) {
- Properties jobProperties = Utilities.getProperties(jobFile, '=');
- JobConf conf = new JobConf(hadoopAdapter.getConf());
- for (Entry entry : jobProperties.entrySet()) {
- conf.set((String) entry.getKey(), (String) entry.getValue());
- System.out.println((String) entry.getKey() + " : "
- + (String) entry.getValue());
- }
- return conf;
- }
+ private List<JobConf> constructHadoopJobConfs(String[] jobFiles) throws Exception {
+ List<JobConf> jobConfs = new ArrayList<JobConf>();
+ for (String jobFile : jobFiles) {
+ jobConfs.add(constructHadoopJobConf(jobFile));
+ }
+ return jobConfs;
+ }
- private String[] getJobs(CompatibilityConfig clConfig) {
- return clConfig.jobFiles == null ? new String[0] : clConfig.jobFiles
- .split(",");
- }
+ private JobConf constructHadoopJobConf(String jobFile) {
+ Properties jobProperties = Utilities.getProperties(jobFile, '=');
+ JobConf conf = new JobConf(hadoopAdapter.getConf());
+ for (Entry entry : jobProperties.entrySet()) {
+ conf.set((String) entry.getKey(), (String) entry.getValue());
+ System.out.println((String) entry.getKey() + " : " + (String) entry.getValue());
+ }
+ return conf;
+ }
- public static void main(String args[]) throws Exception {
- long startTime = System.nanoTime();
- CompatibilityConfig clConfig = new CompatibilityConfig();
- CmdLineParser cp = new CmdLineParser(clConfig);
- try {
- cp.parseArgument(args);
- } catch (Exception e) {
- System.err.println(e.getMessage());
- cp.printUsage(System.err);
- return;
- }
- CompatibilityLayer compatLayer = new CompatibilityLayer(clConfig);
- String applicationName = clConfig.applicationName;
- String[] jobFiles = compatLayer.getJobs(clConfig);
- String[] userLibraries = null;
- if (clConfig.userLibs != null) {
- userLibraries = clConfig.userLibs.split(",");
- }
- try {
- HyracksRunningJob hyraxRunningJob = null;
- if (userLibraries != null) {
- Set<String> userLibs = new HashSet<String>();
- for (String userLib : userLibraries) {
- userLibs.add(userLib);
- }
- hyraxRunningJob = compatLayer.submitJobs(applicationName,
- jobFiles, userLibs);
- } else {
- hyraxRunningJob = compatLayer.submitJobs(applicationName,
- jobFiles);
- }
- compatLayer.waitForCompletion(hyraxRunningJob.getJobId());
- long end_time = System.nanoTime();
- System.out.println("TOTAL TIME (from Launch to Completion):"
- + ((end_time - startTime) / (float) 1000000000.0)
- + " seconds.");
- } catch (Exception e) {
- e.printStackTrace();
- throw e;
- }
- }
+ private String[] getJobs(CompatibilityConfig clConfig) {
+ return clConfig.jobFiles == null ? new String[0] : clConfig.jobFiles.split(",");
+ }
+
+ public static void main(String args[]) throws Exception {
+ long startTime = System.nanoTime();
+ CompatibilityConfig clConfig = new CompatibilityConfig();
+ CmdLineParser cp = new CmdLineParser(clConfig);
+ try {
+ cp.parseArgument(args);
+ } catch (Exception e) {
+ System.err.println(e.getMessage());
+ cp.printUsage(System.err);
+ return;
+ }
+ CompatibilityLayer compatLayer = new CompatibilityLayer(clConfig);
+ String applicationName = clConfig.applicationName;
+ String[] jobFiles = compatLayer.getJobs(clConfig);
+ String[] userLibraries = null;
+ if (clConfig.userLibs != null) {
+ userLibraries = clConfig.userLibs.split(",");
+ }
+ try {
+ HyracksRunningJob hyraxRunningJob = null;
+ if (userLibraries != null) {
+ Set<String> userLibs = new HashSet<String>();
+ for (String userLib : userLibraries) {
+ userLibs.add(userLib);
+ }
+ hyraxRunningJob = compatLayer.submitJobs(applicationName, jobFiles, userLibs);
+ } else {
+ hyraxRunningJob = compatLayer.submitJobs(applicationName, jobFiles);
+ }
+ compatLayer.waitForCompletion(hyraxRunningJob.getJobId());
+ long end_time = System.nanoTime();
+ System.out.println("TOTAL TIME (from Launch to Completion):"
+ + ((end_time - startTime) / (float) 1000000000.0) + " seconds.");
+ } catch (Exception e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
}
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
index f2f7d03..28833d7 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
@@ -40,7 +40,7 @@
import edu.uci.ics.hyracks.dataflow.hadoop.util.ClasspathBasedHadoopClassFactory;
import edu.uci.ics.hyracks.dataflow.hadoop.util.DatatypeHelper;
import edu.uci.ics.hyracks.dataflow.hadoop.util.IHadoopClassFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNHashPartitioningConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
@@ -370,13 +370,13 @@
return externalSortOp;
}
- public static MToNHashPartitioningConnectorDescriptor getMtoNHashPartitioningConnector(
+ public static MToNPartitioningConnectorDescriptor getMtoNHashPartitioningConnector(
JobConf conf, JobSpecification spec) {
Class mapOutputKeyClass = conf.getMapOutputKeyClass();
Class mapOutputValueClass = conf.getMapOutputValueClass();
- MToNHashPartitioningConnectorDescriptor connectorDescriptor = null;
+ MToNPartitioningConnectorDescriptor connectorDescriptor = null;
ITuplePartitionComputerFactory factory = null;
conf.getMapOutputKeyClass();
if (conf.getPartitionerClass() != null
@@ -398,7 +398,7 @@
factory = new HadoopHashTuplePartitionComputerFactory(
mapOutputKeySerializerDerserializer);
}
- connectorDescriptor = new MToNHashPartitioningConnectorDescriptor(spec,
+ connectorDescriptor = new MToNPartitioningConnectorDescriptor(spec,
factory);
return connectorDescriptor;
}
diff --git a/hyracks-server/.classpath b/hyracks-server/.classpath
index d0bec0f..3f62785 100644
--- a/hyracks-server/.classpath
+++ b/hyracks-server/.classpath
@@ -1,6 +1,6 @@
<?xml version="1.0" encoding="UTF-8"?>
<classpath>
- <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.5"/>
+ <classpathentry kind="con" path="org.eclipse.jdt.launching.JRE_CONTAINER/org.eclipse.jdt.internal.debug.ui.launcher.StandardVMType/J2SE-1.4"/>
<classpathentry kind="con" path="org.maven.ide.eclipse.MAVEN2_CLASSPATH_CONTAINER"/>
<classpathentry kind="output" path="target/classes"/>
</classpath>
diff --git a/hyracks-server/.settings/org.eclipse.jdt.core.prefs b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
index b61b886..f362c73 100644
--- a/hyracks-server/.settings/org.eclipse.jdt.core.prefs
+++ b/hyracks-server/.settings/org.eclipse.jdt.core.prefs
@@ -1,6 +1,6 @@
-#Thu Aug 04 11:50:35 PDT 2011
+#Sun Aug 14 10:18:14 PDT 2011
eclipse.preferences.version=1
-org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.5
-org.eclipse.jdt.core.compiler.compliance=1.5
+org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.4
+org.eclipse.jdt.core.compiler.compliance=1.4
org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning
-org.eclipse.jdt.core.compiler.source=1.5
+org.eclipse.jdt.core.compiler.source=1.4
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
index c85c6a0..9b8d4a2 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelper.java
@@ -1,6 +1,6 @@
package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
@@ -15,18 +15,18 @@
public class BTreeOpHelper extends TreeIndexOpHelper {
- public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksStageletContext ctx, int partition,
+ public BTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx, int partition,
IndexHelperOpenMode mode) {
super(opDesc, ctx, partition, mode);
}
-
+
public ITreeIndex createTreeIndex() throws HyracksDataException {
IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0, metaDataFrameFactory);
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, indexFileId, 0,
+ metaDataFrameFactory);
return new BTree(bufferCache, freePageManager, opDesc.getTreeIndexInteriorFactory(),
opDesc.getTreeIndexLeafFactory(), cmp);
}
-
-
-}
+
+}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
index 7802b81..6cc79ab 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeOpHelperFactory.java
@@ -1,15 +1,10 @@
package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexOpHelper;
-import edu.uci.ics.hyracks.storage.am.common.impls.TreeDiskOrderScanCursor;
public class BTreeOpHelperFactory implements ITreeIndexOpHelperFactory {
@@ -17,7 +12,7 @@
@Override
public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
- IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode) {
+ IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
return new BTreeOpHelper(opDesc, ctx, partition, mode);
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 254935b..717c99e 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.btree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -58,7 +58,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, final IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new BTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, isForward, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 9d33bb2..bfb8ed6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -17,7 +17,7 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -31,8 +31,8 @@
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeOpContext;
-import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
+import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
@@ -66,10 +66,11 @@
private RecordDescriptor recDesc;
- public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+ public BTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
- treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition, IndexHelperOpenMode.OPEN);
+ treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
+ IndexHelperOpenMode.OPEN);
this.isForward = isForward;
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
@@ -88,15 +89,15 @@
public void open() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
.getOperatorDescriptor();
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
cursorFrame = opDesc.getTreeIndexLeafFactory().createFrame();
- cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame)cursorFrame);
+ cursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) cursorFrame);
+ writer.open();
try {
-
treeIndexOpHelper.init();
- btree = (BTree)treeIndexOpHelper.getTreeIndex();
+ btree = (BTree) treeIndexOpHelper.getTreeIndex();
// construct range predicate
@@ -128,19 +129,20 @@
rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
highKeySearchCmp);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
- writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+ writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
tb = new ArrayTupleBuilder(btree.getMultiComparator().getFieldCount());
dos = tb.getDataOutput();
- appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
+ appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
- opCtx = btree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(), treeIndexOpHelper
- .getInteriorFrame(), null);
+ opCtx = btree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(),
+ treeIndexOpHelper.getInteriorFrame(), null);
} catch (Exception e) {
treeIndexOpHelper.deinit();
+ throw new HyracksDataException(e);
}
}
diff --git a/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs b/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
index 7b5e618..34ccda1 100644
--- a/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
+++ b/hyracks-storage-am-common/.settings/org.maven.ide.eclipse.prefs
@@ -1,7 +1,8 @@
-#Thu Jul 07 12:23:53 PDT 2011
+#Mon Aug 15 10:50:16 PDT 2011
activeProfiles=
eclipse.preferences.version=1
fullBuildGoals=process-test-resources
+includeModules=false
resolveWorkspaceProjects=true
resourceFilterGoals=process-resources resources\:testResources
skipCompilerPlugin=true
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
index aadcaf9..7971a62 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexRegistryProvider.java
@@ -17,8 +17,8 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
public interface IIndexRegistryProvider<IndexType> extends Serializable {
- public IndexRegistry<IndexType> getRegistry(IHyracksStageletContext ctx);
+ public IndexRegistry<IndexType> getRegistry(IHyracksTaskContext ctx);
}
\ No newline at end of file
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
index fa37fab..cb4d92d 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOpHelperFactory.java
@@ -2,9 +2,9 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
public interface ITreeIndexOpHelperFactory extends Serializable {
public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
- final IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode);
+ final IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
index 6ca4529..9098a78 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/ITreeIndexOperatorDescriptorHelper.java
@@ -1,6 +1,6 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.dataflow.IActivityNode;
+import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -9,7 +9,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-public interface ITreeIndexOperatorDescriptorHelper extends IActivityNode {
+public interface ITreeIndexOperatorDescriptorHelper extends IActivity {
public IFileSplitProvider getTreeIndexFileSplitProvider();
public IBinaryComparatorFactory[] getTreeIndexComparatorFactories();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 7d585b3..fc12f59 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -46,7 +46,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new TreeIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
recordDescProvider);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index 2001039..817aeb4 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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;
@@ -36,9 +36,8 @@
private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
- public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition, int[] fieldPermutation, float fillFactor,
- IRecordDescriptorProvider recordDescProvider) {
+ public TreeIndexBulkLoadOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.CREATE);
this.fillFactor = fillFactor;
@@ -51,7 +50,7 @@
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
.getOperatorDescriptor();
RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
ITreeIndexMetaDataFrame metaFrame = new LIFOMetaDataFrame();
try {
treeIndexOpHelper.init();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index 73b5323..3a6eb62 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.ITypeTrait;
@@ -40,7 +40,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new TreeIndexDiskOrderScanOperatorNodePushable(this, ctx, partition);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 415b9e3..5b5c6e8 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -17,7 +17,7 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -35,7 +35,7 @@
private final TreeIndexOpHelper treeIndexOpHelper;
public TreeIndexDiskOrderScanOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition) {
+ IHyracksTaskContext ctx, int partition) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.OPEN);
}
@@ -57,8 +57,8 @@
treeIndexOpHelper.getTreeIndex().diskOrderScan(cursor, cursorFrame, metaFrame, diskOrderScanOpCtx);
int fieldCount = treeIndexOpHelper.getTreeIndex().getFieldCount();
- ByteBuffer frame = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext()
+ ByteBuffer frame = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext()
.getFrameSize());
appender.reset(frame, true);
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
index 2cfa905..2b1de29 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.job.IOperatorEnvironment;
@@ -42,7 +42,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new TreeIndexDropOperatorNodePushable(ctx, storageManager, treeIndexRegistryProvider, fileSplitProvider,
partition);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
index 71346f7..9b4ecb0 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDropOperatorNodePushable.java
@@ -19,7 +19,7 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.api.io.FileReference;
@@ -33,13 +33,13 @@
public class TreeIndexDropOperatorNodePushable extends AbstractOperatorNodePushable {
private static final Logger LOGGER = Logger.getLogger(TreeIndexDropOperatorNodePushable.class.getName());
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider;
private IStorageManagerInterface storageManager;
private IFileSplitProvider fileSplitProvider;
private int partition;
- public TreeIndexDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
+ public TreeIndexDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
IIndexRegistryProvider<ITreeIndex> treeIndexRegistryProvider, IFileSplitProvider fileSplitProvider,
int partition) {
this.ctx = ctx;
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
index 0961a4f..7812198 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -48,7 +48,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int partitions) throws HyracksDataException {
return new TreeIndexFileEnlistmentOperatorNodePushable(this, ctx, partition);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
index 8ff6586..9cbde53 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexFileEnlistmentOperatorNodePushable.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.std.base.AbstractOperatorNodePushable;
@@ -26,7 +26,7 @@
private final TreeIndexOpHelper treeIndexOpHelper;
public TreeIndexFileEnlistmentOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition) {
+ IHyracksTaskContext ctx, int partition) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.ENLIST);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index c8141b9..0457768 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -50,7 +50,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new TreeIndexInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
recordDescProvider, op);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index 74a9efc..496678b 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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;
@@ -38,7 +38,7 @@
private IndexOpContext opCtx;
public TreeIndexInsertUpdateDeleteOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
+ IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
IRecordDescriptorProvider recordDescProvider, IndexOp op) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.OPEN);
@@ -52,8 +52,9 @@
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
.getOperatorDescriptor();
RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
- writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
+ writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
+ writer.open();
try {
treeIndexOpHelper.init();
treeIndexOpHelper.getTreeIndex().open(treeIndexOpHelper.getIndexFileId());
@@ -96,6 +97,7 @@
} catch (Exception e) {
e.printStackTrace();
+ throw new HyracksDataException(e);
}
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
index fca6d8d..af98bb3 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexOpHelper.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -39,12 +39,12 @@
protected int partition;
protected ITreeIndexOperatorDescriptorHelper opDesc;
- protected IHyracksStageletContext ctx;
+ protected IHyracksTaskContext ctx;
protected IndexHelperOpenMode mode;
- public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx,
- int partition, IndexHelperOpenMode mode) {
+ public TreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx, int partition,
+ IndexHelperOpenMode mode) {
this.opDesc = opDesc;
this.ctx = ctx;
this.mode = mode;
@@ -154,7 +154,7 @@
return treeIndex;
}
- public IHyracksStageletContext getHyracksStageletContext() {
+ public IHyracksTaskContext getHyracksTaskContext() {
return ctx;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 574e727..d221e25 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -1,6 +1,6 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -26,7 +26,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new TreeIndexStatsOperatorNodePushable(this, ctx, partition);
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index f47855f..8c825c2 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.std.base.AbstractOperatorNodePushable;
@@ -26,10 +26,10 @@
public class TreeIndexStatsOperatorNodePushable extends AbstractOperatorNodePushable {
private final TreeIndexOpHelper treeIndexOpHelper;
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private TreeIndexStatsGatherer statsGatherer;
- public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+ public TreeIndexStatsOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.CREATE);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index 83246d6..b5b1393 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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;
@@ -47,7 +47,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, projFields);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 0647f45..1a0091a 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -19,7 +19,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -32,7 +32,7 @@
public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final IBinaryTokenizer tokenizer;
private final int[] tokenFields;
private final int[] projFields;
@@ -45,7 +45,7 @@
private FrameTupleAppender appender;
private ByteBuffer writeBuffer;
- public BinaryTokenizerOperatorNodePushable(IHyracksStageletContext ctx, RecordDescriptor inputRecDesc,
+ public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int[] tokenFields, int[] projFields) {
this.ctx = ctx;
this.tokenizer = tokenizer;
@@ -63,6 +63,7 @@
builderDos = builder.getDataOutput();
appender = new FrameTupleAppender(ctx.getFrameSize());
appender.reset(writeBuffer, true);
+ writer.open();
}
@Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index d003580..1ab72fe 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -57,7 +57,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new InvertedIndexBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, btreeFillFactor,
invListBuilder, recordDescProvider);
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
index 4969124..d2bd395 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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;
@@ -36,7 +36,7 @@
protected final IInvertedListBuilder invListBuilder;
private InvertedIndex.BulkLoadContext bulkLoadCtx;
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private FrameTupleAccessor accessor;
private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
@@ -44,7 +44,7 @@
private IRecordDescriptorProvider recordDescProvider;
public InvertedIndexBulkLoadOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
+ IHyracksTaskContext ctx, int partition, int[] fieldPermutation, float btreeFillFactor,
IInvertedListBuilder invListBuilder, IRecordDescriptorProvider recordDescProvider) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.CREATE);
@@ -61,7 +61,7 @@
AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) treeIndexOpHelper
.getOperatorDescriptor();
RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
// btree
try {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
index c16cfcd..1eb6757 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexOpHelper.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.storage.am.invertedindex.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -37,11 +37,11 @@
private int partition;
private IInvertedIndexOperatorDescriptorHelper opDesc;
- private IHyracksStageletContext ctx;
+ private IHyracksTaskContext ctx;
private IndexHelperOpenMode mode;
- public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksStageletContext ctx,
+ public InvertedIndexOpHelper(IInvertedIndexOperatorDescriptorHelper opDesc, final IHyracksTaskContext ctx,
int partition, IndexHelperOpenMode mode) {
this.opDesc = opDesc;
this.ctx = ctx;
@@ -144,10 +144,6 @@
return invIndex;
}
- public IHyracksStageletContext getHyracksStageletContext() {
- return ctx;
- }
-
public ITreeIndexOperatorDescriptorHelper getOperatorDescriptor() {
return opDesc;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index d1fba3b..7505ae0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -22,7 +22,7 @@
import java.util.List;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -52,7 +52,7 @@
public class TOccurrenceSearcher implements IInvertedIndexSearcher {
- protected final IHyracksStageletContext ctx;
+ protected final IHyracksTaskContext ctx;
protected final FixedSizeFrameTupleAppender resultFrameTupleApp;
protected final FixedSizeFrameTupleAccessor resultFrameTupleAcc;
protected final FixedSizeTupleReference resultTuple;
@@ -87,7 +87,7 @@
protected List<IInvertedListCursor> invListCursorCache = new ArrayList<IInvertedListCursor>(cursorCacheSize);
protected List<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>(cursorCacheSize);
- public TOccurrenceSearcher(IHyracksStageletContext ctx, InvertedIndex invIndex, IBinaryTokenizer queryTokenizer) {
+ public TOccurrenceSearcher(IHyracksTaskContext ctx, InvertedIndex invIndex, IBinaryTokenizer queryTokenizer) {
this.ctx = ctx;
this.invIndex = invIndex;
this.queryTokenizer = queryTokenizer;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
index 30d67f0..0f5439b 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
@@ -19,7 +19,7 @@
import java.nio.ByteBuffer;
import java.util.List;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
@@ -27,7 +27,7 @@
public class TOccurrenceSearcherSuffixProbeOnly extends TOccurrenceSearcher {
- public TOccurrenceSearcherSuffixProbeOnly(IHyracksStageletContext ctx, InvertedIndex invIndex,
+ public TOccurrenceSearcherSuffixProbeOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
IBinaryTokenizer queryTokenizer) {
super(ctx, invIndex, queryTokenizer);
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
index f8bc1ab..b997dc9 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
@@ -19,7 +19,7 @@
import java.nio.ByteBuffer;
import java.util.List;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
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.storage.am.common.ophelpers.MultiComparator;
@@ -28,7 +28,7 @@
public class TOccurrenceSearcherSuffixScanOnly extends TOccurrenceSearcher {
- public TOccurrenceSearcherSuffixScanOnly(IHyracksStageletContext ctx, InvertedIndex invIndex,
+ public TOccurrenceSearcherSuffixScanOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
IBinaryTokenizer queryTokenizer) {
super(ctx, invIndex, queryTokenizer);
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
index 5b8931d..2fcb065 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelper.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -33,7 +33,7 @@
protected MultiComparator interiorCmp;
- public RTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksStageletContext ctx, int partition,
+ public RTreeOpHelper(ITreeIndexOperatorDescriptorHelper opDesc, IHyracksTaskContext ctx, int partition,
IndexHelperOpenMode mode) {
super(opDesc, ctx, partition, mode);
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
index b26f21a..75668e1 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeOpHelperFactory.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOpHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.ITreeIndexOperatorDescriptorHelper;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexHelperOpenMode;
@@ -27,7 +27,7 @@
@Override
public TreeIndexOpHelper createTreeIndexOpHelper(ITreeIndexOperatorDescriptorHelper opDesc,
- IHyracksStageletContext ctx, int partition, IndexHelperOpenMode mode) {
+ IHyracksTaskContext ctx, int partition, IndexHelperOpenMode mode) {
return new RTreeOpHelper(opDesc, ctx, partition, mode);
}
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 8aab763..3dc0963 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.storage.am.rtree.dataflow;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -48,7 +48,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(final IHyracksStageletContext ctx, final IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, final IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new RTreeSearchOperatorNodePushable(this, ctx, partition, recordDescProvider, keyFields);
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 6a13d9f..86c5591 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -18,7 +18,7 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -64,7 +64,7 @@
private RecordDescriptor recDesc;
- public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+ public RTreeSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider, int[] keyFields) {
treeIndexOpHelper = opDesc.getTreeIndexOpHelperFactory().createTreeIndexOpHelper(opDesc, ctx, partition,
IndexHelperOpenMode.OPEN);
@@ -80,12 +80,14 @@
public void open() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexOpHelper
.getOperatorDescriptor();
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
interiorFrame = opDesc.getTreeIndexInteriorFactory().createFrame();
leafFrame = opDesc.getTreeIndexLeafFactory().createFrame();
cursor = new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrame, (IRTreeLeafFrame) leafFrame);
+ writer.open();
+
try {
treeIndexOpHelper.init();
@@ -100,12 +102,12 @@
cmp = new MultiComparator(rtree.getCmp().getTypeTraits(), keySearchComparators);
searchPred = new SearchPredicate(searchKey, cmp);
- accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexOpHelper.getHyracksTaskContext().getFrameSize(), recDesc);
- writeBuffer = treeIndexOpHelper.getHyracksStageletContext().allocateFrame();
+ writeBuffer = treeIndexOpHelper.getHyracksTaskContext().allocateFrame();
tb = new ArrayTupleBuilder(rtree.getCmp().getFieldCount());
dos = tb.getDataOutput();
- appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksStageletContext().getFrameSize());
+ appender = new FrameTupleAppender(treeIndexOpHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
opCtx = rtree.createOpContext(IndexOp.SEARCH, treeIndexOpHelper.getLeafFrame(),
@@ -113,6 +115,7 @@
} catch (Exception e) {
treeIndexOpHelper.deinit();
+ throw new HyracksDataException(e);
}
}
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
index f64af2e..562305e 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/IStorageManagerInterface.java
@@ -16,12 +16,12 @@
import java.io.Serializable;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public interface IStorageManagerInterface extends Serializable {
- public IBufferCache getBufferCache(IHyracksStageletContext ctx);
+ public IBufferCache getBufferCache(IHyracksTaskContext ctx);
- public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx);
+ public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx);
}
\ No newline at end of file
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index f9ac4d5..d7558ba 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -321,6 +321,39 @@
}
}
+ private String dumpState() {
+ StringBuilder buffer = new StringBuilder();
+ buffer.append("Buffer cache state\n");
+ buffer.append("Page Size: ").append(pageSize).append('\n');
+ buffer.append("Number of physical pages: ").append(numPages).append('\n');
+ buffer.append("Hash table size: ").append(pageMap.length).append('\n');
+ buffer.append("Page Map:\n");
+ int nCachedPages = 0;
+ for (int i = 0; i < pageMap.length; ++i) {
+ CacheBucket cb = pageMap[i];
+ cb.bucketLock.lock();
+ try {
+ CachedPage cp = cb.cachedPage;
+ if (cp != null) {
+ buffer.append(" ").append(i).append('\n');
+ while (cp != null) {
+ buffer.append(" ").append(cp.cpid).append(" -> [")
+ .append(BufferedFileHandle.getFileId(cp.dpid)).append(':')
+ .append(BufferedFileHandle.getPageId(cp.dpid)).append(", ").append(cp.pinCount.get())
+ .append(", ").append(cp.valid ? "valid" : "invalid").append(", ")
+ .append(cp.dirty.get() ? "dirty" : "clean").append("]\n");
+ cp = cp.next;
+ ++nCachedPages;
+ }
+ }
+ } finally {
+ cb.bucketLock.unlock();
+ }
+ }
+ buffer.append("Number of cached pages: ").append(nCachedPages).append('\n');
+ return buffer.toString();
+ }
+
private void read(CachedPage cPage) throws HyracksDataException {
BufferedFileHandle fInfo = getFileInfo(cPage);
cPage.buffer.clear();
@@ -645,6 +678,7 @@
public void closeFile(int fileId) throws HyracksDataException {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Closing file: " + fileId + " in cache: " + this);
+ LOGGER.info(dumpState());
}
synchronized (fileInfoMap) {
BufferedFileHandle fInfo = fileInfoMap.get(fileId);
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index 2fc43fc..4f84466 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.test.support;
import java.nio.ByteBuffer;
-import java.util.UUID;
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -23,22 +22,21 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
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.job.profiling.counters.ICounterContext;
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
public class TestJobletContext implements IHyracksJobletContext {
private final INCApplicationContext appContext;
- private UUID jobId;
- private int attempt;
- private ManagedWorkspaceFileFactory fileFactory;
+ private JobId jobId;
+ private WorkspaceFileFactory fileFactory;
- public TestJobletContext(INCApplicationContext appContext, UUID jobId, int attempt) throws HyracksException {
+ public TestJobletContext(INCApplicationContext appContext, JobId jobId) throws HyracksException {
this.appContext = appContext;
this.jobId = jobId;
- this.attempt = attempt;
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
}
@Override
@@ -57,13 +55,18 @@
}
@Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
- return fileFactory.createWorkspaceFile(prefix);
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createManagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createUnmanagedWorkspaceFile(prefix);
}
@Override
public ICounterContext getCounterContext() {
- return new CounterContext(jobId + "." + attempt);
+ return new CounterContext(jobId.toString());
}
@Override
@@ -82,12 +85,7 @@
}
@Override
- public UUID getJobId() {
+ public JobId getJobId() {
return jobId;
}
-
- @Override
- public int getAttempt() {
- return attempt;
- }
}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
deleted file mode 100644
index 4d31326..0000000
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * 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.test.support;
-
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-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.IIOManager;
-import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
-import edu.uci.ics.hyracks.api.resources.IDeallocatable;
-import edu.uci.ics.hyracks.control.nc.io.IOManager;
-import edu.uci.ics.hyracks.control.nc.io.ManagedWorkspaceFileFactory;
-
-public class TestStageletContext implements IHyracksStageletContext {
- private final IHyracksJobletContext jobletContext;
- private UUID stageId;
- private ManagedWorkspaceFileFactory fileFactory;
-
- public TestStageletContext(IHyracksJobletContext jobletContext, UUID stageId) throws HyracksException {
- this.jobletContext = jobletContext;
- this.stageId = stageId;
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
- }
-
- @Override
- public ByteBuffer allocateFrame() {
- return jobletContext.allocateFrame();
- }
-
- @Override
- public int getFrameSize() {
- return jobletContext.getFrameSize();
- }
-
- @Override
- public IIOManager getIOManager() {
- return jobletContext.getIOManager();
- }
-
- @Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
- return fileFactory.createWorkspaceFile(prefix);
- }
-
- @Override
- public IHyracksJobletContext getJobletContext() {
- return jobletContext;
- }
-
- @Override
- public UUID getStageId() {
- return stageId;
- }
-
- @Override
- public ICounterContext getCounterContext() {
- return new CounterContext(jobletContext.getJobId() + "." + jobletContext.getAttempt() + "." + stageId);
- }
-
- @Override
- public void registerDeallocatable(final IDeallocatable deallocatable) {
- Runtime.getRuntime().addShutdownHook(new Thread() {
- @Override
- public void run() {
- deallocatable.deallocate();
- }
- });
- }
-}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index 2325db7..c7bd09d 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.test.support;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
@@ -45,7 +45,7 @@
treeIndexRegistry = null;
}
- public synchronized static IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+ public synchronized static IBufferCache getBufferCache(IHyracksTaskContext ctx) {
if (bufferCache == null) {
ICacheMemoryAllocator allocator = new HeapBufferAllocator();
IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
@@ -56,16 +56,16 @@
return bufferCache;
}
- public synchronized static IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+ public synchronized static IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
if (fileMapProvider == null) {
fileMapProvider = new TransientFileMapManager();
}
return fileMapProvider;
}
- public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksStageletContext ctx) {
+ public synchronized static IndexRegistry<ITreeIndex> getTreeIndexRegistry(IHyracksTaskContext ctx) {
if (treeIndexRegistry == null) {
- treeIndexRegistry = new IndexRegistry<ITreeIndex>();
+ treeIndexRegistry = new IndexRegistry<ITreeIndex>();
}
return treeIndexRegistry;
}
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
index e923cdd..4059ef0 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerInterface.java
@@ -14,7 +14,7 @@
*/
package edu.uci.ics.hyracks.test.support;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -23,12 +23,12 @@
private static final long serialVersionUID = 1L;
@Override
- public IBufferCache getBufferCache(IHyracksStageletContext ctx) {
+ public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
return TestStorageManagerComponentHolder.getBufferCache(ctx);
}
@Override
- public IFileMapProvider getFileMapProvider(IHyracksStageletContext ctx) {
+ public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
return TestStorageManagerComponentHolder.getFileMapProvider(ctx);
}
}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
new file mode 100644
index 0000000..83b608f
--- /dev/null
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -0,0 +1,109 @@
+/*
+ * 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.test.support;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+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.IIOManager;
+import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.resources.IDeallocatable;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+
+public class TestTaskContext implements IHyracksTaskContext {
+ private final IHyracksJobletContext jobletContext;
+ private final TaskAttemptId taskId;
+ private WorkspaceFileFactory fileFactory;
+
+ public TestTaskContext(IHyracksJobletContext jobletContext, TaskAttemptId taskId) throws HyracksException {
+ this.jobletContext = jobletContext;
+ this.taskId = taskId;
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
+ }
+
+ @Override
+ public ByteBuffer allocateFrame() {
+ return jobletContext.allocateFrame();
+ }
+
+ @Override
+ public int getFrameSize() {
+ return jobletContext.getFrameSize();
+ }
+
+ @Override
+ public IIOManager getIOManager() {
+ return jobletContext.getIOManager();
+ }
+
+ @Override
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createManagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createUnmanagedWorkspaceFile(prefix);
+ }
+
+ @Override
+ public IHyracksJobletContext getJobletContext() {
+ return jobletContext;
+ }
+
+ @Override
+ public ICounterContext getCounterContext() {
+ return new CounterContext(jobletContext.getJobId() + "." + taskId);
+ }
+
+ @Override
+ public void registerDeallocatable(final IDeallocatable deallocatable) {
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ @Override
+ public void run() {
+ deallocatable.deallocate();
+ }
+ });
+ }
+
+ @Override
+ public TaskAttemptId getTaskAttemptId() {
+ return taskId;
+ }
+
+ @Override
+ public Object lookupGlobalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException {
+ return null;
+ }
+
+ @Override
+ public Object lookupLocalVariable(ActivityId producerActivity, int partition, String varName)
+ throws HyracksDataException {
+ return null;
+ }
+
+ @Override
+ public void setVariable(String name, Object value) throws HyracksDataException {
+
+ }
+}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
index 630f47f..9925fe8 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTreeIndexRegistryProvider.java
@@ -14,16 +14,16 @@
*/
package edu.uci.ics.hyracks.test.support;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
public class TestTreeIndexRegistryProvider implements IIndexRegistryProvider<ITreeIndex> {
private static final long serialVersionUID = 1L;
-
- @Override
- public IndexRegistry<ITreeIndex> getRegistry(IHyracksStageletContext ctx) {
- return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
- }
+
+ @Override
+ public IndexRegistry<ITreeIndex> getRegistry(IHyracksTaskContext ctx) {
+ return TestStorageManagerComponentHolder.getTreeIndexRegistry(ctx);
+ }
}
\ No newline at end of file
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
index b17f28c..5d488c1 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestUtils.java
@@ -14,22 +14,26 @@
*/
package edu.uci.ics.hyracks.test.support;
-import java.util.UUID;
-
import edu.uci.ics.hyracks.api.application.INCApplicationContext;
import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
public class TestUtils {
- public static IHyracksStageletContext create(int frameSize) {
+ public static IHyracksTaskContext create(int frameSize) {
try {
IHyracksRootContext rootCtx = new TestRootContext(frameSize);
INCApplicationContext appCtx = new TestNCApplicationContext(rootCtx, null);
- IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
- IHyracksStageletContext stageletCtx = new TestStageletContext(jobletCtx, UUID.randomUUID());
- return stageletCtx;
+ IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, new JobId(0));
+ TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityId(new OperatorDescriptorId(0), 0), 0), 0);
+ IHyracksTaskContext taskCtx = new TestTaskContext(jobletCtx, tid);
+ return taskCtx;
} catch (HyracksException e) {
throw new RuntimeException(e);
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
index 2b2bf55..4402e22 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeFieldPrefixNSMTest.java
@@ -24,7 +24,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -59,9 +59,9 @@
private static final int NUM_PAGES = 40;
private static final int MAX_OPEN_FILES = 10;
private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
- private ITupleReference createTuple(IHyracksStageletContext ctx, int f0,
+ private ITupleReference createTuple(IHyracksTaskContext ctx, int f0,
int f1, int f2, boolean print) throws HyracksDataException {
if (print)
LOGGER.info("CREATING: " + f0 + " " + f1 + " " + f2);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 44cf18b..3ef0cc2 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -8,7 +8,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -47,127 +47,116 @@
public class BTreeStatsTest extends AbstractBTreeTest {
- // private static final int PAGE_SIZE = 256;
- // private static final int NUM_PAGES = 10;
- // private static final int PAGE_SIZE = 32768;
+ // private static final int PAGE_SIZE = 256;
+ // private static final int NUM_PAGES = 10;
+ // private static final int PAGE_SIZE = 32768;
private static final int PAGE_SIZE = 4096;
private static final int NUM_PAGES = 1000;
private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
- @Test
- public void test01() throws Exception {
+ private static final int HYRACKS_FRAME_SIZE = 128;
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ @Test
+ public void test01() throws Exception {
- // declare fields
- int fieldCount = 2;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- typeTraits[0] = new TypeTrait(4);
- typeTraits[1] = new TypeTrait(4);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- // declare keys
- int keyFieldCount = 1;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
+ // declare fields
+ int fieldCount = 2;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ typeTraits[0] = new TypeTrait(4);
+ typeTraits[1] = new TypeTrait(4);
- MultiComparator cmp = new MultiComparator(typeTraits, cmps);
+ // declare keys
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
- typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ MultiComparator cmp = new MultiComparator(typeTraits, cmps);
- IBTreeLeafFrame leafFrame = (IBTreeLeafFrame)leafFrameFactory.createFrame();
- IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, fileId, 0, metaFrameFactory);
+ IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) leafFrameFactory.createFrame();
+ IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame) interiorFrameFactory.createFrame();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- BTree btree = new BTree(bufferCache, freePageManager,
- interiorFrameFactory, leafFrameFactory, cmp);
- btree.create(fileId, leafFrame, metaFrame);
- btree.open(fileId);
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, fileId, 0, metaFrameFactory);
- Random rnd = new Random();
- rnd.setSeed(50);
+ BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, cmp);
+ btree.create(fileId, leafFrame, metaFrame);
+ btree.open(fileId);
- long start = System.currentTimeMillis();
+ Random rnd = new Random();
+ rnd.setSeed(50);
- LOGGER.info("INSERTING INTO TREE");
+ long start = System.currentTimeMillis();
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
- DataOutput dos = tb.getDataOutput();
+ LOGGER.info("INSERTING INTO TREE");
- ISerializerDeserializer[] recDescSers = {
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx
- .getFrameSize(), recDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
+ ByteBuffer frame = ctx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
+ DataOutput dos = tb.getDataOutput();
- BTreeOpContext insertOpCtx = btree.createOpContext(
- IndexOp.INSERT, leafFrame, interiorFrame, metaFrame);
+ ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor.reset(frame);
+ FrameTupleReference tuple = new FrameTupleReference();
- // 10000
- for (int i = 0; i < 100000; i++) {
+ BTreeOpContext insertOpCtx = btree.createOpContext(IndexOp.INSERT, leafFrame, interiorFrame, metaFrame);
- int f0 = rnd.nextInt() % 100000;
- int f1 = 5;
+ // 10000
+ for (int i = 0; i < 100000; i++) {
- tb.reset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
- tb.addFieldEndOffset();
+ int f0 = rnd.nextInt() % 100000;
+ int f1 = 5;
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb
- .getSize());
+ tb.reset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(f0, dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(f1, dos);
+ tb.addFieldEndOffset();
- tuple.reset(accessor, 0);
-
- if (i % 10000 == 0) {
- long end = System.currentTimeMillis();
- LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " "
- + (end - start));
- }
+ appender.reset(frame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- try {
- btree.insert(tuple, insertOpCtx);
- } catch (TreeIndexException e) {
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
-
- TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId, btree.getRootPageId());
- TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
- LOGGER.info(stats.toString());
+ tuple.reset(accessor, 0);
- TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager, fileId);
- bufferCacheWarmup.warmup(leafFrame, metaFrame, new int[] {1, 2}, new int[] {2, 5});
-
- btree.close();
- bufferCache.closeFile(fileId);
- bufferCache.close();
- }
+ if (i % 10000 == 0) {
+ long end = System.currentTimeMillis();
+ LOGGER.info("INSERTING " + i + " : " + f0 + " " + f1 + " " + (end - start));
+ }
+
+ try {
+ btree.insert(tuple, insertOpCtx);
+ } catch (TreeIndexException e) {
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+
+ TreeIndexStatsGatherer statsGatherer = new TreeIndexStatsGatherer(bufferCache, freePageManager, fileId,
+ btree.getRootPageId());
+ TreeIndexStats stats = statsGatherer.gatherStats(leafFrame, interiorFrame, metaFrame);
+ LOGGER.info(stats.toString());
+
+ TreeIndexBufferCacheWarmup bufferCacheWarmup = new TreeIndexBufferCacheWarmup(bufferCache, freePageManager,
+ fileId);
+ bufferCacheWarmup.warmup(leafFrame, metaFrame, new int[] { 1, 2 }, new int[] { 2, 5 });
+
+ btree.close();
+ bufferCache.closeFile(fileId);
+ bufferCache.close();
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
index 6fa5d43..a9debd5 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeTest.java
@@ -23,7 +23,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -73,7 +73,7 @@
private static final int NUM_PAGES = 10;
private static final int MAX_OPEN_FILES = 10;
private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
// FIXED-LENGTH KEY TEST
// create a B-tree with one fixed-length "key" field and one fixed-length
@@ -175,8 +175,8 @@
int maxPage = btree.getFreePageManager().getMaxPage(metaFrame);
LOGGER.info("MAXPAGE: " + maxPage);
- LOGGER.info(btree.printStats());
-
+ LOGGER.info(btree.printStats());
+
long end = System.currentTimeMillis();
long duration = end - start;
LOGGER.info("DURATION: " + duration);
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
index 1a3b219..7dada06 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/RangeSearchCursorTest.java
@@ -30,7 +30,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -92,7 +92,7 @@
IBTreeInteriorFrame interiorFrame = (IBTreeInteriorFrame)interiorFrameFactory.createFrame();
ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
ByteBuffer frame = ctx.allocateFrame();
FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
diff --git a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
index 1a510e6..39bedac 100644
--- a/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
+++ b/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java
@@ -22,10 +22,9 @@
import org.junit.Test;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.btree.AbstractBTreeTest;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
@@ -35,238 +34,225 @@
import edu.uci.ics.hyracks.test.support.TestUtils;
public class StorageManagerTest extends AbstractBTreeTest {
- private static final int PAGE_SIZE = 256;
- private static final int NUM_PAGES = 10;
- private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(32768);
+ private static final int PAGE_SIZE = 256;
+ private static final int NUM_PAGES = 10;
+ private static final int MAX_OPEN_FILES = 10;
+ private static final int HYRACKS_FRAME_SIZE = 128;
+ private IHyracksTaskContext ctx = TestUtils.create(32768);
- public class PinnedLatchedPage {
- public final ICachedPage page;
- public final LatchType latch;
- public final int pageId;
+ public class PinnedLatchedPage {
+ public final ICachedPage page;
+ public final LatchType latch;
+ public final int pageId;
- public PinnedLatchedPage(ICachedPage page, int pageId, LatchType latch) {
- this.page = page;
- this.pageId = pageId;
- this.latch = latch;
- }
- }
+ public PinnedLatchedPage(ICachedPage page, int pageId, LatchType latch) {
+ this.page = page;
+ this.pageId = pageId;
+ this.latch = latch;
+ }
+ }
- public enum FileAccessType {
- FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
- }
+ public enum FileAccessType {
+ FTA_READONLY, FTA_WRITEONLY, FTA_MIXED, FTA_UNLATCHED
+ }
- public class FileAccessWorker implements Runnable {
- private int workerId;
- private final IBufferCache bufferCache;
- private final int maxPages;
- private final int fileId;
- private final long thinkTime;
- private final int maxLoopCount;
- private final int maxPinnedPages;
- private final int closeFileChance;
- private final FileAccessType fta;
- private int loopCount = 0;
- private boolean fileIsOpen = false;
- private Random rnd = new Random(50);
- private List<PinnedLatchedPage> pinnedPages = new LinkedList<PinnedLatchedPage>();
+ public class FileAccessWorker implements Runnable {
+ private int workerId;
+ private final IBufferCache bufferCache;
+ private final int maxPages;
+ private final int fileId;
+ private final long thinkTime;
+ private final int maxLoopCount;
+ private final int maxPinnedPages;
+ private final int closeFileChance;
+ private final FileAccessType fta;
+ private int loopCount = 0;
+ private boolean fileIsOpen = false;
+ private Random rnd = new Random(50);
+ private List<PinnedLatchedPage> pinnedPages = new LinkedList<PinnedLatchedPage>();
- public FileAccessWorker(int workerId, IBufferCache bufferCache,
- FileAccessType fta, int fileId, int maxPages,
- int maxPinnedPages, int maxLoopCount, int closeFileChance,
- long thinkTime) {
- this.bufferCache = bufferCache;
- this.fileId = fileId;
- this.maxPages = maxPages;
- this.maxLoopCount = maxLoopCount;
- this.maxPinnedPages = maxPinnedPages;
- this.thinkTime = thinkTime;
- this.closeFileChance = closeFileChance;
- this.workerId = workerId;
- this.fta = fta;
- }
+ public FileAccessWorker(int workerId, IBufferCache bufferCache, FileAccessType fta, int fileId, int maxPages,
+ int maxPinnedPages, int maxLoopCount, int closeFileChance, long thinkTime) {
+ this.bufferCache = bufferCache;
+ this.fileId = fileId;
+ this.maxPages = maxPages;
+ this.maxLoopCount = maxLoopCount;
+ this.maxPinnedPages = maxPinnedPages;
+ this.thinkTime = thinkTime;
+ this.closeFileChance = closeFileChance;
+ this.workerId = workerId;
+ this.fta = fta;
+ }
- private void pinRandomPage() {
- int pageId = Math.abs(rnd.nextInt() % maxPages);
+ private void pinRandomPage() {
+ int pageId = Math.abs(rnd.nextInt() % maxPages);
- LOGGER.info(workerId + " PINNING PAGE: " + pageId);
+ LOGGER.info(workerId + " PINNING PAGE: " + pageId);
- try {
- ICachedPage page = bufferCache.pin(BufferedFileHandle
- .getDiskPageId(fileId, pageId), false);
- LatchType latch = null;
+ try {
+ ICachedPage page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, pageId), false);
+ LatchType latch = null;
- switch (fta) {
+ switch (fta) {
- case FTA_UNLATCHED: {
- latch = null;
- }
- break;
+ case FTA_UNLATCHED: {
+ latch = null;
+ }
+ break;
- case FTA_READONLY: {
- LOGGER.info(workerId + " S LATCHING: " + pageId);
- page.acquireReadLatch();
- latch = LatchType.LATCH_S;
- }
- break;
+ case FTA_READONLY: {
+ LOGGER.info(workerId + " S LATCHING: " + pageId);
+ page.acquireReadLatch();
+ latch = LatchType.LATCH_S;
+ }
+ break;
- case FTA_WRITEONLY: {
- LOGGER.info(workerId + " X LATCHING: " + pageId);
- page.acquireWriteLatch();
- latch = LatchType.LATCH_X;
- }
- break;
+ case FTA_WRITEONLY: {
+ LOGGER.info(workerId + " X LATCHING: " + pageId);
+ page.acquireWriteLatch();
+ latch = LatchType.LATCH_X;
+ }
+ break;
- case FTA_MIXED: {
- if (rnd.nextInt() % 2 == 0) {
- LOGGER.info(workerId + " S LATCHING: " + pageId);
- page.acquireReadLatch();
- latch = LatchType.LATCH_S;
- } else {
- LOGGER.info(workerId + " X LATCHING: " + pageId);
- page.acquireWriteLatch();
- latch = LatchType.LATCH_X;
- }
- }
- break;
+ case FTA_MIXED: {
+ if (rnd.nextInt() % 2 == 0) {
+ LOGGER.info(workerId + " S LATCHING: " + pageId);
+ page.acquireReadLatch();
+ latch = LatchType.LATCH_S;
+ } else {
+ LOGGER.info(workerId + " X LATCHING: " + pageId);
+ page.acquireWriteLatch();
+ latch = LatchType.LATCH_X;
+ }
+ }
+ break;
- }
+ }
- PinnedLatchedPage plPage = new PinnedLatchedPage(page, pageId,
- latch);
- pinnedPages.add(plPage);
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- }
+ PinnedLatchedPage plPage = new PinnedLatchedPage(page, pageId, latch);
+ pinnedPages.add(plPage);
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
- private void unpinRandomPage() {
- int index = Math.abs(rnd.nextInt() % pinnedPages.size());
- try {
- PinnedLatchedPage plPage = pinnedPages.get(index);
+ private void unpinRandomPage() {
+ int index = Math.abs(rnd.nextInt() % pinnedPages.size());
+ try {
+ PinnedLatchedPage plPage = pinnedPages.get(index);
- if (plPage.latch != null) {
- if (plPage.latch == LatchType.LATCH_S) {
- LOGGER.info(workerId + " S UNLATCHING: "
- + plPage.pageId);
- plPage.page.releaseReadLatch();
- } else {
- LOGGER.info(workerId + " X UNLATCHING: "
- + plPage.pageId);
- plPage.page.releaseWriteLatch();
- }
- }
- LOGGER.info(workerId + " UNPINNING PAGE: "
- + plPage.pageId);
+ if (plPage.latch != null) {
+ if (plPage.latch == LatchType.LATCH_S) {
+ LOGGER.info(workerId + " S UNLATCHING: " + plPage.pageId);
+ plPage.page.releaseReadLatch();
+ } else {
+ LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId);
+ plPage.page.releaseWriteLatch();
+ }
+ }
+ LOGGER.info(workerId + " UNPINNING PAGE: " + plPage.pageId);
- bufferCache.unpin(plPage.page);
- pinnedPages.remove(index);
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- }
+ bufferCache.unpin(plPage.page);
+ pinnedPages.remove(index);
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
- private void openFile() {
- LOGGER.info(workerId + " OPENING FILE: " + fileId);
- try {
- bufferCache.openFile(fileId);
- fileIsOpen = true;
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- }
+ private void openFile() {
+ LOGGER.info(workerId + " OPENING FILE: " + fileId);
+ try {
+ bufferCache.openFile(fileId);
+ fileIsOpen = true;
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
- private void closeFile() {
- LOGGER.info(workerId + " CLOSING FILE: " + fileId);
- try {
- bufferCache.closeFile(fileId);
- fileIsOpen = false;
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- }
+ private void closeFile() {
+ LOGGER.info(workerId + " CLOSING FILE: " + fileId);
+ try {
+ bufferCache.closeFile(fileId);
+ fileIsOpen = false;
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ }
- @Override
- public void run() {
+ @Override
+ public void run() {
- openFile();
+ openFile();
- while (loopCount < maxLoopCount) {
- loopCount++;
+ while (loopCount < maxLoopCount) {
+ loopCount++;
- LOGGER.info(workerId + " LOOP: " + loopCount + "/"
- + maxLoopCount);
+ LOGGER.info(workerId + " LOOP: " + loopCount + "/" + maxLoopCount);
- if (fileIsOpen) {
+ if (fileIsOpen) {
- // pin some pages
- int pagesToPin = Math.abs(rnd.nextInt())
- % (maxPinnedPages - pinnedPages.size());
- for (int i = 0; i < pagesToPin; i++) {
- pinRandomPage();
- }
+ // pin some pages
+ int pagesToPin = Math.abs(rnd.nextInt()) % (maxPinnedPages - pinnedPages.size());
+ for (int i = 0; i < pagesToPin; i++) {
+ pinRandomPage();
+ }
- // do some thinking
- try {
- Thread.sleep(thinkTime);
- } catch (InterruptedException e) {
- e.printStackTrace();
- }
+ // do some thinking
+ try {
+ Thread.sleep(thinkTime);
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
- // unpin some pages
- if (!pinnedPages.isEmpty()) {
- int pagesToUnpin = Math.abs(rnd.nextInt())
- % pinnedPages.size();
- for (int i = 0; i < pagesToUnpin; i++) {
- unpinRandomPage();
- }
- }
+ // unpin some pages
+ if (!pinnedPages.isEmpty()) {
+ int pagesToUnpin = Math.abs(rnd.nextInt()) % pinnedPages.size();
+ for (int i = 0; i < pagesToUnpin; i++) {
+ unpinRandomPage();
+ }
+ }
- // possibly close file
- int closeFileCheck = Math.abs(rnd.nextInt())
- % closeFileChance;
- if (pinnedPages.isEmpty() || closeFileCheck == 0) {
- int numPinnedPages = pinnedPages.size();
- for (int i = 0; i < numPinnedPages; i++) {
- unpinRandomPage();
- }
- closeFile();
- }
- } else {
- openFile();
- }
- }
+ // possibly close file
+ int closeFileCheck = Math.abs(rnd.nextInt()) % closeFileChance;
+ if (pinnedPages.isEmpty() || closeFileCheck == 0) {
+ int numPinnedPages = pinnedPages.size();
+ for (int i = 0; i < numPinnedPages; i++) {
+ unpinRandomPage();
+ }
+ closeFile();
+ }
+ } else {
+ openFile();
+ }
+ }
- if (fileIsOpen) {
- int numPinnedPages = pinnedPages.size();
- for (int i = 0; i < numPinnedPages; i++) {
- unpinRandomPage();
- }
- closeFile();
- }
- }
- }
+ if (fileIsOpen) {
+ int numPinnedPages = pinnedPages.size();
+ for (int i = 0; i < numPinnedPages; i++) {
+ unpinRandomPage();
+ }
+ closeFile();
+ }
+ }
+ }
- @Test
- public void oneThreadOneFileTest() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(ctx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(ctx);
- FileReference file = new FileReference(new File(fileName));
- bufferCache.createFile(file);
- int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
+ @Test
+ public void oneThreadOneFileTest() throws Exception {
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
+ FileReference file = new FileReference(new File(fileName));
+ bufferCache.createFile(file);
+ int fileId = fmp.lookupFileId(file);
+ bufferCache.openFile(fileId);
- Thread worker = new Thread(new FileAccessWorker(0, bufferCache,
- FileAccessType.FTA_UNLATCHED, fileId, 10, 10, 100, 10, 0));
+ Thread worker = new Thread(new FileAccessWorker(0, bufferCache, FileAccessType.FTA_UNLATCHED, fileId, 10, 10,
+ 100, 10, 0));
- worker.start();
+ worker.start();
- worker.join();
+ worker.join();
- bufferCache.close();
- }
+ bufferCache.close();
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index b42001d..ea97511 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -10,7 +10,7 @@
import org.junit.Before;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -49,176 +49,154 @@
import edu.uci.ics.hyracks.test.support.TestUtils;
public abstract class AbstractInvIndexSearchTest extends AbstractInvIndexTest {
- protected final int PAGE_SIZE = 32768;
- protected final int NUM_PAGES = 100;
- protected final int MAX_OPEN_FILES = 10;
- protected final int HYRACKS_FRAME_SIZE = 32768;
- protected IHyracksStageletContext stageletCtx = TestUtils
- .create(HYRACKS_FRAME_SIZE);
+ protected final int PAGE_SIZE = 32768;
+ protected final int NUM_PAGES = 100;
+ protected final int MAX_OPEN_FILES = 10;
+ protected final int HYRACKS_FRAME_SIZE = 32768;
+ protected IHyracksTaskContext taskCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
- protected IBufferCache bufferCache;
- protected IFileMapProvider fmp;
+ protected IBufferCache bufferCache;
+ protected IFileMapProvider fmp;
- // --- BTREE ---
+ // --- BTREE ---
- // create file refs
- protected FileReference btreeFile = new FileReference(new File(
- btreeFileName));
- protected int btreeFileId;
+ // create file refs
+ protected FileReference btreeFile = new FileReference(new File(btreeFileName));
+ protected int btreeFileId;
- // declare btree fields
- protected int fieldCount = 5;
- protected ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ // declare btree fields
+ protected int fieldCount = 5;
+ protected ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- // declare btree keys
- protected int btreeKeyFieldCount = 1;
- protected IBinaryComparator[] btreeBinCmps = new IBinaryComparator[btreeKeyFieldCount];
- protected MultiComparator btreeCmp = new MultiComparator(typeTraits,
- btreeBinCmps);
+ // declare btree keys
+ protected int btreeKeyFieldCount = 1;
+ protected IBinaryComparator[] btreeBinCmps = new IBinaryComparator[btreeKeyFieldCount];
+ protected MultiComparator btreeCmp = new MultiComparator(typeTraits, btreeBinCmps);
- // btree frame factories
- protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
- typeTraits);
- protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
- protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ // btree frame factories
+ protected TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ protected ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ protected ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ protected ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- // btree frames
- protected ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
- protected ITreeIndexMetaDataFrame metaFrame = metaFrameFactory
- .createFrame();
+ // btree frames
+ protected ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+ protected ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- protected IFreePageManager freePageManager;
+ protected IFreePageManager freePageManager;
- protected BTree btree;
+ protected BTree btree;
- // --- INVERTED INDEX ---
+ // --- INVERTED INDEX ---
- protected FileReference invListsFile = new FileReference(new File(
- invListsFileName));
- protected int invListsFileId;
+ protected FileReference invListsFile = new FileReference(new File(invListsFileName));
+ protected int invListsFileId;
- protected int invListFields = 1;
- protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+ protected int invListFields = 1;
+ protected ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
- protected int invListKeys = 1;
- protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
- protected MultiComparator invListCmp = new MultiComparator(
- invListTypeTraits, invListBinCmps);
+ protected int invListKeys = 1;
+ protected IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
+ protected MultiComparator invListCmp = new MultiComparator(invListTypeTraits, invListBinCmps);
- protected InvertedIndex invIndex;
+ protected InvertedIndex invIndex;
- protected Random rnd = new Random();
+ protected Random rnd = new Random();
- protected ByteBuffer frame = stageletCtx.allocateFrame();
- protected FrameTupleAppender appender = new FrameTupleAppender(
- stageletCtx.getFrameSize());
- protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
- protected DataOutput dos = tb.getDataOutput();
+ protected ByteBuffer frame = taskCtx.allocateFrame();
+ protected FrameTupleAppender appender = new FrameTupleAppender(taskCtx.getFrameSize());
+ protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
+ protected DataOutput dos = tb.getDataOutput();
- protected ISerializerDeserializer[] insertSerde = {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
- protected IFrameTupleAccessor accessor = new FrameTupleAccessor(
- stageletCtx.getFrameSize(), insertRecDesc);
+ protected ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+ protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
+ protected IFrameTupleAccessor accessor = new FrameTupleAccessor(taskCtx.getFrameSize(), insertRecDesc);
- protected FrameTupleReference tuple = new FrameTupleReference();
+ protected FrameTupleReference tuple = new FrameTupleReference();
- protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
+ protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
- protected int maxId = 1000000;
- // protected int maxId = 1000;
- protected int[] scanCountArray = new int[maxId];
- protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
+ protected int maxId = 1000000;
+ // protected int maxId = 1000;
+ protected int[] scanCountArray = new int[maxId];
+ protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
- protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
- protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
+ protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
+ protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
- protected FrameTupleAppender queryAppender = new FrameTupleAppender(
- stageletCtx.getFrameSize());
- protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(
- querySerde.length);
- protected DataOutput queryDos = queryTb.getDataOutput();
+ protected FrameTupleAppender queryAppender = new FrameTupleAppender(taskCtx.getFrameSize());
+ protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(querySerde.length);
+ protected DataOutput queryDos = queryTb.getDataOutput();
- protected IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(
- stageletCtx.getFrameSize(), queryRecDesc);
- protected FrameTupleReference queryTuple = new FrameTupleReference();
+ protected IFrameTupleAccessor queryAccessor = new FrameTupleAccessor(taskCtx.getFrameSize(), queryRecDesc);
+ protected FrameTupleReference queryTuple = new FrameTupleReference();
- protected ITokenFactory tokenFactory;
- protected IBinaryTokenizer tokenizer;
+ protected ITokenFactory tokenFactory;
+ protected IBinaryTokenizer tokenizer;
- protected TOccurrenceSearcher searcher;
- protected IInvertedIndexResultCursor resultCursor;
+ protected TOccurrenceSearcher searcher;
+ protected IInvertedIndexResultCursor resultCursor;
- /**
- * Initialize members, generate data, and bulk load the inverted index.
- *
- */
- @Before
- public void start() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(stageletCtx);
- fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
+ /**
+ * Initialize members, generate data, and bulk load the inverted index.
+ */
+ @Before
+ public void start() throws Exception {
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ bufferCache = TestStorageManagerComponentHolder.getBufferCache(taskCtx);
+ fmp = TestStorageManagerComponentHolder.getFileMapProvider(taskCtx);
- // --- BTREE ---
+ // --- BTREE ---
- bufferCache.createFile(btreeFile);
- btreeFileId = fmp.lookupFileId(btreeFile);
- bufferCache.openFile(btreeFileId);
+ bufferCache.createFile(btreeFile);
+ btreeFileId = fmp.lookupFileId(btreeFile);
+ bufferCache.openFile(btreeFileId);
- // token (key)
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- // startPageId
- typeTraits[1] = new TypeTrait(4);
- // endPageId
- typeTraits[2] = new TypeTrait(4);
- // startOff
- typeTraits[3] = new TypeTrait(4);
- // numElements
- typeTraits[4] = new TypeTrait(4);
+ // token (key)
+ typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ // startPageId
+ typeTraits[1] = new TypeTrait(4);
+ // endPageId
+ typeTraits[2] = new TypeTrait(4);
+ // startOff
+ typeTraits[3] = new TypeTrait(4);
+ // numElements
+ typeTraits[4] = new TypeTrait(4);
- btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
+ btreeBinCmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- freePageManager = new LinkedListFreePageManager(bufferCache,
- btreeFileId, 0, metaFrameFactory);
+ freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- btree = new BTree(bufferCache, freePageManager, interiorFrameFactory,
- leafFrameFactory, btreeCmp);
- btree.create(btreeFileId, leafFrame, metaFrame);
- btree.open(btreeFileId);
+ btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, btreeCmp);
+ btree.create(btreeFileId, leafFrame, metaFrame);
+ btree.open(btreeFileId);
- // --- INVERTED INDEX ---
+ // --- INVERTED INDEX ---
- bufferCache.createFile(invListsFile);
- invListsFileId = fmp.lookupFileId(invListsFile);
- bufferCache.openFile(invListsFileId);
+ bufferCache.createFile(invListsFile);
+ invListsFileId = fmp.lookupFileId(invListsFile);
+ bufferCache.openFile(invListsFileId);
- invListTypeTraits[0] = new TypeTrait(4);
- invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
+ invListTypeTraits[0] = new TypeTrait(4);
+ invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
- invIndex.open(invListsFileId);
+ invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
+ invIndex.open(invListsFileId);
- rnd.setSeed(50);
+ rnd.setSeed(50);
- accessor.reset(frame);
- queryAccessor.reset(frame);
- }
+ accessor.reset(frame);
+ queryAccessor.reset(frame);
+ }
- @After
- public void deinit() throws HyracksDataException {
- AbstractInvIndexTest.tearDown();
- btree.close();
- invIndex.close();
- bufferCache.closeFile(btreeFileId);
- bufferCache.closeFile(invListsFileId);
- bufferCache.close();
- }
+ @After
+ public void deinit() throws HyracksDataException {
+ AbstractInvIndexTest.tearDown();
+ btree.close();
+ invIndex.close();
+ bufferCache.closeFile(btreeFileId);
+ bufferCache.closeFile(invListsFileId);
+ bufferCache.close();
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index fb0b7b9..bdd8c4a 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -28,7 +28,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -74,272 +74,238 @@
public class BulkLoadTest extends AbstractInvIndexTest {
- private static final int PAGE_SIZE = 32768;
- private static final int NUM_PAGES = 100;
- private static final int MAX_OPEN_FILES = 10;
- private static final int HYRACKS_FRAME_SIZE = 32768;
- private IHyracksStageletContext stageletCtx = TestUtils
- .create(HYRACKS_FRAME_SIZE);
+ private static final int PAGE_SIZE = 32768;
+ private static final int NUM_PAGES = 100;
+ private static final int MAX_OPEN_FILES = 10;
+ private static final int HYRACKS_FRAME_SIZE = 32768;
+ private IHyracksTaskContext stageletCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
- /**
- * This test generates a list of <word-token, id> pairs which are pre-sorted
- * on the token. Those pairs for the input to an inverted-index bulk load.
- * The contents of the inverted lists are verified against the generated
- * data.
- *
- */
- @Test
- public void singleFieldPayloadTest() throws Exception {
+ /**
+ * This test generates a list of <word-token, id> pairs which are pre-sorted
+ * on the token. Those pairs for the input to an inverted-index bulk load.
+ * The contents of the inverted lists are verified against the generated
+ * data.
+ */
+ @Test
+ public void singleFieldPayloadTest() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES,
- MAX_OPEN_FILES);
- IBufferCache bufferCache = TestStorageManagerComponentHolder
- .getBufferCache(stageletCtx);
- IFileMapProvider fmp = TestStorageManagerComponentHolder
- .getFileMapProvider(stageletCtx);
+ TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
+ IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(stageletCtx);
+ IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(stageletCtx);
- // create file refs
- FileReference btreeFile = new FileReference(new File(btreeFileName));
- bufferCache.createFile(btreeFile);
- int btreeFileId = fmp.lookupFileId(btreeFile);
- bufferCache.openFile(btreeFileId);
+ // create file refs
+ FileReference btreeFile = new FileReference(new File(btreeFileName));
+ bufferCache.createFile(btreeFile);
+ int btreeFileId = fmp.lookupFileId(btreeFile);
+ bufferCache.openFile(btreeFileId);
- FileReference invListsFile = new FileReference(new File(
- invListsFileName));
- bufferCache.createFile(invListsFile);
- int invListsFileId = fmp.lookupFileId(invListsFile);
- bufferCache.openFile(invListsFileId);
+ FileReference invListsFile = new FileReference(new File(invListsFileName));
+ bufferCache.createFile(invListsFile);
+ int invListsFileId = fmp.lookupFileId(invListsFile);
+ bufferCache.openFile(invListsFileId);
- // declare btree fields
- int fieldCount = 5;
- ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
- // token (key)
- typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
- // startPageId
- typeTraits[1] = new TypeTrait(4);
- // endPageId
- typeTraits[2] = new TypeTrait(4);
- // startOff
- typeTraits[3] = new TypeTrait(4);
- // numElements
- typeTraits[4] = new TypeTrait(4);
+ // declare btree fields
+ int fieldCount = 5;
+ ITypeTrait[] typeTraits = new ITypeTrait[fieldCount];
+ // token (key)
+ typeTraits[0] = new TypeTrait(ITypeTrait.VARIABLE_LENGTH);
+ // startPageId
+ typeTraits[1] = new TypeTrait(4);
+ // endPageId
+ typeTraits[2] = new TypeTrait(4);
+ // startOff
+ typeTraits[3] = new TypeTrait(4);
+ // numElements
+ typeTraits[4] = new TypeTrait(4);
- // declare btree keys
- int keyFieldCount = 1;
- IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
- cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
+ // declare btree keys
+ int keyFieldCount = 1;
+ IBinaryComparator[] cmps = new IBinaryComparator[keyFieldCount];
+ cmps[0] = UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator btreeCmp = new MultiComparator(typeTraits, cmps);
+ MultiComparator btreeCmp = new MultiComparator(typeTraits, cmps);
- TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(
- typeTraits);
- ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(
- tupleWriterFactory);
- ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(
- tupleWriterFactory);
- ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
+ TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
+ ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
+ ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+ ITreeIndexMetaDataFrameFactory metaFrameFactory = new LIFOMetaDataFrameFactory();
- ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
- ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
- ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
+ ITreeIndexFrame leafFrame = leafFrameFactory.createFrame();
+ ITreeIndexFrame interiorFrame = interiorFrameFactory.createFrame();
+ ITreeIndexMetaDataFrame metaFrame = metaFrameFactory.createFrame();
- IFreePageManager freePageManager = new LinkedListFreePageManager(
- bufferCache, btreeFileId, 0, metaFrameFactory);
+ IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, btreeFileId, 0, metaFrameFactory);
- BTree btree = new BTree(bufferCache, freePageManager,
- interiorFrameFactory, leafFrameFactory, btreeCmp);
- btree.create(btreeFileId, leafFrame, metaFrame);
- btree.open(btreeFileId);
+ BTree btree = new BTree(bufferCache, freePageManager, interiorFrameFactory, leafFrameFactory, btreeCmp);
+ btree.create(btreeFileId, leafFrame, metaFrame);
+ btree.open(btreeFileId);
- int invListFields = 1;
- ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
- invListTypeTraits[0] = new TypeTrait(4);
+ int invListFields = 1;
+ ITypeTrait[] invListTypeTraits = new ITypeTrait[invListFields];
+ invListTypeTraits[0] = new TypeTrait(4);
- int invListKeys = 1;
- IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
- invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE
- .createBinaryComparator();
+ int invListKeys = 1;
+ IBinaryComparator[] invListBinCmps = new IBinaryComparator[invListKeys];
+ invListBinCmps[0] = IntegerBinaryComparatorFactory.INSTANCE.createBinaryComparator();
- MultiComparator invListCmp = new MultiComparator(invListTypeTraits,
- invListBinCmps);
+ MultiComparator invListCmp = new MultiComparator(invListTypeTraits, invListBinCmps);
- InvertedIndex invIndex = new InvertedIndex(bufferCache, btree,
- invListCmp);
- invIndex.open(invListsFileId);
+ InvertedIndex invIndex = new InvertedIndex(bufferCache, btree, invListCmp);
+ invIndex.open(invListsFileId);
- Random rnd = new Random();
- rnd.setSeed(50);
+ Random rnd = new Random();
+ rnd.setSeed(50);
- ByteBuffer frame = stageletCtx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(
- stageletCtx.getFrameSize());
- ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
- DataOutput dos = tb.getDataOutput();
+ ByteBuffer frame = stageletCtx.allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(stageletCtx.getFrameSize());
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
+ DataOutput dos = tb.getDataOutput();
- ISerializerDeserializer[] insertSerde = {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(
- stageletCtx.getFrameSize(), insertRecDesc);
- accessor.reset(frame);
- FrameTupleReference tuple = new FrameTupleReference();
+ ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE };
+ RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), insertRecDesc);
+ accessor.reset(frame);
+ FrameTupleReference tuple = new FrameTupleReference();
- List<String> tokens = new ArrayList<String>();
- tokens.add("compilers");
- tokens.add("computer");
- tokens.add("databases");
- tokens.add("fast");
- tokens.add("hyracks");
- tokens.add("major");
- tokens.add("science");
- tokens.add("systems");
- tokens.add("university");
+ List<String> tokens = new ArrayList<String>();
+ tokens.add("compilers");
+ tokens.add("computer");
+ tokens.add("databases");
+ tokens.add("fast");
+ tokens.add("hyracks");
+ tokens.add("major");
+ tokens.add("science");
+ tokens.add("systems");
+ tokens.add("university");
- ArrayList<ArrayList<Integer>> checkListElements = new ArrayList<ArrayList<Integer>>();
- for (int i = 0; i < tokens.size(); i++) {
- checkListElements.add(new ArrayList<Integer>());
- }
+ ArrayList<ArrayList<Integer>> checkListElements = new ArrayList<ArrayList<Integer>>();
+ for (int i = 0; i < tokens.size(); i++) {
+ checkListElements.add(new ArrayList<Integer>());
+ }
- int maxId = 1000000;
- int addProb = 0;
- int addProbStep = 10;
+ int maxId = 1000000;
+ int addProb = 0;
+ int addProbStep = 10;
- IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(
- invListTypeTraits);
- InvertedIndex.BulkLoadContext ctx = invIndex.beginBulkLoad(
- invListBuilder, HYRACKS_FRAME_SIZE, BTree.DEFAULT_FILL_FACTOR);
+ IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
+ InvertedIndex.BulkLoadContext ctx = invIndex.beginBulkLoad(invListBuilder, HYRACKS_FRAME_SIZE,
+ BTree.DEFAULT_FILL_FACTOR);
- int totalElements = 0;
- for (int i = 0; i < tokens.size(); i++) {
+ int totalElements = 0;
+ for (int i = 0; i < tokens.size(); i++) {
- addProb += addProbStep * (i + 1);
- for (int j = 0; j < maxId; j++) {
- if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
+ addProb += addProbStep * (i + 1);
+ for (int j = 0; j < maxId; j++) {
+ if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
- totalElements++;
+ totalElements++;
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(
- tokens.get(i), dos);
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(j, dos);
- tb.addFieldEndOffset();
+ tb.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), dos);
+ tb.addFieldEndOffset();
+ IntegerSerializerDeserializer.INSTANCE.serialize(j, dos);
+ tb.addFieldEndOffset();
- checkListElements.get(i).add(j);
+ checkListElements.get(i).add(j);
- appender.reset(frame, true);
- appender.append(tb.getFieldEndOffsets(), tb.getByteArray(),
- 0, tb.getSize());
+ appender.reset(frame, true);
+ appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
- tuple.reset(accessor, 0);
+ tuple.reset(accessor, 0);
- try {
- invIndex.bulkLoadAddTuple(ctx, tuple);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
- invIndex.endBulkLoad(ctx);
+ try {
+ invIndex.bulkLoadAddTuple(ctx, tuple);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+ invIndex.endBulkLoad(ctx);
- // ------- START VERIFICATION -----------
+ // ------- START VERIFICATION -----------
- ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor(
- (IBTreeLeafFrame) leafFrame);
- FrameTupleReference searchKey = new FrameTupleReference();
- RangePredicate btreePred = new RangePredicate(true, searchKey,
- searchKey, true, true, btreeCmp, btreeCmp);
+ ITreeIndexCursor btreeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame);
+ FrameTupleReference searchKey = new FrameTupleReference();
+ RangePredicate btreePred = new RangePredicate(true, searchKey, searchKey, true, true, btreeCmp, btreeCmp);
- IInvertedListCursor invListCursor = new FixedSizeElementInvertedListCursor(
- bufferCache, invListsFileId, invListTypeTraits);
+ IInvertedListCursor invListCursor = new FixedSizeElementInvertedListCursor(bufferCache, invListsFileId,
+ invListTypeTraits);
- ISerializerDeserializer[] tokenSerde = { UTF8StringSerializerDeserializer.INSTANCE };
- RecordDescriptor tokenRecDesc = new RecordDescriptor(tokenSerde);
- FrameTupleAppender tokenAppender = new FrameTupleAppender(
- stageletCtx.getFrameSize());
- ArrayTupleBuilder tokenTupleBuilder = new ArrayTupleBuilder(1);
- DataOutput tokenDos = tokenTupleBuilder.getDataOutput();
- IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(
- stageletCtx.getFrameSize(), tokenRecDesc);
- tokenAccessor.reset(frame);
+ ISerializerDeserializer[] tokenSerde = { UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor tokenRecDesc = new RecordDescriptor(tokenSerde);
+ FrameTupleAppender tokenAppender = new FrameTupleAppender(stageletCtx.getFrameSize());
+ ArrayTupleBuilder tokenTupleBuilder = new ArrayTupleBuilder(1);
+ DataOutput tokenDos = tokenTupleBuilder.getDataOutput();
+ IFrameTupleAccessor tokenAccessor = new FrameTupleAccessor(stageletCtx.getFrameSize(), tokenRecDesc);
+ tokenAccessor.reset(frame);
- BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(
- IndexOp.SEARCH, leafFrame, interiorFrame, null);
+ BTreeOpContext btreeOpCtx = invIndex.getBTree().createOpContext(IndexOp.SEARCH, leafFrame, interiorFrame, null);
- // verify created inverted lists one-by-one
- for (int i = 0; i < tokens.size(); i++) {
+ // verify created inverted lists one-by-one
+ for (int i = 0; i < tokens.size(); i++) {
- tokenTupleBuilder.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i),
- tokenDos);
- tokenTupleBuilder.addFieldEndOffset();
+ tokenTupleBuilder.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), tokenDos);
+ tokenTupleBuilder.addFieldEndOffset();
- tokenAppender.reset(frame, true);
- tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(),
- tokenTupleBuilder.getByteArray(), 0,
- tokenTupleBuilder.getSize());
+ tokenAppender.reset(frame, true);
+ tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
+ tokenTupleBuilder.getSize());
- searchKey.reset(tokenAccessor, 0);
+ searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx,
- invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
- invListCursor.pinPagesSync();
- int checkIndex = 0;
- while (invListCursor.hasNext()) {
- invListCursor.next();
- ITupleReference invListTuple = invListCursor.getTuple();
- int invListElement = IntegerSerializerDeserializer.getInt(
- invListTuple.getFieldData(0),
- invListTuple.getFieldStart(0));
- int checkInvListElement = checkListElements.get(i)
- .get(checkIndex).intValue();
- Assert.assertEquals(invListElement, checkInvListElement);
- checkIndex++;
- }
- invListCursor.unpinPages();
- Assert.assertEquals(checkIndex, checkListElements.get(i).size());
- }
+ invListCursor.pinPagesSync();
+ int checkIndex = 0;
+ while (invListCursor.hasNext()) {
+ invListCursor.next();
+ ITupleReference invListTuple = invListCursor.getTuple();
+ int invListElement = IntegerSerializerDeserializer.getInt(invListTuple.getFieldData(0),
+ invListTuple.getFieldStart(0));
+ int checkInvListElement = checkListElements.get(i).get(checkIndex).intValue();
+ Assert.assertEquals(invListElement, checkInvListElement);
+ checkIndex++;
+ }
+ invListCursor.unpinPages();
+ Assert.assertEquals(checkIndex, checkListElements.get(i).size());
+ }
- // check that non-existing tokens have an empty inverted list
- List<String> nonExistingTokens = new ArrayList<String>();
- nonExistingTokens.add("watermelon");
- nonExistingTokens.add("avocado");
- nonExistingTokens.add("lemon");
+ // check that non-existing tokens have an empty inverted list
+ List<String> nonExistingTokens = new ArrayList<String>();
+ nonExistingTokens.add("watermelon");
+ nonExistingTokens.add("avocado");
+ nonExistingTokens.add("lemon");
- for (int i = 0; i < nonExistingTokens.size(); i++) {
+ for (int i = 0; i < nonExistingTokens.size(); i++) {
- tokenTupleBuilder.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(
- nonExistingTokens.get(i), tokenDos);
- tokenTupleBuilder.addFieldEndOffset();
+ tokenTupleBuilder.reset();
+ UTF8StringSerializerDeserializer.INSTANCE.serialize(nonExistingTokens.get(i), tokenDos);
+ tokenTupleBuilder.addFieldEndOffset();
- tokenAppender.reset(frame, true);
- tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(),
- tokenTupleBuilder.getByteArray(), 0,
- tokenTupleBuilder.getSize());
+ tokenAppender.reset(frame, true);
+ tokenAppender.append(tokenTupleBuilder.getFieldEndOffsets(), tokenTupleBuilder.getByteArray(), 0,
+ tokenTupleBuilder.getSize());
- searchKey.reset(tokenAccessor, 0);
+ searchKey.reset(tokenAccessor, 0);
- invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx,
- invListCursor);
+ invIndex.openCursor(btreeCursor, btreePred, btreeOpCtx, invListCursor);
- invListCursor.pinPagesSync();
- Assert.assertEquals(invListCursor.hasNext(), false);
- invListCursor.unpinPages();
- }
+ invListCursor.pinPagesSync();
+ Assert.assertEquals(invListCursor.hasNext(), false);
+ invListCursor.unpinPages();
+ }
- btree.close();
- bufferCache.closeFile(btreeFileId);
- bufferCache.closeFile(invListsFileId);
- bufferCache.close();
- }
+ btree.close();
+ bufferCache.closeFile(btreeFileId);
+ bufferCache.closeFile(invListsFileId);
+ bufferCache.close();
+ }
- @AfterClass
- public static void deinit() {
- AbstractInvIndexTest.tearDown();
- }
+ @AfterClass
+ public static void deinit() {
+ AbstractInvIndexTest.tearDown();
+ }
}
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
index 161f20f..4ef8855 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
@@ -58,7 +58,7 @@
tokenFactory = new UTF8WordTokenFactory();
tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false,
tokenFactory);
- searcher = new TOccurrenceSearcher(stageletCtx, invIndex, tokenizer);
+ searcher = new TOccurrenceSearcher(taskCtx, invIndex, tokenizer);
resultCursor = new SearchResultCursor(
searcher.createResultFrameTupleAccessor(),
searcher.createResultTupleReference());
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index d9fef2c..c87ce9b 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -40,7 +40,7 @@
tokenFactory = new UTF8NGramTokenFactory();
tokenizer = new NGramUTF8StringBinaryTokenizer(3, false, true, false,
tokenFactory);
- searcher = new TOccurrenceSearcher(stageletCtx, invIndex, tokenizer);
+ searcher = new TOccurrenceSearcher(taskCtx, invIndex, tokenizer);
resultCursor = new SearchResultCursor(
searcher.createResultFrameTupleAccessor(),
searcher.createResultTupleReference());
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
index 72c3f38..514b81f 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTest.java
@@ -23,7 +23,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -67,7 +67,7 @@
private static final int NUM_PAGES = 10;
private static final int MAX_OPEN_FILES = 10;
private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
// create an R-tree of two dimensions
// fill the R-tree with random values using insertions
diff --git a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
index 3060765..fb53b2a 100644
--- a/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
+++ b/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/SearchCursorTest.java
@@ -27,7 +27,7 @@
import org.junit.Test;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTrait;
@@ -71,7 +71,7 @@
private static final int NUM_PAGES = 10;
private static final int MAX_OPEN_FILES = 10;
private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
// create an R-tree of two dimensions
// fill the R-tree with random values using insertions
diff --git a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
index 680fd3b..80502eb 100644
--- a/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
+++ b/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java
@@ -13,7 +13,7 @@
import org.junit.Assert;
import org.junit.Test;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
@@ -24,25 +24,25 @@
import edu.uci.ics.hyracks.test.support.TestUtils;
public class BufferCacheTest {
- protected static final List<String> openedFiles = new ArrayList<String>();
+ protected static final List<String> openedFiles = new ArrayList<String>();
protected static final SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
protected static final String tmpDir = System.getProperty("java.io.tmpdir");
- protected static final String sep = System.getProperty("file.separator");
-
+ protected static final String sep = System.getProperty("file.separator");
+
private static final int PAGE_SIZE = 256;
private static final int NUM_PAGES = 10;
private static final int MAX_OPEN_FILES = 20;
private static final int HYRACKS_FRAME_SIZE = PAGE_SIZE;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+
private static final Random rnd = new Random(50);
-
+
private String getFileName() {
String fileName = tmpDir + sep + simpleDateFormat.format(new Date()) + openedFiles.size();
openedFiles.add(fileName);
return fileName;
- }
-
+ }
+
@Test
public void simpleOpenPinCloseTest() throws HyracksDataException {
TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
@@ -54,85 +54,84 @@
int fileId = fmp.lookupFileId(file);
int num = 10;
int testPageId = 0;
-
+
bufferCache.openFile(fileId);
-
+
ICachedPage page = null;
-
+
// tryPin should fail
page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
Assert.assertNull(page);
-
+
// pin page should succeed
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), true);
page.acquireWriteLatch();
- try {
- for(int i = 0; i < num; i++) {
+ try {
+ for (int i = 0; i < num; i++) {
page.getBuffer().putInt(i * 4, i);
- }
-
+ }
+
// try pin should succeed
ICachedPage page2 = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
Assert.assertNotNull(page2);
bufferCache.unpin(page2);
-
+
} finally {
page.releaseWriteLatch();
bufferCache.unpin(page);
- }
-
+ }
+
bufferCache.closeFile(fileId);
-
+
boolean exceptionThrown = false;
-
+
// tryPin should fail since file is not open
try {
page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
- } catch(HyracksDataException e) {
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
Assert.assertTrue(exceptionThrown);
-
+
// pin should fail since file is not open
exceptionThrown = false;
try {
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), false);
- } catch(HyracksDataException e) {
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
Assert.assertTrue(exceptionThrown);
-
-
+
// open file again
bufferCache.openFile(fileId);
-
+
// tryPin should succeed because page should still be cached
- page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
+ page = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, testPageId));
Assert.assertNotNull(page);
page.acquireReadLatch();
try {
// verify contents of page
- for(int i = 0; i < num; i++) {
+ for (int i = 0; i < num; i++) {
Assert.assertEquals(page.getBuffer().getInt(i * 4), i);
}
} finally {
page.releaseReadLatch();
bufferCache.unpin(page);
- }
-
- bufferCache.closeFile(fileId);
+ }
+
+ bufferCache.closeFile(fileId);
bufferCache.close();
}
-
+
@Test
public void simpleMaxOpenFilesTest() throws HyracksDataException {
TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-
+
List<Integer> fileIds = new ArrayList<Integer>();
-
- for(int i = 0; i < MAX_OPEN_FILES; i++) {
+
+ for (int i = 0; i < MAX_OPEN_FILES; i++) {
String fileName = getFileName();
FileReference file = new FileReference(new File(fileName));
bufferCache.createFile(file);
@@ -140,9 +139,9 @@
bufferCache.openFile(fileId);
fileIds.add(fileId);
}
-
+
boolean exceptionThrown = false;
-
+
// since all files are open, next open should fail
try {
String fileName = getFileName();
@@ -150,16 +149,16 @@
bufferCache.createFile(file);
int fileId = fmp.lookupFileId(file);
bufferCache.openFile(fileId);
- } catch(HyracksDataException e) {
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
Assert.assertTrue(exceptionThrown);
-
+
// close a random file
int ix = Math.abs(rnd.nextInt()) % fileIds.size();
bufferCache.closeFile(fileIds.get(ix));
fileIds.remove(ix);
-
+
// now open should succeed again
exceptionThrown = false;
try {
@@ -169,58 +168,58 @@
int fileId = fmp.lookupFileId(file);
bufferCache.openFile(fileId);
fileIds.add(fileId);
-
- } catch(HyracksDataException e) {
+
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
Assert.assertFalse(exceptionThrown);
-
- for(Integer i : fileIds) {
+
+ for (Integer i : fileIds) {
bufferCache.closeFile(i.intValue());
}
-
+
bufferCache.close();
}
-
+
@Test
public void contentCheckingMaxOpenFilesTest() throws HyracksDataException {
TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
IBufferCache bufferCache = TestStorageManagerComponentHolder.getBufferCache(ctx);
IFileMapProvider fmp = TestStorageManagerComponentHolder.getFileMapProvider(ctx);
-
+
List<Integer> fileIds = new ArrayList<Integer>();
- Map<Integer, ArrayList<Integer>> pageContents = new HashMap<Integer, ArrayList<Integer>>();
+ Map<Integer, ArrayList<Integer>> pageContents = new HashMap<Integer, ArrayList<Integer>>();
int num = 10;
- int testPageId = 0;
-
+ int testPageId = 0;
+
// open max number of files and write some stuff into their first page
- for(int i = 0; i < MAX_OPEN_FILES; i++) {
+ for (int i = 0; i < MAX_OPEN_FILES; i++) {
String fileName = getFileName();
FileReference file = new FileReference(new File(fileName));
bufferCache.createFile(file);
int fileId = fmp.lookupFileId(file);
bufferCache.openFile(fileId);
fileIds.add(fileId);
-
- ICachedPage page = null;
+
+ ICachedPage page = null;
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, testPageId), true);
page.acquireWriteLatch();
- try {
- ArrayList<Integer> values = new ArrayList<Integer>();
- for(int j = 0; j < num; j++) {
+ try {
+ ArrayList<Integer> values = new ArrayList<Integer>();
+ for (int j = 0; j < num; j++) {
int x = Math.abs(rnd.nextInt());
page.getBuffer().putInt(j * 4, x);
values.add(x);
}
- pageContents.put(fileId, values);
+ pageContents.put(fileId, values);
} finally {
page.releaseWriteLatch();
bufferCache.unpin(page);
- }
- }
-
+ }
+ }
+
boolean exceptionThrown = false;
-
+
// since all files are open, next open should fail
try {
String fileName = getFileName();
@@ -228,82 +227,82 @@
bufferCache.createFile(file);
int fileId = fmp.lookupFileId(file);
bufferCache.openFile(fileId);
- } catch(HyracksDataException e) {
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
- Assert.assertTrue(exceptionThrown);
-
+ Assert.assertTrue(exceptionThrown);
+
// close a few random files
ArrayList<Integer> closedFileIds = new ArrayList<Integer>();
int filesToClose = 5;
- for(int i = 0; i < filesToClose; i++) {
- int ix = Math.abs(rnd.nextInt()) % fileIds.size();
+ for (int i = 0; i < filesToClose; i++) {
+ int ix = Math.abs(rnd.nextInt()) % fileIds.size();
bufferCache.closeFile(fileIds.get(ix));
closedFileIds.add(fileIds.get(ix));
fileIds.remove(ix);
- }
-
+ }
+
// now open a few new files
- for(int i = 0; i < filesToClose; i++) {
+ for (int i = 0; i < filesToClose; i++) {
String fileName = getFileName();
FileReference file = new FileReference(new File(fileName));
bufferCache.createFile(file);
int fileId = fmp.lookupFileId(file);
bufferCache.openFile(fileId);
fileIds.add(fileId);
- }
-
+ }
+
// since all files are open, next open should fail
try {
String fileName = getFileName();
FileReference file = new FileReference(new File(fileName));
bufferCache.createFile(file);
int fileId = fmp.lookupFileId(file);
- bufferCache.openFile(fileId);
- } catch(HyracksDataException e) {
+ bufferCache.openFile(fileId);
+ } catch (HyracksDataException e) {
exceptionThrown = true;
}
- Assert.assertTrue(exceptionThrown);
-
+ Assert.assertTrue(exceptionThrown);
+
// close a few random files again
- for(int i = 0; i < filesToClose; i++) {
- int ix = Math.abs(rnd.nextInt()) % fileIds.size();
+ for (int i = 0; i < filesToClose; i++) {
+ int ix = Math.abs(rnd.nextInt()) % fileIds.size();
bufferCache.closeFile(fileIds.get(ix));
closedFileIds.add(fileIds.get(ix));
fileIds.remove(ix);
}
-
+
// now open those closed files again and verify their contents
- for(int i = 0; i < filesToClose; i++) {
+ for (int i = 0; i < filesToClose; i++) {
int closedFileId = closedFileIds.get(i);
bufferCache.openFile(closedFileId);
fileIds.add(closedFileId);
-
+
// pin first page and verify contents
- ICachedPage page = null;
+ ICachedPage page = null;
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(closedFileId, testPageId), false);
page.acquireReadLatch();
- try {
- ArrayList<Integer> values = pageContents.get(closedFileId);
- for(int j = 0; j < values.size(); j++) {
- Assert.assertEquals(values.get(j).intValue(), page.getBuffer().getInt(j * 4));
- }
+ try {
+ ArrayList<Integer> values = pageContents.get(closedFileId);
+ for (int j = 0; j < values.size(); j++) {
+ Assert.assertEquals(values.get(j).intValue(), page.getBuffer().getInt(j * 4));
+ }
} finally {
page.releaseReadLatch();
bufferCache.unpin(page);
- }
- }
-
- for(Integer i : fileIds) {
+ }
+ }
+
+ for (Integer i : fileIds) {
bufferCache.closeFile(i.intValue());
}
-
+
bufferCache.close();
}
-
+
@AfterClass
public static void cleanup() throws Exception {
- for(String s : openedFiles) {
+ for (String s : openedFiles) {
File f = new File(s);
f.deleteOnExit();
}