Partial commit. Code compiles, but not complete
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_scheduling@383 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-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
similarity index 74%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
index 2906ba6..dd91f28 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannelMonitor.java
@@ -12,10 +12,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.scheduler;
+package edu.uci.ics.hyracks.api.channels;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+public interface IInputChannelMonitor {
+ public void notifyDataAvailability(IInputChannel channel, int nFrames);
-public interface ISchedule {
- public String[] getPartitions(ActivityNodeId aid);
+ public void notifyEndOfStream(IInputChannel channel);
}
\ 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/IDataReceiveListener.java
deleted file mode 100644
index 463e4e1..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListener.java
+++ /dev/null
@@ -1,23 +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.comm;
-
-import java.io.IOException;
-
-public interface IDataReceiveListener {
- public void dataReceived(IConnectionEntry entry) throws IOException;
-
- public void eos(IConnectionEntry entry);
-}
\ 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/IDataReceiveListenerFactory.java
deleted file mode 100644
index 7e74aee..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IDataReceiveListenerFactory.java
+++ /dev/null
@@ -1,25 +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.comm;
-
-import java.util.UUID;
-
-public interface IDataReceiveListenerFactory {
- public IDataReceiveListener getDataReceiveListener(UUID endpointUUID, IConnectionEntry entry, int senderIndex);
-
- public UUID getJobId();
-
- public UUID getStageId();
-}
\ 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..e7fde8c
--- /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 java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public interface IPartitionCollector {
+ public UUID 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/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
similarity index 84%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
index 59edd97..e15ec6d 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IPartitionWriterFactory.java
@@ -16,6 +16,6 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+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/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-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
index f8fcb89..2e41b0a 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobAttemptSchedulerState.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/IConstraintAcceptor.java
@@ -12,7 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.scheduler;
+package edu.uci.ics.hyracks.api.constraints;
-public interface IJobAttemptSchedulerState {
+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-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
index 2906ba6..7557460 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/constraints/expressions/LValueConstraintExpression.java
@@ -12,10 +12,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.scheduler;
+package edu.uci.ics.hyracks.api.constraints.expressions;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
-
-public interface ISchedule {
- public String[] getPartitions(ActivityNodeId aid);
+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..3901007 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
@@ -26,7 +26,5 @@
public UUID getJobId();
- public int getAttempt();
-
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/IHyracksTaskContext.java
similarity index 82%
rename from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksStageletContext.java
rename to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index ae171fe..a7646fc 100644
--- 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/IHyracksTaskContext.java
@@ -14,16 +14,15 @@
*/
package edu.uci.ics.hyracks.api.context;
-import java.util.UUID;
-
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
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 interface IHyracksTaskContext extends IHyracksCommonContext, IWorkspaceFileFactory, IDeallocatableRegistry {
public IHyracksJobletContext getJobletContext();
- public UUID getStageId();
+ public TaskAttemptId getTaskId();
public ICounterContext getCounterContext();
}
\ No newline at end of file
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/IActivityNode.java
index 39d2eda..9a37661 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/IActivityNode.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.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
@@ -26,6 +26,6 @@
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/IConnectorDescriptor.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IConnectorDescriptor.java
index 2fd317b..1664531 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,19 @@
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.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IEndpointDataWriterFactory;
-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 +60,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 +71,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,7 +91,13 @@
* @param plan
* - Job Plan
*/
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan);
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan);
+
+ /**
+ * 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.
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 e58ac99..b838a80 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
@@ -19,9 +19,9 @@
import org.json.JSONException;
import org.json.JSONObject;
-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.
@@ -73,7 +73,7 @@
* @param plan
* - Job Plan
*/
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan);
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan);
/**
* Translates this operator descriptor to JSON.
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..0b15865
--- /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..9283b8e
--- /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 ActivityNodeId activityId;
+
+ private final int partition;
+
+ public TaskId(ActivityNodeId activityId, int partition) {
+ this.activityId = activityId;
+ this.partition = partition;
+ }
+
+ public ActivityNodeId 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/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/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
new file mode 100644
index 0000000..d2a4f0f
--- /dev/null
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -0,0 +1,204 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+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.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<ActivityNodeId, IActivityNode> activityNodes;
+
+ private final Map<ActivityNodeId, Set<ActivityNodeId>> blocker2blockedMap;
+
+ private final Map<ActivityNodeId, Set<ActivityNodeId>> blocked2blockerMap;
+
+ private final Map<OperatorDescriptorId, Set<ActivityNodeId>> operatorActivityMap;
+
+ private final Map<ActivityNodeId, List<Integer>> activityInputMap;
+
+ private final Map<ActivityNodeId, List<Integer>> activityOutputMap;
+
+ private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorInputMap;
+
+ private final Map<OperatorDescriptorId, List<ActivityNodeId>> operatorOutputMap;
+
+ public JobActivityGraph(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+ this.appName = appName;
+ this.jobSpec = jobSpec;
+ this.jobFlags = jobFlags;
+ activityNodes = new HashMap<ActivityNodeId, IActivityNode>();
+ blocker2blockedMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
+ blocked2blockerMap = new HashMap<ActivityNodeId, Set<ActivityNodeId>>();
+ operatorActivityMap = new HashMap<OperatorDescriptorId, Set<ActivityNodeId>>();
+ activityInputMap = new HashMap<ActivityNodeId, List<Integer>>();
+ activityOutputMap = new HashMap<ActivityNodeId, List<Integer>>();
+ operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
+ operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityNodeId>>();
+ }
+
+ public String getApplicationName() {
+ return appName;
+ }
+
+ 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>> getOperatorActivityMap() {
+ return operatorActivityMap;
+ }
+
+ public Map<ActivityNodeId, List<Integer>> getActivityInputMap() {
+ return activityInputMap;
+ }
+
+ public Map<ActivityNodeId, List<Integer>> getActivityOutputMap() {
+ return activityOutputMap;
+ }
+
+ public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorInputMap() {
+ return operatorInputMap;
+ }
+
+ public Map<OperatorDescriptorId, List<ActivityNodeId>> getOperatorOutputMap() {
+ return operatorOutputMap;
+ }
+
+ public List<IConnectorDescriptor> getActivityInputConnectorDescriptors(ActivityNodeId 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(ActivityNodeId 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 ActivityNodeId 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 (ActivityNodeId 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 ActivityNodeId 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 (ActivityNodeId 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(ActivityNodeId hanId, int inputIndex) {
+ int opInputIndex = getActivityInputMap().get(hanId).get(inputIndex);
+ return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
+ }
+
+ public RecordDescriptor getActivityOutputRecordDescriptor(ActivityNodeId 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();
+ }
+}
\ 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..ab5397d 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,7 +26,7 @@
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;
@@ -51,7 +51,7 @@
private final Map<String, Serializable> properties;
- private final Set<ConstraintExpression> userConstraints;
+ private final Set<Constraint> userConstraints;
private int maxAttempts;
@@ -63,7 +63,7 @@
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>();
}
public void addRoot(IOperatorDescriptor op) {
@@ -181,11 +181,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/comm/IEndpointDataWriterFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
similarity index 68%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
copy to hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
index 59edd97..02eb891 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/IPartition.java
@@ -12,10 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.comm;
+package edu.uci.ics.hyracks.api.partitions;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+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..3eafbd6
--- /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 java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+
+public final class PartitionId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final UUID jobId;
+
+ private final ConnectorDescriptorId cdId;
+
+ private final int senderIndex;
+
+ private final int receiverIndex;
+
+ public PartitionId(UUID jobId, ConnectorDescriptorId cdId, int senderIndex, int receiverIndex) {
+ this.jobId = jobId;
+ this.cdId = cdId;
+ this.senderIndex = senderIndex;
+ this.receiverIndex = receiverIndex;
+ }
+
+ public UUID 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-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 c550ecc..aebd6ce 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
@@ -17,6 +17,7 @@
import java.io.File;
import java.rmi.registry.LocateRegistry;
import java.rmi.registry.Registry;
+import java.util.Collection;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.Hashtable;
@@ -33,9 +34,12 @@
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.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.JobStatus;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
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;
@@ -47,15 +51,17 @@
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.scheduler.DefaultJobScheduler;
+import edu.uci.ics.hyracks.control.cc.scheduler.IJobScheduler;
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.CCConfig;
@@ -64,7 +70,7 @@
import edu.uci.ics.hyracks.control.common.base.NodeParameters;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
public class ClusterControllerService extends AbstractRemoteService implements IClusterController,
IHyracksClientInterface {
@@ -88,7 +94,7 @@
private final JobQueue jobQueue;
- private final IScheduler scheduler;
+ private final IJobScheduler scheduler;
private final Executor taskExecutor;
@@ -106,7 +112,7 @@
webServer = new WebServer(this);
runMap = new HashMap<UUID, JobRun>();
jobQueue = new JobQueue();
- scheduler = new NaiveScheduler(this);
+ scheduler = new DefaultJobScheduler(this);
this.timer = new Timer(true);
ccci = new CCClientInterface(this);
}
@@ -144,7 +150,7 @@
return jobQueue;
}
- public IScheduler getScheduler() {
+ public IJobScheduler getScheduler() {
return scheduler;
}
@@ -191,15 +197,16 @@
}
@Override
- public void notifyStageletComplete(UUID jobId, UUID stageId, int attempt, String nodeId, StageletProfile statistics)
+ public void notifyTaskComplete(UUID 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);
+ public void notifyTaskFailure(UUID jobId, TaskAttemptId taskId, String nodeId, Exception exception)
+ throws Exception {
+ TaskFailureEvent sfe = new TaskFailureEvent(this, jobId, taskId, nodeId, exception);
jobQueue.schedule(sfe);
}
@@ -213,7 +220,7 @@
@Override
public void start(UUID jobId) throws Exception {
JobStartEvent jse = new JobStartEvent(this, jobId);
- jobQueue.scheduleAndSync(jse);
+ jobQueue.schedule(jse);
}
@Override
@@ -269,6 +276,16 @@
return info;
}
+ @Override
+ public void registerPartitionProvider(PartitionId pid, NetworkAddress address) throws Exception {
+ jobQueue.schedule(new RegisterPartitionAvailibilityEvent(this, pid, address));
+ }
+
+ @Override
+ public void registerPartitionRequest(Collection<PartitionId> requiredPartitionIds, String nodeId) {
+ jobQueue.schedule(new RegisterPartitionRequestEvent(this, requiredPartitionIds, nodeId));
+ }
+
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 6763451..e614bc5 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,21 +1,14 @@
package edu.uci.ics.hyracks.control.cc;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.UUID;
-
import edu.uci.ics.hyracks.control.common.base.INodeController;
public class NodeControllerState {
private final INodeController nodeController;
- private final Set<UUID> activeJobIds;
-
private int lastHeartbeatDuration;
public NodeControllerState(INodeController nodeController) {
this.nodeController = nodeController;
- activeJobIds = new HashSet<UUID>();
}
public void notifyHeartbeat() {
@@ -33,8 +26,4 @@
public INodeController getNodeController() {
return nodeController;
}
-
- public Set<UUID> getActiveJobIds() {
- return activeJobIds;
- }
}
\ No newline at end of file
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..1e9261b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.control.cc.scheduler.IActivityClusterStateMachine;
+
+public class ActivityCluster {
+ private final JobRun jobRun;
+
+ private final Set<ActivityNodeId> activities;
+
+ private final Set<ActivityCluster> dependencies;
+
+ private final Set<ActivityCluster> dependents;
+
+ private final Map<ActivityNodeId, TaskState[]> taskStateMap;
+
+ private TaskCluster[] taskClusters;
+
+ private IActivityClusterStateMachine acsm;
+
+ public ActivityCluster(JobRun jobRun, Set<ActivityNodeId> activities) {
+ this.jobRun = jobRun;
+ this.activities = activities;
+ dependencies = new HashSet<ActivityCluster>();
+ dependents = new HashSet<ActivityCluster>();
+ taskStateMap = new HashMap<ActivityNodeId, TaskState[]>();
+ }
+
+ public Set<ActivityNodeId> 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 Map<ActivityNodeId, TaskState[]> getTaskStateMap() {
+ return taskStateMap;
+ }
+
+ public TaskCluster[] getTaskClusters() {
+ return taskClusters;
+ }
+
+ public void setTaskClusters(TaskCluster[] taskClusters) {
+ this.taskClusters = taskClusters;
+ }
+
+ public IActivityClusterStateMachine getStateMachine() {
+ return acsm;
+ }
+
+ public void setStateMachine(IActivityClusterStateMachine acsm) {
+ this.acsm = acsm;
+ }
+
+ public JobRun getJobRun() {
+ return jobRun;
+ }
+}
\ 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/JobActivityGraphBuilder.java
similarity index 62%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobPlanBuilder.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
index bafa7e5..9f9a2e3 100644
--- 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/JobActivityGraphBuilder.java
@@ -6,25 +6,24 @@
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.JobActivityGraph;
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());
+public class JobActivityGraphBuilder implements IActivityGraphBuilder {
+ private static final Logger LOGGER = Logger.getLogger(JobActivityGraphBuilder.class.getName());
- private JobPlan plan;
+ private JobActivityGraph jag;
@Override
public void addBlockingEdge(IActivityNode blocker, IActivityNode blocked) {
- addToValueSet(plan.getBlocker2BlockedMap(), blocker.getActivityNodeId(), blocked.getActivityNodeId());
- addToValueSet(plan.getBlocked2BlockerMap(), blocked.getActivityNodeId(), blocker.getActivityNodeId());
+ addToValueSet(jag.getBlocker2BlockedMap(), blocker.getActivityNodeId(), blocked.getActivityNodeId());
+ addToValueSet(jag.getBlocked2BlockerMap(), blocked.getActivityNodeId(), blocker.getActivityNodeId());
}
@Override
@@ -33,8 +32,8 @@
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,
+ insertIntoIndexedMap(jag.getActivityInputMap(), task.getActivityNodeId(), taskInputIndex, operatorInputIndex);
+ insertIntoIndexedMap(jag.getOperatorInputMap(), task.getOwner().getOperatorId(), operatorInputIndex,
task.getActivityNodeId());
}
@@ -44,15 +43,15 @@
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,
+ insertIntoIndexedMap(jag.getActivityOutputMap(), task.getActivityNodeId(), taskOutputIndex, operatorOutputIndex);
+ insertIntoIndexedMap(jag.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());
+ jag.getActivityNodeMap().put(task.getActivityNodeId(), task);
+ addToValueSet(jag.getOperatorActivityMap(), task.getOwner().getOperatorId(), task.getActivityNodeId());
}
private <K, V> void addToValueSet(Map<K, Set<V>> map, K n1, V n2) {
@@ -71,8 +70,8 @@
}
}
- public void init(String appName, UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
- plan = new JobPlan(appName, jobId, jobSpec, jobFlags);
+ 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) {
@@ -85,7 +84,7 @@
vList.set(index, value);
}
- public JobPlan getPlan() {
- return plan;
+ 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 5f5b89e..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.control.cc.scheduler.IJobAttemptSchedulerState;
-import edu.uci.ics.hyracks.control.cc.scheduler.IScheduler;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-
-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/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..fb7dbf8 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,72 @@
*/
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 java.util.UUID;
-import edu.uci.ics.hyracks.api.constraints.expressions.ConstraintExpression;
-import edu.uci.ics.hyracks.api.job.JobPlan;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
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.scheduler.IJobRunStateMachine;
+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 UUID jobId;
+
+ private final JobActivityGraph jag;
+
+ private final Map<PartitionId, NetworkAddress> partitionAvailabilityMap;
+
+ private final Map<PartitionId, String> partitionRequestorMap;
+
+ private final Set<String> participatingNodeIds;
+
+ private final JobProfile profile;
+
+ private final Map<ActivityNodeId, ActivityCluster> activityClusterMap;
+
+ private IJobRunStateMachine jsm;
+
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(UUID jobId, JobActivityGraph plan) {
+ this.jobId = jobId;
+ this.jag = plan;
+ partitionAvailabilityMap = new HashMap<PartitionId, NetworkAddress>();
+ partitionRequestorMap = new HashMap<PartitionId, String>();
+ participatingNodeIds = new HashSet<String>();
+ profile = new JobProfile(jobId);
+ activityClusterMap = new HashMap<ActivityNodeId, ActivityCluster>();
}
- public JobPlan getJobPlan() {
- return plan;
+ public UUID getJobId() {
+ return jobId;
}
- public synchronized void setStatus(JobStatus status) {
+ public JobActivityGraph getJobActivityGraph() {
+ return jag;
+ }
+
+ 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 +87,36 @@
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 setStateMachine(IJobRunStateMachine jsm) {
+ this.jsm = jsm;
+ }
+
+ public IJobRunStateMachine getStateMachine() {
+ return jsm;
+ }
+
+ public Map<PartitionId, NetworkAddress> getPartitionAvailabilityMap() {
+ return partitionAvailabilityMap;
+ }
+
+ public Map<PartitionId, String> getPartitionRequestorMap() {
+ return partitionRequestorMap;
+ }
+
+ public Map<ActivityNodeId, ActivityCluster> getActivityClusterMap() {
+ return activityClusterMap;
}
}
\ 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/TaskAttempt.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
new file mode 100644
index 0000000..6d61ccb
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskAttempt.java
@@ -0,0 +1,80 @@
+/*
+ * 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;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public class TaskAttempt {
+ public enum TaskStatus {
+ INITIALIZED,
+ RUNNING,
+ COMPLETED,
+ FAILED,
+ ABORTED,
+ }
+
+ private final TaskAttemptId taskId;
+
+ private final TaskState taskState;
+
+ private String nodeId;
+
+ private TaskStatus status;
+
+ private Exception exception;
+
+ public TaskAttempt(TaskAttemptId taskId, TaskState taskState) {
+ this.taskId = taskId;
+ this.taskState = taskState;
+ }
+
+ public TaskAttemptId getTaskAttemptId() {
+ return taskId;
+ }
+
+ public TaskState 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;
+ }
+
+ public void notifyTaskComplete() throws HyracksException {
+ taskState.getTaskCluster().notifyTaskComplete(this);
+ }
+
+ public void notifyTaskFailure(Exception exception) throws HyracksException {
+ taskState.getTaskCluster().notifyTaskFailure(this, 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..78ac34b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskCluster.java
@@ -0,0 +1,66 @@
+/*
+ * 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.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public class TaskCluster {
+ private final ActivityCluster activityCluster;
+
+ private final TaskState[] tasks;
+
+ private final Set<TaskCluster> blockers;
+
+ private final Set<TaskCluster> dependencies;
+
+ private final List<TaskClusterAttempt> taskClusterAttempts;
+
+ public TaskCluster(ActivityCluster activityCluster, TaskState[] tasks) {
+ this.activityCluster = activityCluster;
+ this.tasks = tasks;
+ this.blockers = new HashSet<TaskCluster>();
+ this.dependencies = new HashSet<TaskCluster>();
+ taskClusterAttempts = new ArrayList<TaskClusterAttempt>();
+ }
+
+ public TaskState[] getTasks() {
+ return tasks;
+ }
+
+ public Set<TaskCluster> getDependencies() {
+ return dependencies;
+ }
+
+ public Set<TaskCluster> getBlockers() {
+ return blockers;
+ }
+
+ public List<TaskClusterAttempt> getAttempts() {
+ return taskClusterAttempts;
+ }
+
+ public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
+ activityCluster.getStateMachine().notifyTaskComplete(ta);
+ }
+
+ public void notifyTaskFailure(TaskAttempt ta, Exception exception) throws HyracksException {
+ activityCluster.getStateMachine().notifyTaskFailure(ta, exception);
+ }
+}
\ 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..f0d951e
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskClusterAttempt.java
@@ -0,0 +1,59 @@
+/*
+ * 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 {
+ INITIALIZED,
+ RUNNING,
+ COMPLETED,
+ FAILED,
+ }
+
+ private final TaskAttempt[] taskAttempts;
+
+ private TaskClusterStatus status;
+
+ private int pendingTaskCounter;
+
+ public TaskClusterAttempt(TaskAttempt[] taskAttempts) {
+ this.taskAttempts = taskAttempts;
+ }
+
+ public TaskAttempt[] getTaskAttempts() {
+ return taskAttempts;
+ }
+
+ 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/TaskState.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskState.java
new file mode 100644
index 0000000..3bbba36
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/TaskState.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.api.dataflow.TaskId;
+
+public class TaskState {
+ private final TaskId taskId;
+
+ private TaskCluster taskCluster;
+
+ public TaskState(TaskId taskId) {
+ this.taskId = taskId;
+ }
+
+ public TaskId getTaskId() {
+ return taskId;
+ }
+
+ public TaskCluster getTaskCluster() {
+ return taskCluster;
+ }
+
+ public void setTaskCluster(TaskCluster taskCluster) {
+ this.taskCluster = taskCluster;
+ }
+}
\ 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/GetJobProfileJSONEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/GetJobProfileJSONEvent.java
index dc397ef..f53fb5f 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,14 +14,11 @@
*/
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.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;
public class GetJobProfileJSONEvent extends SynchronizableRunnable {
@@ -38,17 +35,7 @@
@Override
protected void doRun() throws Exception {
- JobRun run = ccs.getRunMap().get(jobId);
profile = new JSONObject();
- if (run == null) {
- return;
- }
- List<JobAttempt> attempts = run.getAttempts();
- if (attempts.size() <= attempt) {
- return;
- }
- JobAttempt ja = attempts.get(attempt);
- profile = ja.getJobProfile().toJSON();
}
public JSONObject getProfile() {
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..509dcd2 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
@@ -39,7 +39,7 @@
spec = new JSONObject();
return;
}
- spec = run.getJobPlan().getJobSpecification().toJSON();
+ spec = run.getJobActivityGraph().getJobSpecification().toJSON();
}
public JSONObject getSpecification() {
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..3c19f59 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
@@ -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/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..87ba685 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,16 +14,13 @@
*/
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.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.remote.RemoteRunner;
import edu.uci.ics.hyracks.control.cc.remote.ops.JobCompleteNotifier;
@@ -31,21 +28,20 @@
public class JobCleanupEvent implements Runnable {
private ClusterControllerService ccs;
private UUID jobId;
- private int attempt;
private JobStatus status;
+ private Exception exception;
- public JobCleanupEvent(ClusterControllerService ccs, UUID jobId, int attempt, JobStatus status) {
+ public JobCleanupEvent(ClusterControllerService ccs, UUID 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) {
@@ -64,7 +60,8 @@
ccs.getJobQueue().schedule(new Runnable() {
@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 +69,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 3b03d25..d6b09db 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,24 +15,18 @@
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.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;
@@ -60,51 +54,27 @@
throw new HyracksException("No application with id " + appName + " found");
}
JobSpecification spec = appCtx.createJobSpecification(jobId, jobSpec);
- JobRun run = plan(jobId, spec, jobFlags);
- run.setStatus(JobStatus.INITIALIZED);
- ccs.getRunMap().put(jobId, run);
- appCtx.notifyJobCreation(jobId, spec);
- }
-
- public UUID getJobId() {
- return jobId;
- }
-
- private JobRun plan(UUID jobId, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- final JobPlanBuilder builder = new JobPlanBuilder();
- builder.init(appName, jobId, jobSpec, jobFlags);
- PlanUtils.visit(jobSpec, new IOperatorDescriptorVisitor() {
+ final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
+ builder.init(appName, spec, jobFlags);
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
@Override
public void visit(IOperatorDescriptor op) {
op.contributeTaskGraph(builder);
}
});
- final JobPlan plan = builder.getPlan();
+ final JobActivityGraph jag = builder.getActivityGraph();
- 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);
- }
- });
- PlanUtils.visit(jobSpec, new IConnectorDescriptorVisitor() {
- @Override
- public void visit(IConnectorDescriptor conn) {
- conn.contributeSchedulingConstraints(acceptor, plan);
- }
- });
- contributedConstraints.addAll(jobSpec.getUserConstraints());
+ JobRun run = new JobRun(UUID.randomUUID(), jag);
- JobRun run = new JobRun(plan, contributedConstraints);
+ run.setStatus(JobStatus.INITIALIZED, null);
- return run;
+ ccs.getRunMap().put(jobId, run);
+ ccs.getScheduler().notifyJobCreation(run);
+ appCtx.notifyJobCreation(jobId, spec);
+ }
+
+ public UUID getJobId() {
+ return 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/JobStartEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/JobStartEvent.java
index 742885e..7b9a3c6 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
@@ -39,8 +39,7 @@
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);
+ run.getStateMachine().schedule();
}
}
\ No newline at end of file
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..0269122
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionAvailibilityEvent.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.cc.job.manager.events;
+
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+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.common.base.INodeController;
+
+public class RegisterPartitionAvailibilityEvent implements Runnable {
+ private final ClusterControllerService ccs;
+ private final PartitionId pid;
+ private final NetworkAddress networkAddress;
+
+ public RegisterPartitionAvailibilityEvent(ClusterControllerService ccs, PartitionId pid,
+ NetworkAddress networkAddress) {
+ this.ccs = ccs;
+ this.pid = pid;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public void run() {
+ JobRun run = ccs.getRunMap().get(pid.getJobId());
+ if (run == null) {
+ return;
+ }
+ Map<PartitionId, NetworkAddress> partitionAvailabilityMap = run.getPartitionAvailabilityMap();
+ partitionAvailabilityMap.put(pid, networkAddress);
+
+ Map<PartitionId, String> partitionRequestorMap = run.getPartitionRequestorMap();
+ String requestor = partitionRequestorMap.get(pid);
+ if (requestor != null) {
+ NodeControllerState ncs = ccs.getNodeMap().get(requestor);
+ if (ncs != null) {
+ try {
+ INodeController nc = ncs.getNodeController();
+ nc.reportPartitionAvailability(pid, networkAddress);
+ } 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/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..4d34e6e
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RegisterPartitionRequestEvent.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.cc.job.manager.events;
+
+import java.util.Collection;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+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.common.base.INodeController;
+
+public class RegisterPartitionRequestEvent implements Runnable {
+ private final ClusterControllerService ccs;
+ private final Collection<PartitionId> requiredPartitionIds;
+ private final String nodeId;
+
+ public RegisterPartitionRequestEvent(ClusterControllerService ccs, Collection<PartitionId> requiredPartitionIds,
+ String nodeId) {
+ this.ccs = ccs;
+ this.requiredPartitionIds = requiredPartitionIds;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public void run() {
+ NodeControllerState ncs = ccs.getNodeMap().get(nodeId);
+ if (ncs != null) {
+ for (PartitionId pid : requiredPartitionIds) {
+ JobRun run = ccs.getRunMap().get(pid.getJobId());
+ if (run == null) {
+ return;
+ }
+ Map<PartitionId, String> partitionRequestorMap = run.getPartitionRequestorMap();
+ partitionRequestorMap.put(pid, nodeId);
+
+ Map<PartitionId, NetworkAddress> partitionAvailabilityMap = run.getPartitionAvailabilityMap();
+ NetworkAddress networkAddress = partitionAvailabilityMap.get(pid);
+ if (networkAddress != null) {
+ try {
+ INodeController nc = ncs.getNodeController();
+ nc.reportPartitionAvailability(pid, networkAddress);
+ } 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/RemoveDeadNodesEvent.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/RemoveDeadNodesEvent.java
index 38fb7ae..8a1703b 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,12 +17,10 @@
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.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
public class RemoveDeadNodesEvent implements Runnable {
private static Logger LOGGER = Logger.getLogger(RemoveDeadNodesEvent.class.getName());
@@ -46,12 +44,7 @@
}
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.
}
}
}
\ 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 beb8732..24f884a 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
@@ -19,7 +19,6 @@
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.common.job.profiling.om.JobProfile;
@@ -37,10 +36,6 @@
Map<UUID, 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);
- }
}
}
}
\ 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..f9fa355
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskCompleteEvent.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.manager.events;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+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.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+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.TaskState;
+
+public class TaskCompleteEvent implements Runnable {
+ private final ClusterControllerService ccs;
+ private final UUID jobId;
+ private final TaskAttemptId taId;
+ private final String nodeId;
+
+ public TaskCompleteEvent(ClusterControllerService ccs, UUID jobId, TaskAttemptId taId, String nodeId) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.taId = taId;
+ this.nodeId = nodeId;
+ }
+
+ @Override
+ public void run() {
+ JobRun run = ccs.getRunMap().get(jobId);
+ if (run != null) {
+ TaskId tid = taId.getTaskId();
+ Map<ActivityNodeId, ActivityCluster> activityClusterMap = run.getActivityClusterMap();
+ ActivityCluster ac = activityClusterMap.get(tid.getActivityId());
+ if (ac != null) {
+ Map<ActivityNodeId, TaskState[]> taskStateMap = ac.getTaskStateMap();
+ TaskState[] taskStates = taskStateMap.get(tid.getActivityId());
+ if (taskStates != null && taskStates.length > tid.getPartition()) {
+ TaskState 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());
+ TaskAttempt ta = tca.getTaskAttempts()[tid.getPartition()];
+ try {
+ ta.notifyTaskComplete();
+ } catch (HyracksException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "TaskCompleteEvent[" + 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/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..710c2b1
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/manager/events/TaskFailureEvent.java
@@ -0,0 +1,80 @@
+/*
+ * 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 java.util.UUID;
+
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+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.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+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.TaskState;
+
+public class TaskFailureEvent implements Runnable {
+ private final ClusterControllerService ccs;
+ private final UUID jobId;
+ private final TaskAttemptId taId;
+ private final String nodeId;
+ private final Exception exception;
+
+ public TaskFailureEvent(ClusterControllerService ccs, UUID jobId, TaskAttemptId taId, String nodeId, Exception exception) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.taId = taId;
+ this.nodeId = nodeId;
+ this.exception = exception;
+ }
+
+ @Override
+ public void run() {
+ JobRun run = ccs.getRunMap().get(jobId);
+ if (run != null) {
+ TaskId tid = taId.getTaskId();
+ Map<ActivityNodeId, ActivityCluster> activityClusterMap = run.getActivityClusterMap();
+ ActivityCluster ac = activityClusterMap.get(tid.getActivityId());
+ if (ac != null) {
+ Map<ActivityNodeId, TaskState[]> taskStateMap = ac.getTaskStateMap();
+ TaskState[] taskStates = taskStateMap.get(tid.getActivityId());
+ if (taskStates != null && taskStates.length > tid.getPartition()) {
+ TaskState 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());
+ TaskAttempt ta = tca.getTaskAttempts()[tid.getPartition()];
+ try {
+ ta.notifyTaskFailure(exception);
+ } catch (HyracksException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "TaskCompleteEvent[" + jobId + ":" + taId + ":" + nodeId + "]";
+ }
+}
\ No newline at end of file
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 7885dcd..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.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-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 0373864..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.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;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-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 3f1c251..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.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;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-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 3d98f0f..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.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-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/StageStarter.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/remote/ops/StageStarter.java
deleted file mode 100644
index b614235..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.control.cc.remote.RemoteOp;
-import edu.uci.ics.hyracks.control.common.base.INodeController;
-
-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/DefaultActivityClusterStateMachine.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultActivityClusterStateMachine.java
new file mode 100644
index 0000000..1e68f76
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultActivityClusterStateMachine.java
@@ -0,0 +1,259 @@
+/*
+ * 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.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Executor;
+import java.util.logging.Logger;
+
+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.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+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.JobRun;
+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.TaskState;
+
+public class DefaultActivityClusterStateMachine implements IActivityClusterStateMachine {
+ private static final Logger LOGGER = Logger.getLogger(DefaultActivityClusterStateMachine.class.getName());
+
+ private final ClusterControllerService ccs;
+
+ private final DefaultJobRunStateMachine jsm;
+
+ private final ActivityCluster ac;
+
+ public DefaultActivityClusterStateMachine(ClusterControllerService ccs, DefaultJobRunStateMachine jsm,
+ ActivityCluster ac) {
+ this.ccs = ccs;
+ this.jsm = jsm;
+ this.ac = ac;
+ }
+
+ @Override
+ public void schedule() throws HyracksException {
+ startRunnableTaskClusters();
+ }
+
+ private void assignTaskLocations(TaskCluster tc, Map<String, List<TaskAttemptId>> taskAttemptMap)
+ throws HyracksException {
+ TaskState[] tasks = tc.getTasks();
+ List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
+ int attempts = tcAttempts.size();
+ TaskAttempt[] taskAttempts = new TaskAttempt[tasks.length];
+ Map<TaskId, LValueConstraintExpression> locationMap = new HashMap<TaskId, LValueConstraintExpression>();
+ for (int i = 0; i < tasks.length; ++i) {
+ TaskState ts = tasks[i];
+ TaskId tid = ts.getTaskId();
+ TaskAttempt taskAttempt = new TaskAttempt(new TaskAttemptId(new TaskId(tid.getActivityId(),
+ tid.getPartition()), attempts), ts);
+ locationMap.put(tid,
+ new PartitionLocationExpression(tid.getActivityId().getOperatorDescriptorId(), tid.getPartition()));
+ taskAttempts[i] = taskAttempt;
+ }
+ PartitionConstraintSolver solver = jsm.getSolver();
+ solver.solve(locationMap.values());
+ for (int i = 0; i < tasks.length; ++i) {
+ TaskState ts = tasks[i];
+ TaskId tid = ts.getTaskId();
+ TaskAttempt taskAttempt = taskAttempts[i];
+ LValueConstraintExpression pLocationExpr = locationMap.get(tid);
+ Object location = solver.getValue(pLocationExpr);
+ String nodeId = null;
+ Set<String> liveNodes = ccs.getNodeMap().keySet();
+ 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;
+ if (!liveNodes.contains(nodeId)) {
+ throw new HyracksException("Node " + nodeId + " not live");
+ }
+ } 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() + ")");
+ }
+ taskAttempt.setNodeId(nodeId);
+ List<TaskAttemptId> taIds = taskAttemptMap.get(nodeId);
+ if (taIds == null) {
+ taIds = new ArrayList<TaskAttemptId>();
+ taskAttemptMap.put(nodeId, taIds);
+ }
+ taIds.add(taskAttempt.getTaskAttemptId());
+ }
+ TaskClusterAttempt tcAttempt = new TaskClusterAttempt(taskAttempts);
+ tcAttempt.initializePendingTaskCounter();
+ tcAttempts.add(tcAttempt);
+
+ }
+
+ @Override
+ public void notifyTaskComplete(TaskAttempt ta) throws HyracksException {
+ TaskAttemptId taId = ta.getTaskAttemptId();
+ TaskCluster tc = ta.getTaskState().getTaskCluster();
+ List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
+ int lastAttempt = tcAttempts.size() - 1;
+ if (taId.getAttempt() == lastAttempt) {
+ TaskClusterAttempt tcAttempt = tcAttempts.get(lastAttempt);
+ TaskAttempt.TaskStatus taStatus = ta.getStatus();
+ if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+ ta.setStatus(TaskAttempt.TaskStatus.COMPLETED, null);
+ if (tcAttempt.decrementPendingTasksCounter() == 0) {
+ tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.COMPLETED);
+ startRunnableTaskClusters();
+ }
+ } else {
+ LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
+ }
+ } else {
+ LOGGER.warning("Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+ }
+ }
+
+ private void startRunnableTaskClusters() throws HyracksException {
+ TaskCluster[] taskClusters = ac.getTaskClusters();
+
+ Map<String, List<TaskAttemptId>> taskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+ for (TaskCluster tc : taskClusters) {
+ Set<TaskCluster> dependencies = tc.getDependencies();
+ boolean runnable = true;
+ for (TaskCluster depTC : dependencies) {
+ List<TaskClusterAttempt> tcAttempts = depTC.getAttempts();
+ if (tcAttempts.isEmpty()) {
+ runnable = false;
+ break;
+ }
+ TaskClusterAttempt tcAttempt = tcAttempts.get(tcAttempts.size() - 1);
+ if (tcAttempt.getStatus() != TaskClusterAttempt.TaskClusterStatus.COMPLETED) {
+ runnable = false;
+ break;
+ }
+ }
+ if (runnable) {
+ assignTaskLocations(tc, taskAttemptMap);
+ }
+ }
+
+ startTasks(taskAttemptMap);
+ }
+
+ private void startTasks(Map<String, List<TaskAttemptId>> taskAttemptMap) {
+ Executor executor = ccs.getExecutor();
+ JobRun jobRun = ac.getJobRun();
+ final UUID jobId = jobRun.getJobId();
+ final JobActivityGraph jag = jobRun.getJobActivityGraph();
+ final String appName = jag.getApplicationName();
+ for (Map.Entry<String, List<TaskAttemptId>> e : taskAttemptMap.entrySet()) {
+ String nodeId = e.getKey();
+ final List<TaskAttemptId> taskIds = e.getValue();
+ final NodeControllerState node = ccs.getNodeMap().get(nodeId);
+ if (node != null) {
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ node.getNodeController().startTasks(appName, jobId, JavaSerializationUtils.serialize(jag),
+ taskIds, null);
+ } catch (IOException e) {
+ e.printStackTrace();
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+ }
+ }
+
+ @Override
+ public void notifyTaskFailure(TaskAttempt ta, Exception exception) throws HyracksException {
+ TaskAttemptId taId = ta.getTaskAttemptId();
+ TaskCluster tc = ta.getTaskState().getTaskCluster();
+ List<TaskClusterAttempt> tcAttempts = tc.getAttempts();
+ int lastAttempt = tcAttempts.size() - 1;
+ if (taId.getAttempt() == lastAttempt) {
+ TaskClusterAttempt tcAttempt = tcAttempts.get(lastAttempt);
+ TaskAttempt.TaskStatus taStatus = ta.getStatus();
+ Map<String, List<TaskAttemptId>> abortTaskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+ if (taStatus == TaskAttempt.TaskStatus.RUNNING) {
+ ta.setStatus(TaskAttempt.TaskStatus.FAILED, exception);
+ for (TaskAttempt ta2 : tcAttempt.getTaskAttempts()) {
+ if (ta2.getStatus() == TaskAttempt.TaskStatus.RUNNING
+ || ta2.getStatus() == TaskAttempt.TaskStatus.INITIALIZED) {
+ ta2.setStatus(TaskAttempt.TaskStatus.ABORTED, null);
+ List<TaskAttemptId> abortTaskAttempts = abortTaskAttemptMap.get(ta2.getNodeId());
+ if (abortTaskAttempts == null) {
+ abortTaskAttempts = new ArrayList<TaskAttemptId>();
+ abortTaskAttemptMap.put(ta2.getNodeId(), abortTaskAttempts);
+ }
+ abortTaskAttempts.add(ta2.getTaskAttemptId());
+ }
+ }
+ JobRun jobRun = ac.getJobRun();
+ final UUID 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) {
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ node.getNodeController().abortTasks(jobId, abortTaskAttempts);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ });
+ }
+ }
+ tcAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
+ Map<String, List<TaskAttemptId>> taskAttemptMap = new HashMap<String, List<TaskAttemptId>>();
+ assignTaskLocations(tc, taskAttemptMap);
+ startTasks(taskAttemptMap);
+ } else {
+ LOGGER.warning("Spurious task complete notification: " + taId + " Current state = " + taStatus);
+ }
+ } else {
+ LOGGER.warning("Ignoring task complete notification: " + taId + " -- Current last attempt = " + lastAttempt);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobRunStateMachine.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobRunStateMachine.java
new file mode 100644
index 0000000..9e70b42
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobRunStateMachine.java
@@ -0,0 +1,457 @@
+/*
+ * 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.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.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.PartitionCountExpression;
+import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+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.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.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.api.job.JobStatus;
+import edu.uci.ics.hyracks.api.util.Pair;
+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.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.TaskAttempt;
+import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
+import edu.uci.ics.hyracks.control.cc.job.TaskState;
+import edu.uci.ics.hyracks.control.cc.job.manager.events.JobCleanupEvent;
+import edu.uci.ics.hyracks.control.common.job.dataflow.IConnectorPolicy;
+
+public class DefaultJobRunStateMachine implements IJobRunStateMachine {
+ private static final Logger LOGGER = Logger.getLogger(DefaultJobRunStateMachine.class.getName());
+
+ private final ClusterControllerService ccs;
+
+ private final JobRun jobRun;
+
+ private final Set<ActivityCluster> completedClusters;
+
+ private final Set<ActivityCluster> inProgressClusters;
+
+ private PartitionConstraintSolver solver;
+
+ private ActivityCluster rootActivityCluster;
+
+ public DefaultJobRunStateMachine(ClusterControllerService ccs, JobRun jobRun) {
+ this.ccs = ccs;
+ this.jobRun = jobRun;
+ completedClusters = new HashSet<ActivityCluster>();
+ inProgressClusters = new HashSet<ActivityCluster>();
+ }
+
+ public PartitionConstraintSolver getSolver() {
+ return solver;
+ }
+
+ private static Pair<ActivityNodeId, ActivityNodeId> findMergePair(JobActivityGraph jag, JobSpecification spec,
+ Set<ActivityCluster> eqSets) {
+ Map<ActivityNodeId, IActivityNode> activityNodeMap = jag.getActivityNodeMap();
+ for (ActivityCluster eqSet : eqSets) {
+ for (ActivityNodeId t : eqSet.getActivities()) {
+ IOperatorDescriptor owner = activityNodeMap.get(t).getOwner();
+ List<Integer> inputList = jag.getActivityInputMap().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 = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
+ if (!eqSet.getActivities().contains(inTask)) {
+ return new Pair<ActivityNodeId, ActivityNodeId>(t, inTask);
+ }
+ }
+ }
+ List<Integer> outputList = jag.getActivityOutputMap().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 = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
+ if (!eqSet.getActivities().contains(outTask)) {
+ return new Pair<ActivityNodeId, ActivityNodeId>(t, outTask);
+ }
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ private ActivityCluster inferStages(JobActivityGraph jag) {
+ JobSpecification spec = jag.getJobSpecification();
+
+ /*
+ * Build initial equivalence sets map. We create a map such that for each IOperatorTask, t -> { t }
+ */
+ Map<ActivityNodeId, ActivityCluster> stageMap = new HashMap<ActivityNodeId, ActivityCluster>();
+ Set<ActivityCluster> stages = new HashSet<ActivityCluster>();
+ for (Set<ActivityNodeId> taskIds : jag.getOperatorActivityMap().values()) {
+ for (ActivityNodeId taskId : taskIds) {
+ Set<ActivityNodeId> eqSet = new HashSet<ActivityNodeId>();
+ eqSet.add(taskId);
+ ActivityCluster stage = new ActivityCluster(jobRun, eqSet);
+ stageMap.put(taskId, stage);
+ stages.add(stage);
+ }
+ }
+
+ boolean changed = true;
+ while (changed) {
+ changed = false;
+ Pair<ActivityNodeId, ActivityNodeId> pair = findMergePair(jag, spec, stages);
+ if (pair != null) {
+ merge(stageMap, stages, pair.first, pair.second);
+ changed = true;
+ }
+ }
+
+ ActivityCluster endStage = new ActivityCluster(jobRun, new HashSet<ActivityNodeId>());
+ Map<ActivityNodeId, Set<ActivityNodeId>> blocker2BlockedMap = jag.getBlocker2BlockedMap();
+ for (ActivityCluster s : stages) {
+ endStage.addDependency(s);
+ s.addDependent(endStage);
+ Set<ActivityCluster> blockedStages = new HashSet<ActivityCluster>();
+ for (ActivityNodeId t : s.getActivities()) {
+ Set<ActivityNodeId> blockedTasks = blocker2BlockedMap.get(t);
+ if (blockedTasks != null) {
+ for (ActivityNodeId bt : blockedTasks) {
+ blockedStages.add(stageMap.get(bt));
+ }
+ }
+ }
+ for (ActivityCluster bs : blockedStages) {
+ bs.addDependency(s);
+ s.addDependent(bs);
+ }
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("Inferred " + (stages.size() + 1) + " stages");
+ for (ActivityCluster s : stages) {
+ LOGGER.info(s.toString());
+ }
+ LOGGER.info("SID: ENDSTAGE");
+ }
+ return endStage;
+ }
+
+ private void merge(Map<ActivityNodeId, ActivityCluster> eqSetMap, Set<ActivityCluster> eqSets, ActivityNodeId t1,
+ ActivityNodeId t2) {
+ ActivityCluster stage1 = eqSetMap.get(t1);
+ Set<ActivityNodeId> s1 = stage1.getActivities();
+ ActivityCluster stage2 = eqSetMap.get(t2);
+ Set<ActivityNodeId> s2 = stage2.getActivities();
+
+ Set<ActivityNodeId> mergedSet = new HashSet<ActivityNodeId>();
+ mergedSet.addAll(s1);
+ mergedSet.addAll(s2);
+
+ eqSets.remove(stage1);
+ eqSets.remove(stage2);
+ ActivityCluster mergedStage = new ActivityCluster(jobRun, mergedSet);
+ eqSets.add(mergedStage);
+
+ for (ActivityNodeId t : mergedSet) {
+ eqSetMap.put(t, mergedStage);
+ }
+ }
+
+ private void findRunnableClusters(Set<ActivityCluster> frontier, ActivityCluster candidate) {
+ if (completedClusters.contains(candidate) || frontier.contains(candidate)) {
+ return;
+ }
+ boolean runnable = true;
+ for (ActivityCluster s : candidate.getDependencies()) {
+ if (!completedClusters.contains(s)) {
+ runnable = false;
+ findRunnableClusters(frontier, s);
+ }
+ }
+ if (runnable && candidate != rootActivityCluster) {
+ frontier.add(candidate);
+ }
+ }
+
+ private void findRunnableClusters(Set<ActivityCluster> frontier) {
+ findRunnableClusters(frontier, rootActivityCluster);
+ }
+
+ @Override
+ public void schedule() throws HyracksException {
+ try {
+ solver = new PartitionConstraintSolver();
+ final JobActivityGraph jag = jobRun.getJobActivityGraph();
+ 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);
+ }
+ });
+ PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
+ @Override
+ public void visit(IConnectorDescriptor conn) {
+ conn.contributeSchedulingConstraints(acceptor, jag);
+ }
+ });
+ contributedConstraints.addAll(spec.getUserConstraints());
+ solver.addConstraints(contributedConstraints);
+
+ rootActivityCluster = inferStages(jag);
+ Set<ActivityCluster> runnableClusters = new HashSet<ActivityCluster>();
+ findRunnableClusters(runnableClusters);
+ if (runnableClusters.isEmpty() && inProgressClusters.isEmpty()) {
+ ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobRun.getJobId(), JobStatus.TERMINATED, null));
+ return;
+ }
+ for (ActivityCluster ac : runnableClusters) {
+ inProgressClusters.add(ac);
+ buildTaskClusters(ac);
+ IActivityClusterStateMachine acsm = new DefaultActivityClusterStateMachine(ccs, this, ac);
+ ac.setStateMachine(acsm);
+ acsm.schedule();
+ }
+ } catch (Exception e) {
+ ccs.getJobQueue().schedule(new JobCleanupEvent(ccs, jobRun.getJobId(), JobStatus.FAILURE, e));
+ throw new HyracksException(e);
+ }
+ }
+
+ private Map<ActivityNodeId, Integer> computePartitionCounts(ActivityCluster ac) throws HyracksException {
+ Set<LValueConstraintExpression> lValues = new HashSet<LValueConstraintExpression>();
+ for (ActivityNodeId 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<ActivityNodeId, Integer> activityPartsMap = new HashMap<ActivityNodeId, Integer>();
+ for (ActivityNodeId anId : ac.getActivities()) {
+ activityPartsMap.put(anId, nPartMap.get(anId.getOperatorDescriptorId()));
+ }
+ return activityPartsMap;
+ }
+
+ private void buildTaskClusters(ActivityCluster ac) throws HyracksException {
+ Map<ActivityNodeId, Integer> pcMap = computePartitionCounts(ac);
+ Map<ActivityNodeId, TaskState[]> taskStateMap = ac.getTaskStateMap();
+
+ for (ActivityNodeId anId : ac.getActivities()) {
+ int nParts = pcMap.get(anId);
+ TaskState[] taskStates = new TaskState[nParts];
+ for (int i = 0; i < nParts; ++i) {
+ taskStates[i] = new TaskState(new TaskId(anId, i));
+ }
+ taskStateMap.put(anId, taskStates);
+ }
+
+ Set<ActivityNodeId> activities = ac.getActivities();
+
+ Map<ConnectorDescriptorId, IConnectorPolicy> connectorPolicies = new HashMap<ConnectorDescriptorId, IConnectorPolicy>();
+
+ Map<TaskId, Set<TaskId>> taskClusterMap = new HashMap<TaskId, Set<TaskId>>();
+ for (ActivityNodeId anId : activities) {
+ TaskState[] taskStates = taskStateMap.get(anId);
+ for (TaskState ts : taskStates) {
+ Set<TaskId> cluster = new HashSet<TaskId>();
+ cluster.add(ts.getTaskId());
+ taskClusterMap.put(ts.getTaskId(), cluster);
+ }
+ }
+
+ Map<TaskId, List<Pair<TaskId, IConnectorPolicy>>> connectionInfo = new HashMap<TaskId, List<Pair<TaskId, IConnectorPolicy>>>();
+ JobActivityGraph jag = jobRun.getJobActivityGraph();
+ BitSet targetBitmap = new BitSet();
+ for (ActivityNodeId ac1 : activities) {
+ TaskState[] ac1TaskStates = taskStateMap.get(ac1);
+ int nProducers = ac1TaskStates.length;
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+ if (outputConns != null) {
+ for (IConnectorDescriptor c : outputConns) {
+ ConnectorDescriptorId cdId = c.getConnectorId();
+ IConnectorPolicy cPolicy = connectorPolicies.get(cdId);
+ ActivityNodeId ac2 = jag.getConsumerActivity(cdId);
+ TaskState[] ac2TaskStates = taskStateMap.get(ac2);
+ int nConsumers = ac2TaskStates.length;
+ for (int i = 0; i < nProducers; ++i) {
+ c.indicateTargetPartitions(nProducers, nConsumers, i, targetBitmap);
+ List<Pair<TaskId, IConnectorPolicy>> cInfoList = connectionInfo.get(ac1TaskStates[i]
+ .getTaskId());
+ if (cInfoList == null) {
+ cInfoList = new ArrayList<Pair<TaskId, IConnectorPolicy>>();
+ connectionInfo.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)) {
+ cInfoList.add(new Pair<TaskId, IConnectorPolicy>(ac2TaskStates[j].getTaskId(), cPolicy));
+ if (cPolicy.requiresProducerConsumerCoscheduling()) {
+ cluster.add(ac2TaskStates[j].getTaskId());
+ }
+ }
+ }
+ }
+ }
+ }
+
+ boolean done = false;
+ while (!done) {
+ done = true;
+ Set<TaskId> set = new HashSet<TaskId>();
+ for (Map.Entry<TaskId, Set<TaskId>> e : taskClusterMap.entrySet()) {
+ set.clear();
+ 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 : set) {
+ taskClusterMap.put(tid, set);
+ }
+ }
+
+ Set<Set<TaskId>> clusters = new HashSet<Set<TaskId>>(taskClusterMap.values());
+ Set<TaskCluster> tcSet = new HashSet<TaskCluster>();
+ for (Set<TaskId> cluster : clusters) {
+ Set<TaskState> taskStates = new HashSet<TaskState>();
+ for (TaskId tid : cluster) {
+ taskStates.add(taskStateMap.get(tid.getActivityId())[tid.getPartition()]);
+ }
+ TaskCluster tc = new TaskCluster(ac, taskStates.toArray(new TaskState[taskStates.size()]));
+ tcSet.add(tc);
+ for (TaskId tid : cluster) {
+ taskStateMap.get(tid.getActivityId())[tid.getPartition()].setTaskCluster(tc);
+ }
+ }
+ ac.setTaskClusters(tcSet.toArray(new TaskCluster[tcSet.size()]));
+
+ for (TaskCluster tc : tcSet) {
+ for (TaskState ts : tc.getTasks()) {
+ TaskId tid = ts.getTaskId();
+ List<Pair<TaskId, IConnectorPolicy>> cInfoList = connectionInfo.get(tid);
+ if (cInfoList != null) {
+ for (Pair<TaskId, IConnectorPolicy> p : cInfoList) {
+ TaskState targetTS = taskStateMap.get(p.first.getActivityId())[p.first.getPartition()];
+ TaskCluster targetTC = targetTS.getTaskCluster();
+ if (targetTC != tc) {
+ targetTC.getDependencies().add(tc);
+ if (p.second.consumerWaitsForProducerToFinish()) {
+ targetTC.getBlockers().add(tc);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ computeBlockerClosure(tcSet);
+ computeDependencyClosure(tcSet);
+ }
+
+ private void computeDependencyClosure(Set<TaskCluster> tcSet) {
+ boolean done = false;
+ while (!done) {
+ done = true;
+ for (TaskCluster tc : tcSet) {
+ Set<TaskCluster> deps = tc.getDependencies();
+ if (!deps.isEmpty()) {
+ Set<TaskCluster> copy = new HashSet<TaskCluster>(deps);
+ for (TaskCluster tc2 : copy) {
+ for (TaskCluster tc3 : tc2.getDependencies()) {
+ if (!deps.contains(tc3)) {
+ deps.add(tc3);
+ done = false;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+
+ private void computeBlockerClosure(Set<TaskCluster> tcSet) {
+ boolean done = false;
+ while (!done) {
+ done = true;
+ for (TaskCluster tc : tcSet) {
+ Set<TaskCluster> blockers = tc.getBlockers();
+ if (!blockers.isEmpty()) {
+ Set<TaskCluster> copy = new HashSet<TaskCluster>(blockers);
+ for (TaskCluster tc2 : copy) {
+ for (TaskCluster tc3 : tc2.getBlockers()) {
+ if (!blockers.contains(tc3)) {
+ blockers.add(tc3);
+ done = false;
+ }
+ }
+ }
+ }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobScheduler.java b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobScheduler.java
new file mode 100644
index 0000000..c71b23b
--- /dev/null
+++ b/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/DefaultJobScheduler.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2010 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.control.cc.scheduler;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
+
+public class DefaultJobScheduler implements IJobScheduler {
+ private final ClusterControllerService ccs;
+
+ public DefaultJobScheduler(ClusterControllerService ccs) {
+ this.ccs = ccs;
+ }
+
+ @Override
+ public void notifyJobCreation(JobRun run) throws HyracksException {
+ IJobRunStateMachine jsm = new DefaultJobRunStateMachine(ccs, run);
+ run.setStateMachine(jsm);
+ }
+}
\ 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/IActivityClusterStateMachine.java
similarity index 69%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IActivityClusterStateMachine.java
index 3893a2d..2c4a4ae 100644
--- 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/IActivityClusterStateMachine.java
@@ -14,14 +14,13 @@
*/
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;
+import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
-public interface IScheduler {
- public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja);
+public interface IActivityClusterStateMachine {
+ public void schedule() throws HyracksException;
- public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException;
+ public void notifyTaskComplete(TaskAttempt ta) throws HyracksException;
+
+ public void notifyTaskFailure(TaskAttempt ta, Exception exception) throws HyracksException;
}
\ 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/IJobRunStateMachine.java
similarity index 81%
rename from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
rename to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobRunStateMachine.java
index 2906ba6..fcf0a79 100644
--- 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/IJobRunStateMachine.java
@@ -14,8 +14,8 @@
*/
package edu.uci.ics.hyracks.control.cc.scheduler;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-public interface ISchedule {
- public String[] getPartitions(ActivityNodeId aid);
+public interface IJobRunStateMachine {
+ public void schedule() throws HyracksException;
}
\ 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/IJobScheduler.java
similarity index 76%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ISchedule.java
copy to hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IJobScheduler.java
index 2906ba6..d359731 100644
--- 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/IJobScheduler.java
@@ -14,8 +14,9 @@
*/
package edu.uci.ics.hyracks.control.cc.scheduler;
-import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.control.cc.job.JobRun;
-public interface ISchedule {
- public String[] getPartitions(ActivityNodeId aid);
+public interface IJobScheduler {
+ public void notifyJobCreation(JobRun run) throws HyracksException;
}
\ 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/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 f40e8cd..0000000
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/naive/NaiveScheduler.java
+++ /dev/null
@@ -1,212 +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);
- System.err.println("Constraints: " + opConstraints);
- 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) {
- System.err.println(ce);
- if (ce.getTag() == ExpressionTag.RELATIONAL) {
- RelationalExpression re = (RelationalExpression) ce;
- if (re.getOperator() == RelationalExpression.Operator.EQUAL) {
- System.err.println("Left: " + re.getLeft());
- System.err.println("Right: " + re.getRight());
- 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);
- }
- }
- }
-}
\ No newline at end of file
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
index 02b8f9e..1d95390 100644
--- 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
@@ -15,23 +15,30 @@
package edu.uci.ics.hyracks.control.common.base;
import java.rmi.Remote;
+import java.util.Collection;
import java.util.List;
import java.util.UUID;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.control.common.job.profiling.om.JobProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.StageletProfile;
+import edu.uci.ics.hyracks.control.common.job.profiling.om.TaskProfile;
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 notifyTaskComplete(UUID jobId, TaskAttemptId taskId, String nodeId, TaskProfile statistics) throws Exception;
- public void notifyStageletFailure(UUID jobId, UUID stageId, int attempt, String nodeId) throws Exception;
+ public void notifyTaskFailure(UUID 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(PartitionId pid, NetworkAddress address) throws Exception;
+
+ public void registerPartitionRequest(Collection<PartitionId> requiredPartitionIds, String nodeId) 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
index 8fd79f3..f0ceed6 100644
--- 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
@@ -15,14 +15,14 @@
package edu.uci.ics.hyracks.control.common.base;
import java.rmi.Remote;
+import java.util.List;
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.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.PortInstanceId;
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
public interface INodeController extends Remote {
public String getId() throws Exception;
@@ -31,26 +31,19 @@
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 startTasks(String appName, UUID jobId, byte[] planBytes, List<TaskAttemptId> 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 abortTasks(UUID jobId, List<TaskAttemptId> tasks) 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;
+
+ 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/comm/IEndpointDataWriterFactory.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/IConnectorPolicy.java
similarity index 70%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/IConnectorPolicy.java
index 59edd97..da659ce 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/IConnectorPolicy.java
@@ -12,10 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.comm;
+package edu.uci.ics.hyracks.control.common.job.dataflow;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import java.io.Serializable;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+public interface IConnectorPolicy extends Serializable {
+ public boolean requiresProducerConsumerCoscheduling();
+
+ public boolean consumerWaitsForProducerToFinish();
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/PipelinedConnectorPolicy.java
similarity index 62%
copy from hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
copy to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/PipelinedConnectorPolicy.java
index 59edd97..bc98e93 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IEndpointDataWriterFactory.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/dataflow/PipelinedConnectorPolicy.java
@@ -12,10 +12,18 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.api.comm;
+package edu.uci.ics.hyracks.control.common.job.dataflow;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+public final class PipelinedConnectorPolicy implements IConnectorPolicy {
+ private static final long serialVersionUID = 1L;
-public interface IEndpointDataWriterFactory {
- public IFrameWriter createFrameWriter(int index) throws HyracksDataException;
+ @Override
+ public boolean requiresProducerConsumerCoscheduling() {
+ return true;
+ }
+
+ @Override
+ public boolean consumerWaitsForProducerToFinish() {
+ return true;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
index 77ac2b5..cc0b1a2 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobProfile.java
@@ -12,13 +12,10 @@
private final UUID jobId;
- private final int attempt;
-
private final Map<String, JobletProfile> jobletProfiles;
- public JobProfile(UUID jobId, int attempt) {
+ public JobProfile(UUID jobId) {
this.jobId = jobId;
- this.attempt = attempt;
jobletProfiles = new HashMap<String, JobletProfile>();
}
@@ -26,10 +23,6 @@
return jobId;
}
- public int getAttempt() {
- return attempt;
- }
-
public Map<String, JobletProfile> getJobletProfiles() {
return jobletProfiles;
}
@@ -40,7 +33,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-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
index f847cf9..5069fc3 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/JobletProfile.java
@@ -16,29 +16,30 @@
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/StageletProfile.java b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
similarity index 63%
rename from hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/StageletProfile.java
rename to hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
index 33beaf6..3ab1090 100644
--- a/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/StageletProfile.java
+++ b/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/job/profiling/om/TaskProfile.java
@@ -14,30 +14,32 @@
*/
package edu.uci.ics.hyracks.control.common.job.profiling.om;
-import java.util.UUID;
-
import org.json.JSONException;
import org.json.JSONObject;
-public class StageletProfile extends AbstractProfile {
+import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+
+public class TaskProfile extends AbstractProfile {
private static final long serialVersionUID = 1L;
- private final UUID stageId;
+ private final TaskAttemptId taskAttemptId;
- public StageletProfile(UUID stageId) {
- this.stageId = stageId;
+ public TaskProfile(TaskAttemptId taskAttemptId) {
+ this.taskAttemptId = taskAttemptId;
}
- public UUID getStageId() {
- return stageId;
+ public TaskAttemptId getTaskId() {
+ return taskAttemptId;
}
@Override
public JSONObject toJSON() throws JSONException {
JSONObject json = new JSONObject();
- json.put("type", "joblet-profile");
- json.put("stage-id", stageId.toString());
+ 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;
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 f4fa95b..6599eb0 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,28 +15,35 @@
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.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.profiling.counters.ICounter;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
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.common.job.profiling.om.JobletProfile;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.StageletProfile;
+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 {
@@ -48,28 +55,28 @@
private final UUID 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 DefaultDeallocatableRegistry deallocatableRegistry;
private final IWorkspaceFileFactory fileFactory;
- public Joblet(NodeControllerService nodeController, UUID jobId, int attempt, INCApplicationContext appCtx) {
+ public Joblet(NodeControllerService nodeController, UUID jobId, INCApplicationContext appCtx) {
this.nodeController = nodeController;
this.appCtx = appCtx;
this.jobId = jobId;
- this.attempt = attempt;
- stageletMap = new HashMap<UUID, Stagelet>();
+ partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
envMap = new HashMap<OperatorDescriptorId, Map<Integer, IOperatorEnvironment>>();
+ taskMap = new HashMap<TaskAttemptId, Task>();
counterMap = new HashMap<String, Counter>();
deallocatableRegistry = new DefaultDeallocatableRegistry();
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
}
@Override
@@ -88,6 +95,14 @@
return opEnvMap.get(partition);
}
+ public void addTask(Task task) {
+ taskMap.put(task.getTaskId(), task);
+ }
+
+ public Map<TaskAttemptId, Task> getTaskMap() {
+ return taskMap;
+ }
+
private static final class OperatorEnvironmentImpl implements IOperatorEnvironment {
private final Map<String, Object> map;
@@ -106,26 +121,20 @@
}
}
- public void setStagelet(UUID stageId, Stagelet stagelet) {
- stageletMap.put(stageId, stagelet);
- }
-
- public Stagelet getStagelet(UUID stageId) throws Exception {
- return stageletMap.get(stageId);
- }
-
public Executor getExecutor() {
return nodeController.getExecutor();
}
- public synchronized 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.getTaskId());
+ task.dumpProfile(taskProfile);
+ nodeController.notifyTaskComplete(jobId, task.getTaskId(), taskProfile);
}
- public void notifyStageletFailed(UUID stageId, int attempt) throws Exception {
- stageletMap.remove(stageId);
- nodeController.notifyStageFailed(jobId, stageId, attempt);
+ public synchronized void notifyStageletFailed(Task task, Exception exception) throws Exception {
+ taskMap.remove(task);
+ nodeController.notifyTaskFailed(jobId, task.getTaskId(), exception);
}
public NodeControllerService getNodeController() {
@@ -137,10 +146,10 @@
for (Map.Entry<String, Counter> e : counterMap.entrySet()) {
counters.put(e.getKey(), e.getValue().get());
}
- 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.getTaskId());
+ task.dumpProfile(taskProfile);
+ jProfile.getTaskProfiles().put(task.getTaskId(), taskProfile);
}
}
@@ -150,11 +159,6 @@
}
@Override
- public int getAttempt() {
- return attempt;
- }
-
- @Override
public ICounterContext getCounterContext() {
return this;
}
@@ -184,12 +188,13 @@
}
@Override
- public FileReference createWorkspaceFile(String prefix) throws HyracksDataException {
- return fileFactory.createWorkspaceFile(prefix);
+ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createManagedWorkspaceFile(prefix);
}
- public Map<UUID, Stagelet> getStageletMap() {
- return stageletMap;
+ @Override
+ public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException {
+ return fileFactory.createUnmanagedWorkspaceFile(prefix);
}
@Override
@@ -201,4 +206,19 @@
}
return counter;
}
+
+ public synchronized void advertisePartitionRequest(Collection<PartitionId> requiredPartitionIds,
+ IPartitionCollector collector) throws Exception {
+ for (PartitionId pid : requiredPartitionIds) {
+ partitionRequestMap.put(pid, collector);
+ }
+ nodeController.getClusterController().registerPartitionRequest(requiredPartitionIds, nodeController.getId());
+ }
+
+ 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/NodeControllerService.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0bb3904..bdd30a9 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,7 +28,6 @@
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;
@@ -47,31 +46,25 @@
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.IEndpointDataWriterFactory;
-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.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.IConnectorDescriptor;
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.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.JobActivityGraph;
+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;
@@ -82,12 +75,12 @@
import edu.uci.ics.hyracks.control.common.context.ServerContext;
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.StageletProfile;
+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.PartitionManager;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
public class NodeControllerService extends AbstractRemoteService implements INodeController {
@@ -103,6 +96,8 @@
private final NodeCapability nodeCapability;
+ private final PartitionManager partitionManager;
+
private final ConnectionManager connectionManager;
private final Timer timer;
@@ -129,6 +124,9 @@
}
nodeCapability = computeNodeCapability();
connectionManager = new ConnectionManager(ctx, getIpAddress(ncConfig));
+ partitionManager = new PartitionManager(this, connectionManager.getNetworkAddress());
+ connectionManager.setPartitionRequestListener(partitionManager);
+
jobletMap = new HashMap<UUID, Joblet>();
timer = new Timer(true);
serverCtx = new ServerContext(ServerContext.ServerType.NODE_CONTROLLER, new File(new File(
@@ -186,6 +184,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());
@@ -209,15 +215,11 @@
return InetAddress.getByAddress(ipBytes);
}
- @Override
- public Map<PortInstanceId, Endpoint> initializeJobletPhase1(String appName, UUID jobId, int attempt,
- byte[] planBytes, UUID stageId, Map<ActivityNodeId, Set<Integer>> tasks,
+ public void startTasks(String appName, UUID jobId, byte[] jagBytes, List<TaskAttemptId> tasks,
Map<OperatorDescriptorId, Integer> opNumPartitions) 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);
IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@Override
@@ -231,158 +233,66 @@
}
};
- 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);
+ for (TaskAttemptId tid : tasks) {
+ IActivityNode han = plan.getActivityNodeMap().get(tid.getTaskId().getActivityId());
if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Initializing " + hanId + " -> " + han);
+ LOGGER.finest("Initializing " + tid + " -> " + 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);
- stagelet.setOperator(op.getOperatorId(), i, or);
- if (inputs != null) {
- for (int j = 0; j < inputs.size(); ++j) {
- if (j >= 1) {
- throw new IllegalStateException();
- }
- IConnectorDescriptor conn = inputs.get(j);
+ int partition = tid.getTaskId().getPartition();
+ Task task = new Task(joblet, tid);
+ IOperatorNodePushable operator = han.createPushRuntime(task, joblet.getEnvironment(op, partition), rdp,
+ partition, opNumPartitions.get(op.getOperatorId()));
+
+ IPartitionCollector collector = null;
+
+ List<IConnectorDescriptor> inputs = plan.getActivityInputConnectorDescriptors(tid.getTaskId()
+ .getActivityId());
+ if (inputs != null) {
+ for (int i = 0; i < inputs.size(); ++i) {
+ if (i >= 1) {
+ throw new HyracksException("Multiple inputs to an activity not currently supported");
+ }
+ if (!inputs.isEmpty()) {
+ IConnectorDescriptor conn = inputs.get(0);
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,
- attempt, stageId, conn.getConnectorId());
- 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,
+ RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+ collector = conn.createPartitionCollector(task, recordDesc, partition,
opNumPartitions.get(producerOpId), opNumPartitions.get(consumerOpId));
- or.setFrameReader(reader);
}
}
- honMap.put(new OperatorInstanceId(op.getOperatorId(), i), or);
}
- }
-
- stagelet.setEndpointList(endpointList);
-
- 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));
- }
+ List<IConnectorDescriptor> outputs = plan.getActivityOutputConnectorDescriptors(tid.getTaskId()
+ .getActivityId());
+ if (outputs != null) {
+ for (int i = 0; i < outputs.size(); ++i) {
+ IConnectorDescriptor conn = outputs.get(i);
+ OperatorDescriptorId producerOpId = plan.getJobSpecification().getProducer(conn)
+ .getOperatorId();
+ OperatorDescriptorId consumerOpId = plan.getJobSpecification().getConsumer(conn)
+ .getOperatorId();
+ RecordDescriptor recordDesc = plan.getJobSpecification().getConnectorRecordDescriptor(conn);
+ IPartitionWriterFactory pwFactory = new IPartitionWriterFactory() {
+ @Override
+ public IFrameWriter createFrameWriter(int receiverIndex) throws HyracksDataException {
+ return null;
+ }
+ };
+ IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition,
+ opNumPartitions.get(producerOpId), opNumPartitions.get(consumerOpId));
+ operator.setOutputFrameWriter(i, writer, recordDesc);
}
- stagelet.installRunnable(new OperatorInstanceId(op.getOperatorId(), i));
}
+
+ task.setTaskRuntime(collector, operator);
+ joblet.addTask(task);
+
+ task.start();
}
} catch (Exception e) {
e.printStackTrace();
@@ -390,64 +300,10 @@
}
}
- 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;
- }
-
- @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());
- }
- si.setEndpointList(null);
- }
-
- private synchronized Joblet getLocalJoblet(UUID jobId) throws Exception {
+ private synchronized Joblet getOrCreateLocalJoblet(UUID jobId, INCApplicationContext appCtx) throws Exception {
Joblet ji = jobletMap.get(jobId);
- return ji;
- }
-
- private synchronized Joblet getOrCreateLocalJoblet(UUID jobId, int attempt, INCApplicationContext appCtx)
- throws Exception {
- Joblet ji = jobletMap.get(jobId);
- if (ji == null || ji.getAttempt() != attempt) {
- ji = new Joblet(this, jobId, attempt, appCtx);
+ if (ji == null) {
+ ji = new Joblet(this, jobId, appCtx);
jobletMap.put(jobId, ji);
}
return ji;
@@ -464,34 +320,23 @@
}
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(UUID 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(UUID jobId, TaskAttemptId taskId, Exception exception) throws Exception {
try {
- ccs.notifyStageletFailure(jobId, stageId, attempt, id);
+ ccs.notifyTaskFailure(jobId, taskId, id, exception);
} catch (Exception e) {
e.printStackTrace();
throw e;
@@ -539,7 +384,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) {
@@ -563,20 +408,21 @@
}
@Override
- public synchronized void abortJoblet(UUID jobId, int attempt) throws Exception {
+ public synchronized void abortTasks(UUID 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();
+ task.close();
+ }
}
- for (Stagelet stagelet : ji.getStageletMap().values()) {
- stagelet.abort();
- stagelet.close();
- connectionManager.abortConnections(jobId, stagelet.getStageId());
- }
+ ji.close();
}
}
@@ -617,4 +463,12 @@
appCtx.deinitialize();
}
}
+
+ @Override
+ public void reportPartitionAvailability(PartitionId pid, NetworkAddress networkAddress) throws Exception {
+ Joblet ji = jobletMap.get(pid.getJobId());
+ 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 157b0a7..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Stagelet.java
+++ /dev/null
@@ -1,238 +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.resources.IDeallocatable;
-import edu.uci.ics.hyracks.control.common.job.profiling.counters.Counter;
-import edu.uci.ics.hyracks.control.common.job.profiling.om.StageletProfile;
-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 synchronized void notifyOperatorCompletion(OperatorInstanceId opIId) {
- pendingOperators.remove(opIId);
- if (pendingOperators.isEmpty()) {
- 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 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..a39d1b7
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -0,0 +1,192 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+
+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.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.resources.IDeallocatable;
+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 taskId;
+
+ private final IWorkspaceFileFactory fileFactory;
+
+ private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+ private final Map<String, Counter> counterMap;
+
+ private IPartitionCollector collector;
+
+ private IOperatorNodePushable operator;
+
+ private volatile boolean aborted;
+
+ public Task(Joblet joblet, TaskAttemptId taskId) {
+ this.joblet = joblet;
+ this.taskId = taskId;
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) joblet.getIOManager());
+ deallocatableRegistry = new DefaultDeallocatableRegistry();
+ counterMap = new HashMap<String, Counter>();
+ }
+
+ public void setTaskRuntime(IPartitionCollector collector, IOperatorNodePushable operator) {
+ this.collector = collector;
+ 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);
+ }
+
+ public void close() {
+ deallocatableRegistry.close();
+ }
+
+ @Override
+ public IHyracksJobletContext getJobletContext() {
+ return joblet;
+ }
+
+ @Override
+ public TaskAttemptId getTaskId() {
+ return taskId;
+ }
+
+ @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;
+ collector.abort();
+ }
+
+ @Override
+ public void run() {
+ try {
+ operator.initialize();
+ try {
+ if (collector != null) {
+ if (aborted) {
+ return;
+ }
+ collector.open();
+ try {
+ joblet.advertisePartitionRequest(collector.getRequiredPartitionIds(), collector);
+ IFrameReader reader = collector.getReader();
+ reader.open();
+ try {
+ IFrameWriter writer = operator.getInputFrameWriter(0);
+ 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();
+ }
+ }
+ } finally {
+ operator.deinitialize();
+ }
+ } catch (Exception 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 d44fadd..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionEntry.java
+++ /dev/null
@@ -1,186 +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();
- }
- }
-
- 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;
- }
-
- public void setJobId(UUID jobId) {
- this.jobId = jobId;
- }
-
- @Override
- public UUID getStageId() {
- return stageId;
- }
-
- public void setStageId(UUID stageId) {
- this.stageId = stageId;
- }
-
- 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 658f229..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/ConnectionManager.java
+++ /dev/null
@@ -1,400 +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();
- ((ConnectionEntry) 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 {
- ConnectionEntry ce = (ConnectionEntry) entry;
- ByteBuffer buffer = ce.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) {
- ce.close();
- return;
- }
- }
-
- newListener = connectionReceiver.getDataReceiveListener(endpointID, ce, senderId);
- ce.setDataReceiveListener(newListener);
- ce.setJobId(connectionReceiver.getJobId());
- ce.setStageId(connectionReceiver.getStageId());
- synchronized (ConnectionManager.this) {
- connections.add(ce);
- }
- byte[] ack = new byte[4];
- ByteBuffer ackBuffer = ByteBuffer.wrap(ack);
- ackBuffer.clear();
- ackBuffer.putInt(FrameConstants.SIZE_LEN);
- ackBuffer.flip();
- ce.write(ackBuffer);
- }
- buffer.compact();
- if (newListener != null && buffer.remaining() > 0) {
- newListener.dataReceived(ce);
- }
- }
-
- @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 cb6e936..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/DemuxDataReceiveListenerFactory.java
+++ /dev/null
@@ -1,157 +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.dataflow.ConnectorDescriptorId;
-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 final UUID jobId;
- private final int attempt;
- private final UUID stageId;
- private final ConnectorDescriptorId cdId;
-
- public DemuxDataReceiveListenerFactory(IHyracksStageletContext ctx, UUID jobId, int attempt, UUID stageId,
- ConnectorDescriptorId cdId) {
- frameSize = ctx.getFrameSize();
- this.jobId = jobId;
- this.attempt = attempt;
- this.stageId = stageId;
- this.cdId = cdId;
- 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/comm/Sender.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/Sender.java
deleted file mode 100644
index 8f4c846..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/comm/Sender.java
+++ /dev/null
@@ -1,71 +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.util.UUID;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.comm.ISender;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
-import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-
-public class Sender implements ISender {
- private final IHyracksStageletContext ctx;
-
- private final UUID jobId;
-
- private final int attempt;
-
- private final UUID stageId;
-
- private final ConnectorDescriptorId cdId;
-
- private final int senderIndex;
-
- public Sender(IHyracksStageletContext ctx, UUID jobId, int attempt, UUID stageId, ConnectorDescriptorId cdId,
- int senderIndex) {
- this.ctx = ctx;
- this.jobId = jobId;
- this.attempt = attempt;
- this.stageId = stageId;
- this.cdId = cdId;
- this.senderIndex = senderIndex;
- }
-
- @Override
- public IFrameWriter createSenderWriter(int receiverIndex) {
- return null;
- }
-
- public UUID getJobId() {
- return jobId;
- }
-
- public int getAttempt() {
- return attempt;
- }
-
- public UUID getStageId() {
- return stageId;
- }
-
- public ConnectorDescriptorId getConnectorDescriptorId() {
- return cdId;
- }
-
- public int getSenderIndex() {
- return senderIndex;
- }
-}
\ 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..fbddabd
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/ConnectionManager.java
@@ -0,0 +1,265 @@
+/*
+ * 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.net.SocketAddress;
+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.Iterator;
+import java.util.List;
+import java.util.UUID;
+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.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.nc.partitions.IPartitionRequestListener;
+
+public class ConnectionManager {
+ private static final Logger LOGGER = Logger.getLogger(ConnectionManager.class.getName());
+
+ private static final int INITIAL_MESSAGE_SIZE = 40;
+
+ 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");
+ }
+
+ @Override
+ public void run() {
+ while (!stopped) {
+ try {
+ SocketChannel sc = serverChannel.accept();
+ dataListener.addIncomingConnection(sc);
+ } catch (AsynchronousCloseException e) {
+ // do nothing
+ e.printStackTrace();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ private final class DataListenerThread extends Thread {
+ private Selector selector;
+
+ private final List<SocketChannel> pendingIncomingConnections;
+ private final List<SocketChannel> pendingNegotiations;
+ private final List<INetworkChannel> pendingOutgoingConnections;
+ private final List<INetworkChannel> pendingAbortConnections;
+
+ public DataListenerThread() {
+ super("Hyracks Data Listener Thread");
+ try {
+ selector = Selector.open();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ pendingIncomingConnections = new ArrayList<SocketChannel>();
+ pendingNegotiations = new ArrayList<SocketChannel>();
+ pendingOutgoingConnections = new ArrayList<INetworkChannel>();
+ pendingAbortConnections = 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();
+ }
+
+ synchronized void addPendingAbortConnections(List<INetworkChannel> 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 (!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) {
+ SocketAddress rAddr = nc.getRemoteAddress();
+ SocketChannel sc = SocketChannel.open();
+ sc.configureBlocking(false);
+ int interestOps = SelectionKey.OP_READ;
+ if (!sc.connect(rAddr)) {
+ interestOps |= SelectionKey.OP_CONNECT;
+ }
+ SelectionKey scKey = sc.register(selector, interestOps);
+ scKey.attach(nc);
+ nc.setSelectionKey(scKey);
+ }
+ pendingOutgoingConnections.clear();
+ }
+ if (!pendingAbortConnections.isEmpty()) {
+ for (INetworkChannel nc : pendingAbortConnections) {
+ SelectionKey key = nc.getSelectionKey();
+ nc.abort();
+ nc.dispatchNetworkEvent();
+ key.cancel();
+ }
+ 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();
+ 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);
+ 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) {
+ UUID jobId = readUUID(buffer);
+ ConnectorDescriptorId cdid = new ConnectorDescriptorId(readUUID(buffer));
+ int senderIndex = buffer.getInt();
+ int receiverIndex = buffer.getInt();
+ return new PartitionId(jobId, cdid, senderIndex, receiverIndex);
+ }
+
+ private UUID readUUID(ByteBuffer buffer) {
+ long msb = buffer.getLong();
+ long lsb = buffer.getLong();
+ return new UUID(msb, lsb);
+ }
+ }
+
+ 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..98215ee
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/INetworkChannel.java
@@ -0,0 +1,17 @@
+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();
+}
\ 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..24a6412
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
@@ -0,0 +1,188 @@
+/*
+ * 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.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;
+
+ 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) {
+ emptyQueue.add(buffer);
+ if (!eos && !aborted) {
+ int ops = key.interestOps();
+ if ((ops & SelectionKey.OP_READ) == 0) {
+ key.interestOps(ops | SelectionKey.OP_READ);
+ }
+ }
+ }
+
+ @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.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());
+ }
+ 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;
+ }
+
+ @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;
+ }
+}
\ 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..eb1ec89
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
@@ -0,0 +1,167 @@
+/*
+ * 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);
+ }
+ int interestOps = key.interestOps();
+ if ((interestOps & SelectionKey.OP_WRITE) == 0) {
+ key.interestOps(interestOps | SelectionKey.OP_WRITE);
+ }
+ }
+
+ @Override
+ public void flush() throws HyracksDataException {
+
+ }
+
+ @Override
+ public synchronized void close() throws HyracksDataException {
+ eos = true;
+ }
+}
\ 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-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
similarity index 63%
copy from hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
copy to hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
index 3893a2d..20ed49c 100644
--- a/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/IScheduler.java
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/IPartitionRequestListener.java
@@ -12,16 +12,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.cc.scheduler;
+package edu.uci.ics.hyracks.control.nc.partitions;
-import java.util.Set;
-
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
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;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
-public interface IScheduler {
- public IJobAttemptSchedulerState createJobAttemptState(JobAttempt ja);
-
- public void schedule(Set<JobStageAttempt> runnableStageAttempts) throws HyracksException;
+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..6833cbe
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartition.java
@@ -0,0 +1,88 @@
+/*
+ * 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);
+ 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");
+ }
+ buffer.flip();
+ writer.nextFrame(buffer);
+ }
+ } finally {
+ writer.close();
+ }
+ } 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/PartitionManager.java b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
new file mode 100644
index 0000000..27afb8c
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -0,0 +1,82 @@
+/*
+ * 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.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+
+public class PartitionManager implements IPartitionRequestListener {
+ private final NetworkAddress dataPort;
+
+ private final NodeControllerService ncs;
+
+ private final Map<PartitionId, IPartition> partitionMap;
+
+ public PartitionManager(NodeControllerService ncs, NetworkAddress dataPort) {
+ this.dataPort = dataPort;
+ this.ncs = ncs;
+ partitionMap = new HashMap<PartitionId, IPartition>();
+ }
+
+ public void registerPartition(PartitionId pid, IPartition partition) throws HyracksDataException {
+ synchronized (this) {
+ partitionMap.put(pid, partition);
+ }
+ try {
+ ncs.getClusterController().registerPartitionProvider(pid, dataPort);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ public synchronized IPartition getPartition(PartitionId pid) {
+ return partitionMap.get(pid);
+ }
+
+ public synchronized void unregisterPartitions(UUID jobId) {
+ for (Iterator<Map.Entry<PartitionId, IPartition>> i = partitionMap.entrySet().iterator(); i.hasNext();) {
+ Map.Entry<PartitionId, IPartition> e = i.next();
+ PartitionId pid = e.getKey();
+ if (jobId.equals(pid.getJobId())) {
+ e.getValue().deallocate();
+ i.remove();
+ }
+ }
+ }
+
+ @Override
+ public synchronized void registerPartitionRequest(PartitionId partitionId, IFrameWriter writer)
+ throws HyracksException {
+ IPartition partition = partitionMap.get(partitionId);
+ if (partition != null) {
+ partition.writeTo(writer);
+ if (!partition.isReusable()) {
+ partitionMap.remove(partitionId);
+ }
+ } else {
+ throw new HyracksException("Request for unknown partition " + partitionId);
+ }
+ }
+}
\ 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..c667ef4
--- /dev/null
+++ b/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PipelinedPartition.java
@@ -0,0 +1,79 @@
+/*
+ * 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.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.IPartition;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+
+public class PipelinedPartition implements IFrameWriter, IPartition {
+ private final PartitionManager manager;
+
+ private final PartitionId pid;
+
+ private IFrameWriter delegate;
+
+ public PipelinedPartition(PartitionManager manager, PartitionId pid) {
+ this.manager = manager;
+ this.pid = pid;
+ }
+
+ @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, this);
+ 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 {
+ delegate.close();
+ }
+}
\ 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 eca7fd0..0000000
--- a/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/runtime/OperatorRunnable.java
+++ /dev/null
@@ -1,82 +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 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 IOperatorNodePushable opNode;
- private IFrameReader reader;
- private ByteBuffer buffer;
- private volatile boolean abort;
-
- public OperatorRunnable(IHyracksStageletContext ctx, IOperatorNodePushable opNode) {
- this.opNode = opNode;
- buffer = ctx.allocateFrame();
- }
-
- public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
- opNode.setOutputFrameWriter(index, writer, recordDesc);
- }
-
- public void setFrameReader(IFrameReader reader) {
- this.reader = reader;
- }
-
- public void abort() {
- abort = true;
- }
-
- @Override
- public void run() {
- try {
- opNode.initialize();
- if (reader != null) {
- IFrameWriter writer = opNode.getInputFrameWriter(0);
- writer.open();
- reader.open();
- while (readFrame()) {
- if (abort) {
- break;
- }
- buffer.flip();
- writer.nextFrame(buffer);
- buffer.compact();
- }
- reader.close();
- writer.close();
- }
- opNode.deinitialize();
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- protected boolean readFrame() throws HyracksDataException {
- return reader.nextFrame(buffer);
- }
-
- @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/AbstractPartitionCollector.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/AbstractPartitionCollector.java
new file mode 100644
index 0000000..4326a9f
--- /dev/null
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/AbstractPartitionCollector.java
@@ -0,0 +1,50 @@
+/*
+ * 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.util.UUID;
+
+import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
+
+public abstract class AbstractPartitionCollector implements IPartitionCollector {
+ protected final IHyracksTaskContext stageletContext;
+
+ protected final ConnectorDescriptorId connectorId;
+
+ protected final int receiverIndex;
+
+ public AbstractPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex) {
+ this.stageletContext = ctx;
+ this.connectorId = connectorId;
+ this.receiverIndex = receiverIndex;
+ }
+
+ @Override
+ public UUID getJobId() {
+ return stageletContext.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-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/NonDeterministicPartitionCollector.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicPartitionCollector.java
new file mode 100644
index 0000000..fb6a883
--- /dev/null
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/NonDeterministicPartitionCollector.java
@@ -0,0 +1,181 @@
+/*
+ * 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.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 IInputChannel[] channels;
+
+ private BitSet frameAvailability;
+
+ private int[] availableFrameCounts;
+
+ private BitSet eosSenders;
+
+ private BitSet closedSenders;
+
+ private int lastReadSender;
+
+ public NonDeterministicPartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId,
+ int receiverIndex, BitSet expectedPartitions) {
+ super(ctx, connectorId, receiverIndex);
+ this.expectedPartitions = expectedPartitions;
+ int nSenders = expectedPartitions.size();
+ reader = new FrameReader();
+ channels = new IInputChannel[nSenders];
+ eosSenders = new BitSet(nSenders);
+ closedSenders = new BitSet(nSenders);
+ closedSenders.or(expectedPartitions);
+ closedSenders.flip(0, nSenders);
+ frameAvailability = new BitSet(nSenders);
+ availableFrameCounts = new int[nSenders];
+ }
+
+ @Override
+ public void open() throws HyracksException {
+ lastReadSender = 0;
+ }
+
+ @Override
+ public synchronized 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);
+ 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 {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ while (true) {
+ switch (lastReadSender) {
+ default:
+ lastReadSender = frameAvailability.nextSetBit(lastReadSender);
+ if (lastReadSender >= 0) {
+ break;
+ }
+ case 0:
+ lastReadSender = frameAvailability.nextSetBit(0);
+ }
+ if (lastReadSender >= 0) {
+ assert availableFrameCounts[lastReadSender] > 0;
+ if (--availableFrameCounts[lastReadSender] == 0) {
+ frameAvailability.clear(lastReadSender);
+ }
+ ByteBuffer srcFrame = channels[lastReadSender].getNextBuffer();
+ FrameUtils.copy(srcFrame, buffer);
+ channels[lastReadSender].recycleBuffer(srcFrame);
+ return true;
+ }
+ for (int i = eosSenders.nextSetBit(0); i >= 0; i = eosSenders.nextSetBit(i)) {
+ channels[i].close();
+ eosSenders.clear(i);
+ closedSenders.set(i);
+ }
+ if (closedSenders.nextClearBit(0) < 0) {
+ return false;
+ }
+ try {
+ NonDeterministicPartitionCollector.this.wait();
+ } catch (InterruptedException e) {
+ e.printStackTrace();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ synchronized (NonDeterministicPartitionCollector.this) {
+ for (int i = closedSenders.nextClearBit(0); i >= 0; i = closedSenders.nextClearBit(i)) {
+ if (channels[i] != null) {
+ channels[i].close();
+ }
+ }
+ }
+ }
+
+ @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)) {
+ 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-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/SortMergePartitionCollector.java b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergePartitionCollector.java
new file mode 100644
index 0000000..0977002
--- /dev/null
+++ b/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/SortMergePartitionCollector.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.dataflow.common.comm;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+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.HyracksException;
+import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+
+public class SortMergePartitionCollector extends AbstractPartitionCollector {
+ private final FrameTuplePairComparator tpc;
+
+ private final FrameTupleAppender appender;
+
+ private final RecordDescriptor recordDescriptor;
+
+ private final int maxFramesLimit;
+
+ private IInputChannel[] channels;
+
+ public SortMergePartitionCollector(IHyracksTaskContext ctx, ConnectorDescriptorId connectorId, int receiverIndex,
+ int[] sortFields, IBinaryComparator[] comparators, RecordDescriptor recordDescriptor, int maxFramesLimit,
+ int nSenders) {
+ super(ctx, connectorId, receiverIndex);
+ tpc = new FrameTuplePairComparator(sortFields, sortFields, comparators);
+ appender = new FrameTupleAppender(ctx.getFrameSize());
+ this.recordDescriptor = recordDescriptor;
+ this.maxFramesLimit = maxFramesLimit;
+ channels = new IInputChannel[nSenders];
+ }
+
+ @Override
+ public void open() throws HyracksException {
+ }
+
+ @Override
+ public void addPartitions(Collection<PartitionChannel> partitions) throws HyracksException {
+
+ }
+
+ @Override
+ public IFrameReader getReader() throws HyracksException {
+ return null;
+ }
+
+ @Override
+ public void close() throws HyracksException {
+
+ }
+
+ @Override
+ public Collection<PartitionId> getRequiredPartitionIds() throws HyracksException {
+ return null;
+ }
+
+ @Override
+ public void abort() {
+
+ }
+}
\ 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 c3167cb..ca37f37 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;
@@ -350,7 +350,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();
@@ -391,7 +391,7 @@
}
}
- private IOperatorNodePushable createSelfReadingMapper(final IHyracksStageletContext ctx, IOperatorEnvironment env,
+ private IOperatorNodePushable createSelfReadingMapper(final IHyracksTaskContext ctx, IOperatorEnvironment env,
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 2d44c79..0e05997 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
@@ -34,7 +34,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;
@@ -335,7 +335,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/MultiAggregatorFactory.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/aggregators/MultiAggregatorFactory.java
index 957c453..dc89abe 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;
@@ -36,7 +36,7 @@
}
@Override
- public ISpillableAccumulatingAggregator createSpillableAggregator(IHyracksStageletContext ctx,
+ public ISpillableAccumulatingAggregator createSpillableAggregator(IHyracksTaskContext ctx,
RecordDescriptor inRecordDesc, final RecordDescriptor outRecordDescriptor) {
final ISpillableFieldValueResultingAggregator aggregators[] = new ISpillableFieldValueResultingAggregator[aFactories.length];
for (int i = 0; i < aFactories.length; ++i) {
@@ -122,7 +122,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/AbstractConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractConnectorDescriptor.java
index 0eebd93..19965b9 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
@@ -19,10 +19,10 @@
import org.json.JSONException;
import org.json.JSONObject;
-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 {
@@ -50,7 +50,7 @@
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan) {
// do nothing
}
}
\ No newline at end of file
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 7e97975..8b16068 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
@@ -19,11 +19,11 @@
import org.json.JSONException;
import org.json.JSONObject;
-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 {
@@ -68,7 +68,7 @@
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan) {
// do nothing
}
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 882ee16..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.IEndpointDataWriterFactory;
-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.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 5e73e23..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.IEndpointDataWriterFactory;
-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.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..6037393
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.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.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.common.comm.NonDeterministicPartitionCollector;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+
+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, 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..a8bcfa5
--- /dev/null
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.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.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.common.comm.SortMergePartitionCollector;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+
+public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ITuplePartitionComputerFactory tpcf;
+ private final int[] sortFields;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+
+ public MToNPartitioningMergingConnectorDescriptor(JobSpecification spec, ITuplePartitionComputerFactory tpcf,
+ int[] sortFields, IBinaryComparatorFactory[] comparatorFactories) {
+ super(spec);
+ this.tpcf = tpcf;
+ this.sortFields = sortFields;
+ this.comparatorFactories = comparatorFactories;
+ }
+
+ @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);
+ }
+}
\ 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 e085f29..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.IEndpointDataWriterFactory;
-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.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 deb8d8d..957734c 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.IEndpointDataWriterFactory;
-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.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.common.comm.NonDeterministicPartitionCollector;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
-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,10 @@
}
@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, 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 640ce35..a15a64f 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,20 +14,21 @@
*/
package edu.uci.ics.hyracks.dataflow.std.connectors;
-import edu.uci.ics.hyracks.api.comm.IConnectionDemultiplexer;
-import edu.uci.ics.hyracks.api.comm.IEndpointDataWriterFactory;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import java.util.BitSet;
+
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.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.common.comm.NonDeterministicPartitionCollector;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
public class OneToOneConnectorDescriptor extends AbstractConnectorDescriptor {
@@ -38,27 +39,34 @@
}
@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, expectedPartitions);
}
@Override
- public void contributeSchedulingConstraints(IConstraintExpressionAcceptor constraintAcceptor, JobPlan plan) {
+ public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan) {
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 2f1c8e7..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
@@ -17,31 +17,30 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.comm.IEndpointDataWriterFactory;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksStageletContext;
+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 530207d..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,9 +61,9 @@
* @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 {
+ throws HyracksDataException {
// Output files
final FileSplit[] splits = fileSplitProvider.getFileSplits();
// Frame accessor
@@ -95,7 +95,7 @@
frameTupleAccessor.reset(buffer);
for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
int start = frameTupleAccessor.getTupleStartOffset(tIndex)
- + frameTupleAccessor.getFieldSlotsLength();
+ + frameTupleAccessor.getFieldSlotsLength();
bbis.setByteBuffer(buffer, start);
Object[] record = new Object[recordDescriptor.getFields().length];
for (int i = 0; i < record.length; ++i) {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalHashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalHashGroupOperatorDescriptor.java
index b13b087..4257083 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalHashGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ExternalHashGroupOperatorDescriptor.java
@@ -24,7 +24,7 @@
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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -198,15 +198,14 @@
private static final long serialVersionUID = 1L;
@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) {
// Create the in-memory hash table
final SpillableGroupingHashTable gTable = new SpillableGroupingHashTable(ctx, keyFields,
comparatorFactories, tpcf, aggregatorFactory, recordDescProvider.getInputRecordDescriptor(
getOperatorId(), 0), recordDescriptors[0],
- // Always take one frame for the input records
- framesLimit - 1, tableSize);
+ // Always take one frame for the input records
+ framesLimit - 1, tableSize);
// Create the tuple accessor
final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
@@ -255,7 +254,7 @@
flushFramesToRun();
if (!gTable.insert(accessor, i))
throw new HyracksDataException(
- "Failed to insert a new buffer into the aggregate operator!");
+ "Failed to insert a new buffer into the aggregate operator!");
}
}
@@ -280,7 +279,7 @@
gTable.sortFrames();
FileReference runFile;
try {
- runFile = ctx.getJobletContext().createWorkspaceFile(
+ runFile = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalHashGroupOperatorDescriptor.class.getSimpleName());
} catch (IOException e) {
throw new HyracksDataException(e);
@@ -319,9 +318,8 @@
private static final long serialVersionUID = 1L;
@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 IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -400,7 +398,7 @@
* @throws IOException
*/
private void doPass(LinkedList<RunFileReader> runs, int passCount) throws HyracksDataException,
- IOException {
+ IOException {
FileReference newRun = null;
IFrameWriter writer = this.writer;
boolean finalPass = false;
@@ -424,7 +422,7 @@
}
} else {
// Otherwise, a new run file will be created
- newRun = ctx.getJobletContext().createWorkspaceFile(
+ newRun = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalHashGroupOperatorDescriptor.class.getSimpleName());
writer = new RunFileWriter(newRun, ctx.getIOManager());
writer.open();
@@ -610,7 +608,7 @@
* @throws HyracksDataException
*/
private void closeRun(int index, RunFileReader[] runCursors, IFrameTupleAccessor[] tupleAccessor)
- throws HyracksDataException {
+ throws HyracksDataException {
runCursors[index].close();
runCursors[index] = null;
tupleAccessor[index] = null;
@@ -673,10 +671,10 @@
for (int f = 0; f < keyFields.length; ++f) {
int fIdx = keyFields[f];
int s1 = fta1.getTupleStartOffset(j1) + fta1.getFieldSlotsLength()
- + fta1.getFieldStartOffset(j1, fIdx);
+ + 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);
+ + 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) {
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..8d92c54 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -73,9 +73,8 @@
private static final long serialVersionUID = 1L;
@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() {
@@ -118,7 +117,7 @@
private static final long serialVersionUID = 1L;
@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
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 3fc7d79..5cf94d2 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,14 +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.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;
- ISpillableAccumulatingAggregator createSpillableAggregator(IHyracksStageletContext ctx,
- RecordDescriptor inRecordDesc, RecordDescriptor outRecordDescriptor) throws HyracksDataException;
+ ISpillableAccumulatingAggregator createSpillableAggregator(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/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/group/SpillableGroupingHashTable.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/SpillableGroupingHashTable.java
index b3b9f24..4104c5f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/SpillableGroupingHashTable.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/SpillableGroupingHashTable.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;
@@ -43,7 +43,7 @@
/**
* Context.
*/
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
/**
* Columns for group-by
@@ -164,7 +164,7 @@
* @param tableSize
* The size of the table, which specified the number of partitions of the table.
*/
- public SpillableGroupingHashTable(IHyracksStageletContext ctx, int[] fields,
+ public SpillableGroupingHashTable(IHyracksTaskContext ctx, int[] fields,
IBinaryComparatorFactory[] comparatorFactories, ITuplePartitionComputerFactory tpcf,
IAccumulatingAggregatorFactory aggregatorFactory, RecordDescriptor inRecordDescriptor,
RecordDescriptor outputRecordDescriptor, int framesLimit, int tableSize) {
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 bff101e..1e77547 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -108,9 +108,8 @@
}
@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();
@@ -151,7 +150,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;
@@ -208,9 +207,8 @@
private static final long serialVersionUID = 1L;
@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();
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 d7c1086..faa7afb 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -118,9 +118,8 @@
}
@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];
@@ -279,7 +278,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;
@@ -306,9 +305,8 @@
}
@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];
@@ -484,7 +482,7 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = sWriters[i];
if (writer == null) {
- FileReference file = ctx.createWorkspaceFile(largeRelation);
+ FileReference file = ctx.createManagedWorkspaceFile(largeRelation);
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
sWriters[i] = writer;
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 4d98c6a..a49c21a 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
@@ -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.ITuplePartitionComputer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -38,7 +38,7 @@
private final FrameTuplePairComparator tpComparator;
private final ByteBuffer outBuffer;
- public InMemoryHashJoin(IHyracksStageletContext ctx, int tableSize, FrameTupleAccessor accessor0,
+ public InMemoryHashJoin(IHyracksTaskContext ctx, int tableSize, FrameTupleAccessor accessor0,
ITuplePartitionComputer tpc0, FrameTupleAccessor accessor1, ITuplePartitionComputer tpc1,
FrameTuplePairComparator comparator) {
table = new Link[tableSize];
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 74f0146..6f46932 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -79,9 +79,8 @@
private static final long serialVersionUID = 1L;
@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];
@@ -131,9 +130,8 @@
private static final long serialVersionUID = 1L;
@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;
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..cf46c0c 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;
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..f4fb240 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -60,15 +60,14 @@
private static final long serialVersionUID = 1L;
@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();
@@ -101,9 +100,8 @@
private static final long serialVersionUID = 1L;
@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 {
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 7f097e0..fb02cec 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;
@@ -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 DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
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..a12e49e 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,7 +17,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;
import edu.uci.ics.hyracks.api.dataflow.ActivityNodeId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataWriter;
@@ -48,7 +48,7 @@
}
@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;
@@ -88,7 +88,7 @@
}
@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;
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..c860b2b 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,7 +17,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.api.dataflow.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -86,9 +86,8 @@
}
@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() {
@@ -127,9 +126,8 @@
}
@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/ExternalSortRunGenerator.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 50b968a..742a5f9 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
@@ -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.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -29,12 +29,12 @@
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 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;
@@ -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 {
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..096adaa 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
@@ -21,7 +21,7 @@
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.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -36,7 +36,7 @@
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 int[] sortFields;
@@ -48,7 +48,7 @@
private ByteBuffer outFrame;
private FrameTupleAppender outFrameAppender;
- public ExternalSortRunMerger(IHyracksStageletContext ctx, FrameSorter frameSorter, List<RunFileReader> runs,
+ public ExternalSortRunMerger(IHyracksTaskContext ctx, FrameSorter frameSorter, List<RunFileReader> runs,
int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDesc,
int framesLimit, IFrameWriter writer) {
this.ctx = ctx;
@@ -116,7 +116,7 @@
inFrames.remove(i);
}
} else {
- newRun = ctx.createWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+ newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.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/sort/InMemorySortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 3d41e18..afa27ba 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,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.IActivityGraphBuilder;
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -78,9 +78,8 @@
}
@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() {
@@ -117,9 +116,8 @@
}
@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/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-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 b5468ad..80f0d97 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
@@ -37,7 +37,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;
@@ -198,7 +198,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 204bdf5..bf3e04c 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
@@ -35,7 +35,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.sort.ExternalSortOperatorDescriptor;
@@ -164,7 +164,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/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
index 57b10a3..b618c82 100644
--- a/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.java
+++ b/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/BTreeRegistryProvider.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.btree.dataflow.BTreeRegistry;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
@@ -28,7 +28,7 @@
}
@Override
- public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+ public BTreeRegistry getBTreeRegistry(IHyracksTaskContext ctx) {
return RuntimeContext.get(ctx).getBTreeRegistry();
}
}
\ No newline at end of file
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 9f99468..62ce19d 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.btree.dataflow.BTreeRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -57,7 +57,7 @@
return btreeRegistry;
}
- 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/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 9aac876..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);
- 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/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 4958a35..c0d0211 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
@@ -33,7 +33,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;
@@ -334,12 +334,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);
@@ -413,12 +413,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);
@@ -492,12 +492,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);
@@ -570,12 +570,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);
@@ -654,12 +654,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 694f183..f1942f1 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
@@ -43,7 +43,7 @@
import edu.uci.ics.hyracks.dataflow.std.aggregators.MinMaxAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.MultiAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.aggregators.SumAggregatorFactory;
-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;
@@ -112,7 +112,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -168,7 +168,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
@@ -232,7 +232,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys,
new IBinaryHashFunctionFactory[] { UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, ordScanner, 0, grouper, 0);
@@ -300,7 +300,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
UTF8StringBinaryHashFunctionFactory.INSTANCE, UTF8StringBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, ordScanner, 0, grouper, 0);
@@ -368,7 +368,7 @@
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID);
- IConnectorDescriptor conn1 = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
IntegerBinaryHashFunctionFactory.INSTANCE, IntegerBinaryHashFunctionFactory.INSTANCE }));
spec.connect(conn1, ordScanner, 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 8d22bca..235d966 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
@@ -46,7 +46,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;
@@ -207,7 +207,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 }));
@@ -221,7 +221,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 }));
@@ -251,7 +251,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 }));
@@ -270,7 +270,7 @@
createPartitionConstraint(spec, grouper, outSplits);
// Connect scanner with the grouper
- IConnectorDescriptor scanGroupConnDef = new MToNHashPartitioningConnectorDescriptor(spec,
+ IConnectorDescriptor scanGroupConnDef = new MToNPartitioningConnectorDescriptor(spec,
new FieldHashPartitionComputerFactory(keys, new IBinaryHashFunctionFactory[] {
// IntegerBinaryHashFunctionFactory.INSTANCE,
IntegerBinaryHashFunctionFactory.INSTANCE }));
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 4f20774d..ace8a83 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
@@ -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;
@@ -137,7 +137,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);
@@ -149,7 +149,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 65b9011..13a2c80 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/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..d476a9d 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
@@ -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;
@@ -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/util/HadoopAdapter.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/util/HadoopAdapter.java
index d0df7f1..4c91906 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;
@@ -328,13 +328,13 @@
return externalSortOp;
}
- public static MToNHashPartitioningConnectorDescriptor getMtoNHashPartitioningConnector(JobConf conf,
+ 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 && !conf.getPartitionerClass().getName().startsWith("org.apache.hadoop")) {
@@ -349,7 +349,7 @@
.createSerializerDeserializer(mapOutputKeyClass);
factory = new HadoopHashTuplePartitionComputerFactory(mapOutputKeySerializerDerserializer);
}
- connectorDescriptor = new MToNHashPartitioningConnectorDescriptor(spec, factory);
+ connectorDescriptor = new MToNPartitioningConnectorDescriptor(spec, factory);
return connectorDescriptor;
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
index a4c4b61..48740ed 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorDescriptor.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;
@@ -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) {
return new BTreeBulkLoadOperatorNodePushable(this, ctx, partition, fieldPermutation, fillFactor,
recordDescProvider);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
index 6e13ed7..11456e9 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeBulkLoadOperatorNodePushable.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;
@@ -26,70 +26,62 @@
import edu.uci.ics.hyracks.storage.am.btree.frames.MetaDataFrame;
import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-public class BTreeBulkLoadOperatorNodePushable extends
- AbstractUnaryInputSinkOperatorNodePushable {
- private float fillFactor;
- private final BTreeOpHelper btreeOpHelper;
- private FrameTupleAccessor accessor;
- private BTree.BulkLoadContext bulkLoadCtx;
+public class BTreeBulkLoadOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+ private float fillFactor;
+ private final BTreeOpHelper btreeOpHelper;
+ private FrameTupleAccessor accessor;
+ private BTree.BulkLoadContext bulkLoadCtx;
- private IRecordDescriptorProvider recordDescProvider;
+ private IRecordDescriptorProvider recordDescProvider;
- private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
+ private PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
- public BTreeBulkLoadOperatorNodePushable(
- AbstractBTreeOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
- float fillFactor, IRecordDescriptorProvider recordDescProvider) {
- btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition,
- BTreeOpHelper.BTreeMode.CREATE_BTREE);
- this.fillFactor = fillFactor;
- this.recordDescProvider = recordDescProvider;
- tuple.setFieldPermutation(fieldPermutation);
- }
+ public BTreeBulkLoadOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, int[] fieldPermutation, float fillFactor, IRecordDescriptorProvider recordDescProvider) {
+ btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.CREATE_BTREE);
+ this.fillFactor = fillFactor;
+ this.recordDescProvider = recordDescProvider;
+ tuple.setFieldPermutation(fieldPermutation);
+ }
- @Override
- public void open() throws HyracksDataException {
- AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper
- .getOperatorDescriptor();
- RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(
- opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(btreeOpHelper
- .getHyracksStageletContext().getFrameSize(), recDesc);
- IBTreeMetaDataFrame metaFrame = new MetaDataFrame();
- try {
- btreeOpHelper.init();
- btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
- bulkLoadCtx = btreeOpHelper.getBTree().beginBulkLoad(fillFactor,
- btreeOpHelper.getLeafFrame(),
- btreeOpHelper.getInteriorFrame(), metaFrame);
- } catch (Exception e) {
- // cleanup in case of failure
- btreeOpHelper.deinit();
- throw new HyracksDataException(e);
- }
- }
+ @Override
+ public void open() throws HyracksDataException {
+ AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
+ RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
+ IBTreeMetaDataFrame metaFrame = new MetaDataFrame();
+ try {
+ btreeOpHelper.init();
+ btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
+ bulkLoadCtx = btreeOpHelper.getBTree().beginBulkLoad(fillFactor, btreeOpHelper.getLeafFrame(),
+ btreeOpHelper.getInteriorFrame(), metaFrame);
+ } catch (Exception e) {
+ // cleanup in case of failure
+ btreeOpHelper.deinit();
+ throw new HyracksDataException(e);
+ }
+ }
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- accessor.reset(buffer);
- int tupleCount = accessor.getTupleCount();
- for (int i = 0; i < tupleCount; i++) {
- tuple.reset(accessor, i);
- btreeOpHelper.getBTree().bulkLoadAddTuple(bulkLoadCtx, tuple);
- }
- }
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+ btreeOpHelper.getBTree().bulkLoadAddTuple(bulkLoadCtx, tuple);
+ }
+ }
- @Override
- public void close() throws HyracksDataException {
- try {
- btreeOpHelper.getBTree().endBulkLoad(bulkLoadCtx);
- } finally {
- btreeOpHelper.deinit();
- }
- }
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ btreeOpHelper.getBTree().endBulkLoad(bulkLoadCtx);
+ } finally {
+ btreeOpHelper.deinit();
+ }
+ }
- @Override
- public void flush() throws HyracksDataException {
- }
+ @Override
+ public void flush() throws HyracksDataException {
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
index 17fbf97..af23ad6 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorDescriptor.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.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 BTreeDiskOrderScanOperatorNodePushable(this, ctx, partition);
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
index c362183..5d9748c 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDiskOrderScanOperatorNodePushable.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;
@@ -33,7 +33,7 @@
public class BTreeDiskOrderScanOperatorNodePushable extends AbstractUnaryOutputSourceOperatorNodePushable {
private final BTreeOpHelper btreeOpHelper;
- public BTreeDiskOrderScanOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+ public BTreeDiskOrderScanOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
}
@@ -46,55 +46,55 @@
IBTreeMetaDataFrame metaFrame = new MetaDataFrame();
try {
-
- btreeOpHelper.init();
-
- try {
- btreeOpHelper.getBTree().diskOrderScan(cursor, cursorFrame, metaFrame);
- MultiComparator cmp = btreeOpHelper.getBTree().getMultiComparator();
- ByteBuffer frame = btreeOpHelper.getHyracksStageletContext().allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
- appender.reset(frame, true);
- ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
- DataOutput dos = tb.getDataOutput();
+ btreeOpHelper.init();
- while (cursor.hasNext()) {
- tb.reset();
- cursor.next();
+ try {
+ btreeOpHelper.getBTree().diskOrderScan(cursor, cursorFrame, metaFrame);
- ITupleReference frameTuple = cursor.getTuple();
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ MultiComparator cmp = btreeOpHelper.getBTree().getMultiComparator();
+ ByteBuffer frame = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+ FrameTupleAppender appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext()
+ .getFrameSize());
+ appender.reset(frame, true);
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(cmp.getFieldCount());
+ DataOutput dos = tb.getDataOutput();
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(frame, writer);
- appender.reset(frame, true);
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new IllegalStateException();
- }
- }
- }
+ while (cursor.hasNext()) {
+ tb.reset();
+ cursor.next();
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
- }
- finally {
- cursor.close();
- writer.close();
- }
+ ITupleReference frameTuple = cursor.getTuple();
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
- } catch(Exception e) {
- deinitialize();
- throw new HyracksDataException(e);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(frame, writer);
+ appender.reset(frame, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(frame, writer);
+ }
+ } finally {
+ cursor.close();
+ writer.close();
+ }
+
+ } catch (Exception e) {
+ deinitialize();
+ throw new HyracksDataException(e);
}
}
@Override
public void deinitialize() throws HyracksDataException {
- btreeOpHelper.deinit();
+ btreeOpHelper.deinit();
}
}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
index 970d44a..efbb257 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorDescriptor.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.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
@@ -41,8 +41,9 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
- return new BTreeDropOperatorNodePushable(ctx, storageManager, btreeRegistryProvider, fileSplitProvider, partition);
+ return new BTreeDropOperatorNodePushable(ctx, storageManager, btreeRegistryProvider, fileSplitProvider,
+ partition);
}
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
index c087fdd..6bea88c 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeDropOperatorNodePushable.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;
@@ -30,15 +30,15 @@
import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
public class BTreeDropOperatorNodePushable extends AbstractOperatorNodePushable {
- private static final Logger LOGGER = Logger.getLogger(BTreeDropOperatorNodePushable.class.getName());
-
- private final IHyracksStageletContext ctx;
+ private static final Logger LOGGER = Logger.getLogger(BTreeDropOperatorNodePushable.class.getName());
+
+ private final IHyracksTaskContext ctx;
private IBTreeRegistryProvider btreeRegistryProvider;
private IStorageManagerInterface storageManager;
private IFileSplitProvider fileSplitProvider;
private int partition;
- public BTreeDropOperatorNodePushable(IHyracksStageletContext ctx, IStorageManagerInterface storageManager,
+ public BTreeDropOperatorNodePushable(IHyracksTaskContext ctx, IStorageManagerInterface storageManager,
IBTreeRegistryProvider btreeRegistryProvider, IFileSplitProvider fileSplitProvider, int partition) {
this.ctx = ctx;
this.storageManager = storageManager;
@@ -63,39 +63,40 @@
@Override
public void initialize() throws HyracksDataException {
- try {
+ try {
- BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry(ctx);
- IBufferCache bufferCache = storageManager.getBufferCache(ctx);
- IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
+ BTreeRegistry btreeRegistry = btreeRegistryProvider.getBTreeRegistry(ctx);
+ IBufferCache bufferCache = storageManager.getBufferCache(ctx);
+ IFileMapProvider fileMapProvider = storageManager.getFileMapProvider(ctx);
- FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
+ FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
- boolean fileIsMapped = fileMapProvider.isMapped(f);
- if (!fileIsMapped) {
- throw new HyracksDataException("Cannot drop B-Tree with name " + f.toString() + ". No file mapping exists.");
- }
+ boolean fileIsMapped = fileMapProvider.isMapped(f);
+ if (!fileIsMapped) {
+ throw new HyracksDataException("Cannot drop B-Tree with name " + f.toString()
+ + ". No file mapping exists.");
+ }
- int btreeFileId = fileMapProvider.lookupFileId(f);
+ int btreeFileId = fileMapProvider.lookupFileId(f);
- // unregister btree instance
- btreeRegistry.lock();
- try {
- btreeRegistry.unregister(btreeFileId);
- } finally {
- btreeRegistry.unlock();
- }
+ // unregister btree instance
+ btreeRegistry.lock();
+ try {
+ btreeRegistry.unregister(btreeFileId);
+ } finally {
+ btreeRegistry.unlock();
+ }
- // remove name to id mapping
- bufferCache.deleteFile(btreeFileId);
- }
- // TODO: for the time being we don't throw,
- // with proper exception handling (no hanging job problem) we should throw
- catch (Exception e) {
- if (LOGGER.isLoggable(Level.WARNING)) {
+ // remove name to id mapping
+ bufferCache.deleteFile(btreeFileId);
+ }
+ // TODO: for the time being we don't throw,
+ // with proper exception handling (no hanging job problem) we should throw
+ catch (Exception e) {
+ if (LOGGER.isLoggable(Level.WARNING)) {
LOGGER.warning("BTRee Drop Operator Failed Due To Exception: " + e.getMessage());
}
- }
+ }
}
@Override
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
index c105274..9939c6f 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorDescriptor.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;
@@ -47,7 +47,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 BTreeFileEnlistmentOperatorNodePushable(this, ctx, partition);
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
index 023cd40..6099095 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeFileEnlistmentOperatorNodePushable.java
@@ -16,7 +16,7 @@
package edu.uci.ics.hyracks.storage.am.btree.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;
@@ -25,7 +25,7 @@
private final BTreeOpHelper btreeOpHelper;
- public BTreeFileEnlistmentOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksStageletContext ctx,
+ public BTreeFileEnlistmentOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition) {
btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.ENLIST_BTREE);
}
@@ -47,10 +47,9 @@
@Override
public void initialize() throws HyracksDataException {
try {
- btreeOpHelper.init();
- }
- finally {
- btreeOpHelper.deinit();
+ btreeOpHelper.init();
+ } finally {
+ btreeOpHelper.deinit();
}
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
index 7d50fff..bcae5e1 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorDescriptor.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;
@@ -49,7 +49,7 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksStageletContext ctx, IOperatorEnvironment env,
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IOperatorEnvironment env,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new BTreeInsertUpdateDeleteOperatorNodePushable(this, ctx, partition, fieldPermutation,
recordDescProvider, op);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
index acb7d0f..ec6da88 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeInsertUpdateDeleteOperatorNodePushable.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;
@@ -37,9 +37,8 @@
private ByteBuffer writeBuffer;
private BTreeOpContext opCtx;
- public BTreeInsertUpdateDeleteOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition, int[] fieldPermutation,
- IRecordDescriptorProvider recordDescProvider, BTreeOp op) {
+ public BTreeInsertUpdateDeleteOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, BTreeOp op) {
btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
this.recordDescProvider = recordDescProvider;
this.op = op;
@@ -48,20 +47,20 @@
@Override
public void open() throws HyracksDataException {
- AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
- RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
- writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
- try {
- btreeOpHelper.init();
- btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
- opCtx = btreeOpHelper.getBTree().createOpContext(op, btreeOpHelper.getLeafFrame(),
- btreeOpHelper.getInteriorFrame(), new MetaDataFrame());
- } catch(Exception e) {
- // cleanup in case of failure
- btreeOpHelper.deinit();
- throw new HyracksDataException(e);
- }
+ AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), inputRecDesc);
+ writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+ try {
+ btreeOpHelper.init();
+ btreeOpHelper.getBTree().open(btreeOpHelper.getBTreeFileId());
+ opCtx = btreeOpHelper.getBTree().createOpContext(op, btreeOpHelper.getLeafFrame(),
+ btreeOpHelper.getInteriorFrame(), new MetaDataFrame());
+ } catch (Exception e) {
+ // cleanup in case of failure
+ btreeOpHelper.deinit();
+ throw new HyracksDataException(e);
+ }
}
@Override
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 880cc25..528b8cf 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
@@ -14,7 +14,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.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -29,31 +29,32 @@
final class BTreeOpHelper {
- public enum BTreeMode {
- OPEN_BTREE, CREATE_BTREE, ENLIST_BTREE
- }
-
- private IBTreeInteriorFrame interiorFrame;
- private IBTreeLeafFrame leafFrame;
+ public enum BTreeMode {
+ OPEN_BTREE,
+ CREATE_BTREE,
+ ENLIST_BTREE
+ }
- private BTree btree;
- private int btreeFileId = -1;
- private int partition;
+ private IBTreeInteriorFrame interiorFrame;
+ private IBTreeLeafFrame leafFrame;
+
+ private BTree btree;
+ private int btreeFileId = -1;
+ private int partition;
private AbstractBTreeOperatorDescriptor opDesc;
- private IHyracksStageletContext ctx;
+ private IHyracksTaskContext ctx;
- private BTreeMode mode;
+ private BTreeMode mode;
- BTreeOpHelper(AbstractBTreeOperatorDescriptor opDesc, final IHyracksStageletContext ctx, int partition,
- BTreeMode mode) {
+ BTreeOpHelper(AbstractBTreeOperatorDescriptor opDesc, final IHyracksTaskContext ctx, int partition, BTreeMode mode) {
this.opDesc = opDesc;
this.ctx = ctx;
this.mode = mode;
this.partition = partition;
}
- void init() throws HyracksDataException {
+ void init() throws HyracksDataException {
IBufferCache bufferCache = opDesc.getStorageManager().getBufferCache(ctx);
IFileMapProvider fileMapProvider = opDesc.getStorageManager().getFileMapProvider(ctx);
IFileSplitProvider fileSplitProvider = opDesc.getFileSplitProvider();
@@ -61,72 +62,67 @@
FileReference f = fileSplitProvider.getFileSplits()[partition].getLocalFile();
boolean fileIsMapped = fileMapProvider.isMapped(f);
- switch (mode) {
-
- case OPEN_BTREE: {
- if (!fileIsMapped) {
- throw new HyracksDataException(
- "Trying to open btree from unmapped file " + f.toString());
- }
- }
- break;
+ switch (mode) {
- case CREATE_BTREE:
- case ENLIST_BTREE: {
- if (!fileIsMapped) {
- bufferCache.createFile(f);
- }
- }
- break;
-
- }
+ case OPEN_BTREE: {
+ if (!fileIsMapped) {
+ throw new HyracksDataException("Trying to open btree from unmapped file " + f.toString());
+ }
+ }
+ break;
+
+ case CREATE_BTREE:
+ case ENLIST_BTREE: {
+ if (!fileIsMapped) {
+ bufferCache.createFile(f);
+ }
+ }
+ break;
+
+ }
btreeFileId = fileMapProvider.lookupFileId(f);
bufferCache.openFile(btreeFileId);
- interiorFrame = opDesc.getInteriorFactory().getFrame();
- leafFrame = opDesc.getLeafFactory().getFrame();
+ interiorFrame = opDesc.getInteriorFactory().getFrame();
+ leafFrame = opDesc.getLeafFactory().getFrame();
BTreeRegistry btreeRegistry = opDesc.getBtreeRegistryProvider().getBTreeRegistry(ctx);
btree = btreeRegistry.get(btreeFileId);
if (btree == null) {
- // create new btree and register it
- btreeRegistry.lock();
- try {
- // check if btree has already been registered by another thread
- btree = btreeRegistry.get(btreeFileId);
- if (btree == null) {
- // this thread should create and register the btree
+ // create new btree and register it
+ btreeRegistry.lock();
+ try {
+ // check if btree has already been registered by another thread
+ btree = btreeRegistry.get(btreeFileId);
+ if (btree == null) {
+ // this thread should create and register the btree
- IBinaryComparator[] comparators = new IBinaryComparator[opDesc
- .getComparatorFactories().length];
- for (int i = 0; i < opDesc.getComparatorFactories().length; i++) {
- comparators[i] = opDesc.getComparatorFactories()[i]
- .createBinaryComparator();
- }
+ IBinaryComparator[] comparators = new IBinaryComparator[opDesc.getComparatorFactories().length];
+ for (int i = 0; i < opDesc.getComparatorFactories().length; i++) {
+ comparators[i] = opDesc.getComparatorFactories()[i].createBinaryComparator();
+ }
- MultiComparator cmp = new MultiComparator(opDesc
- .getTypeTraits(), comparators);
+ MultiComparator cmp = new MultiComparator(opDesc.getTypeTraits(), comparators);
- btree = new BTree(bufferCache, opDesc.getInteriorFactory(),
- opDesc.getLeafFactory(), cmp);
- if (mode == BTreeMode.CREATE_BTREE) {
- MetaDataFrame metaFrame = new MetaDataFrame();
- try {
- btree.create(btreeFileId, leafFrame, metaFrame);
- btree.open(btreeFileId);
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- }
- btreeRegistry.register(btreeFileId, btree);
- }
- } finally {
- btreeRegistry.unlock();
- }
- }
- }
+ btree = new BTree(bufferCache, opDesc.getInteriorFactory(), opDesc.getLeafFactory(), cmp);
+ if (mode == BTreeMode.CREATE_BTREE) {
+ MetaDataFrame metaFrame = new MetaDataFrame();
+ try {
+ btree.create(btreeFileId, leafFrame, metaFrame);
+ btree.open(btreeFileId);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ btreeRegistry.register(btreeFileId, btree);
+ }
+ } finally {
+ btreeRegistry.unlock();
+ }
+ }
+ }
public void deinit() throws HyracksDataException {
if (btreeFileId != -1) {
@@ -135,27 +131,27 @@
}
}
- public BTree getBTree() {
- return btree;
- }
+ public BTree getBTree() {
+ return btree;
+ }
- public IHyracksStageletContext getHyracksStageletContext() {
+ public IHyracksTaskContext getHyracksStageletContext() {
return ctx;
}
- public AbstractBTreeOperatorDescriptor getOperatorDescriptor() {
- return opDesc;
- }
+ public AbstractBTreeOperatorDescriptor getOperatorDescriptor() {
+ return opDesc;
+ }
- public IBTreeLeafFrame getLeafFrame() {
- return leafFrame;
- }
+ public IBTreeLeafFrame getLeafFrame() {
+ return leafFrame;
+ }
- public IBTreeInteriorFrame getInteriorFrame() {
- return interiorFrame;
- }
+ public IBTreeInteriorFrame getInteriorFrame() {
+ return interiorFrame;
+ }
- public int getBTreeFileId() {
- return btreeFileId;
- }
+ public int getBTreeFileId() {
+ return btreeFileId;
+ }
}
\ No newline at end of file
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 21f37a3..efb616b 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;
@@ -55,7 +55,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 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 415f169..819dea6 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;
@@ -37,194 +37,172 @@
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangeSearchCursor;
-public class BTreeSearchOperatorNodePushable extends
- AbstractUnaryInputUnaryOutputOperatorNodePushable {
- private BTreeOpHelper btreeOpHelper;
- private FrameTupleAccessor accessor;
+public class BTreeSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+ private BTreeOpHelper btreeOpHelper;
+ private FrameTupleAccessor accessor;
- private ByteBuffer writeBuffer;
- private FrameTupleAppender appender;
- private ArrayTupleBuilder tb;
- private DataOutput dos;
+ private ByteBuffer writeBuffer;
+ private FrameTupleAppender appender;
+ private ArrayTupleBuilder tb;
+ private DataOutput dos;
- private BTree btree;
- private boolean isForward;
- private PermutingFrameTupleReference lowKey;
- private PermutingFrameTupleReference highKey;
- private boolean lowKeyInclusive;
- private boolean highKeyInclusive;
- private RangePredicate rangePred;
- private MultiComparator lowKeySearchCmp;
- private MultiComparator highKeySearchCmp;
- private IBTreeCursor cursor;
- private IBTreeLeafFrame cursorFrame;
- private BTreeOpContext opCtx;
+ private BTree btree;
+ private boolean isForward;
+ private PermutingFrameTupleReference lowKey;
+ private PermutingFrameTupleReference highKey;
+ private boolean lowKeyInclusive;
+ private boolean highKeyInclusive;
+ private RangePredicate rangePred;
+ private MultiComparator lowKeySearchCmp;
+ private MultiComparator highKeySearchCmp;
+ private IBTreeCursor cursor;
+ private IBTreeLeafFrame cursorFrame;
+ private BTreeOpContext opCtx;
- private RecordDescriptor recDesc;
+ private RecordDescriptor recDesc;
- public BTreeSearchOperatorNodePushable(
- AbstractBTreeOperatorDescriptor opDesc,
- IHyracksStageletContext ctx, int partition,
- IRecordDescriptorProvider recordDescProvider, boolean isForward,
- int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
- boolean highKeyInclusive) {
- btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition,
- BTreeOpHelper.BTreeMode.OPEN_BTREE);
- this.isForward = isForward;
- this.lowKeyInclusive = lowKeyInclusive;
- this.highKeyInclusive = highKeyInclusive;
- this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc
- .getOperatorId(), 0);
- if (lowKeyFields != null && lowKeyFields.length > 0) {
- lowKey = new PermutingFrameTupleReference();
- lowKey.setFieldPermutation(lowKeyFields);
- }
- if (highKeyFields != null && highKeyFields.length > 0) {
- highKey = new PermutingFrameTupleReference();
- highKey.setFieldPermutation(highKeyFields);
- }
- }
+ public BTreeSearchOperatorNodePushable(AbstractBTreeOperatorDescriptor opDesc, IHyracksTaskContext ctx,
+ int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, int[] lowKeyFields,
+ int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) {
+ btreeOpHelper = new BTreeOpHelper(opDesc, ctx, partition, BTreeOpHelper.BTreeMode.OPEN_BTREE);
+ this.isForward = isForward;
+ this.lowKeyInclusive = lowKeyInclusive;
+ this.highKeyInclusive = highKeyInclusive;
+ this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ if (lowKeyFields != null && lowKeyFields.length > 0) {
+ lowKey = new PermutingFrameTupleReference();
+ lowKey.setFieldPermutation(lowKeyFields);
+ }
+ if (highKeyFields != null && highKeyFields.length > 0) {
+ highKey = new PermutingFrameTupleReference();
+ highKey.setFieldPermutation(highKeyFields);
+ }
+ }
- @Override
- public void open() throws HyracksDataException {
- AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper
- .getOperatorDescriptor();
- accessor = new FrameTupleAccessor(btreeOpHelper
- .getHyracksStageletContext().getFrameSize(), recDesc);
+ @Override
+ public void open() throws HyracksDataException {
+ AbstractBTreeOperatorDescriptor opDesc = btreeOpHelper.getOperatorDescriptor();
+ accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
- cursorFrame = opDesc.getLeafFactory().getFrame();
- cursor = new RangeSearchCursor(cursorFrame);
+ cursorFrame = opDesc.getLeafFactory().getFrame();
+ cursor = new RangeSearchCursor(cursorFrame);
- try {
+ try {
- btreeOpHelper.init();
- btree = btreeOpHelper.getBTree();
+ btreeOpHelper.init();
+ btree = btreeOpHelper.getBTree();
- // construct range predicate
+ // construct range predicate
- int lowKeySearchFields = btree.getMultiComparator()
- .getComparators().length;
- int highKeySearchFields = btree.getMultiComparator()
- .getComparators().length;
- if (lowKey != null)
- lowKeySearchFields = lowKey.getFieldCount();
- if (highKey != null)
- highKeySearchFields = highKey.getFieldCount();
+ int lowKeySearchFields = btree.getMultiComparator().getComparators().length;
+ int highKeySearchFields = btree.getMultiComparator().getComparators().length;
+ if (lowKey != null)
+ lowKeySearchFields = lowKey.getFieldCount();
+ if (highKey != null)
+ highKeySearchFields = highKey.getFieldCount();
- IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
- for (int i = 0; i < lowKeySearchFields; i++) {
- lowKeySearchComparators[i] = btree.getMultiComparator()
- .getComparators()[i];
- }
- lowKeySearchCmp = new MultiComparator(btree.getMultiComparator()
- .getTypeTraits(), lowKeySearchComparators);
+ IBinaryComparator[] lowKeySearchComparators = new IBinaryComparator[lowKeySearchFields];
+ for (int i = 0; i < lowKeySearchFields; i++) {
+ lowKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
+ }
+ lowKeySearchCmp = new MultiComparator(btree.getMultiComparator().getTypeTraits(), lowKeySearchComparators);
- if (lowKeySearchFields == highKeySearchFields) {
- highKeySearchCmp = lowKeySearchCmp;
- } else {
- IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
- for (int i = 0; i < highKeySearchFields; i++) {
- highKeySearchComparators[i] = btree.getMultiComparator()
- .getComparators()[i];
- }
- highKeySearchCmp = new MultiComparator(btree
- .getMultiComparator().getTypeTraits(),
- highKeySearchComparators);
+ if (lowKeySearchFields == highKeySearchFields) {
+ highKeySearchCmp = lowKeySearchCmp;
+ } else {
+ IBinaryComparator[] highKeySearchComparators = new IBinaryComparator[highKeySearchFields];
+ for (int i = 0; i < highKeySearchFields; i++) {
+ highKeySearchComparators[i] = btree.getMultiComparator().getComparators()[i];
+ }
+ highKeySearchCmp = new MultiComparator(btree.getMultiComparator().getTypeTraits(),
+ highKeySearchComparators);
- }
+ }
- rangePred = new RangePredicate(isForward, null, null,
- lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
- highKeySearchCmp);
+ rangePred = new RangePredicate(isForward, null, null, lowKeyInclusive, highKeyInclusive, lowKeySearchCmp,
+ highKeySearchCmp);
- accessor = new FrameTupleAccessor(btreeOpHelper
- .getHyracksStageletContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(btreeOpHelper.getHyracksStageletContext().getFrameSize(), recDesc);
- writeBuffer = btreeOpHelper.getHyracksStageletContext()
- .allocateFrame();
- tb = new ArrayTupleBuilder(btree.getMultiComparator()
- .getFieldCount());
- dos = tb.getDataOutput();
- appender = new FrameTupleAppender(btreeOpHelper
- .getHyracksStageletContext().getFrameSize());
- appender.reset(writeBuffer, true);
+ writeBuffer = btreeOpHelper.getHyracksStageletContext().allocateFrame();
+ tb = new ArrayTupleBuilder(btree.getMultiComparator().getFieldCount());
+ dos = tb.getDataOutput();
+ appender = new FrameTupleAppender(btreeOpHelper.getHyracksStageletContext().getFrameSize());
+ appender.reset(writeBuffer, true);
- opCtx = btree.createOpContext(BTreeOp.BTO_SEARCH, btreeOpHelper
- .getLeafFrame(), btreeOpHelper.getInteriorFrame(), null);
+ opCtx = btree.createOpContext(BTreeOp.BTO_SEARCH, btreeOpHelper.getLeafFrame(),
+ btreeOpHelper.getInteriorFrame(), null);
- } catch (Exception e) {
- btreeOpHelper.deinit();
- }
- }
+ } catch (Exception e) {
+ btreeOpHelper.deinit();
+ }
+ }
- private void writeSearchResults() throws Exception {
- while (cursor.hasNext()) {
- tb.reset();
- cursor.next();
+ private void writeSearchResults() throws Exception {
+ while (cursor.hasNext()) {
+ tb.reset();
+ cursor.next();
- ITupleReference frameTuple = cursor.getTuple();
- for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple
- .getFieldStart(i), frameTuple.getFieldLength(i));
- tb.addFieldEndOffset();
- }
+ ITupleReference frameTuple = cursor.getTuple();
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0,
- tb.getSize())) {
- FrameUtils.flushFrame(writeBuffer, writer);
- appender.reset(writeBuffer, true);
- if (!appender.append(tb.getFieldEndOffsets(),
- tb.getByteArray(), 0, tb.getSize())) {
- throw new IllegalStateException();
- }
- }
- }
- }
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ appender.reset(writeBuffer, true);
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+ }
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- accessor.reset(buffer);
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
- int tupleCount = accessor.getTupleCount();
- try {
- for (int i = 0; i < tupleCount; i++) {
- if (lowKey != null)
- lowKey.reset(accessor, i);
- if (highKey != null)
- highKey.reset(accessor, i);
- rangePred.setLowKey(lowKey, lowKeyInclusive);
- rangePred.setHighKey(highKey, highKeyInclusive);
+ int tupleCount = accessor.getTupleCount();
+ try {
+ for (int i = 0; i < tupleCount; i++) {
+ if (lowKey != null)
+ lowKey.reset(accessor, i);
+ if (highKey != null)
+ highKey.reset(accessor, i);
+ rangePred.setLowKey(lowKey, lowKeyInclusive);
+ rangePred.setHighKey(highKey, highKeyInclusive);
- cursor.reset();
- btree.search(cursor, rangePred, opCtx);
- writeSearchResults();
- }
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- }
+ cursor.reset();
+ btree.search(cursor, rangePred, opCtx);
+ writeSearchResults();
+ }
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ }
- @Override
- public void close() throws HyracksDataException {
- try {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(writeBuffer, writer);
- }
- writer.close();
- try {
- cursor.close();
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- } finally {
- btreeOpHelper.deinit();
- }
- }
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ }
+ writer.close();
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ } finally {
+ btreeOpHelper.deinit();
+ }
+ }
- @Override
- public void flush() throws HyracksDataException {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(writeBuffer, writer);
- }
- }
+ @Override
+ public void flush() throws HyracksDataException {
+ if (appender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(writeBuffer, writer);
+ }
+ }
}
\ No newline at end of file
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java
index 7b66d0a..caf8615 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/IBTreeRegistryProvider.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 IBTreeRegistryProvider extends Serializable {
- public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx);
+ public BTreeRegistry getBTreeRegistry(IHyracksTaskContext ctx);
}
\ No newline at end of file
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 059a1e2..6e7e0d3 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.createBinaryTokenizer(), 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 9dac535..c6f6e5b 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;
@@ -31,7 +31,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;
@@ -44,7 +44,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;
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
index 6ad5a9c..f88439c 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/SimpleConjunctiveSearcher.java
@@ -21,7 +21,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;
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.RecordDescriptor;
@@ -54,7 +54,7 @@
private final IBinaryComparator[] valueCmps;
private final BTree btree;
- private final IHyracksStageletContext ctx;
+ private final IHyracksTaskContext ctx;
private final ArrayTupleBuilder resultTupleBuilder;
private final FrameTupleAppender resultTupleAppender;
private final FrameTupleAccessor resultFrameAccessor;
@@ -73,7 +73,7 @@
private final IBinaryTokenizer queryTokenizer;
- public SimpleConjunctiveSearcher(IHyracksStageletContext ctx, BTree btree, RecordDescriptor btreeRecDesc,
+ public SimpleConjunctiveSearcher(IHyracksTaskContext ctx, BTree btree, RecordDescriptor btreeRecDesc,
IBinaryTokenizer queryTokenizer, int numKeyFields, int numValueFields) {
this.ctx = ctx;
this.btree = btree;
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-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
index 3ab0427..864f916 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestBTreeRegistryProvider.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.btree.dataflow.BTreeRegistry;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.IBTreeRegistryProvider;
@@ -22,7 +22,7 @@
private static final long serialVersionUID = 1L;
@Override
- public BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+ public BTreeRegistry getBTreeRegistry(IHyracksTaskContext ctx) {
return TestStorageManagerComponentHolder.getBTreeRegistry(ctx);
}
}
\ No newline at end of file
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..620e1a5 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
@@ -26,19 +26,17 @@
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 WorkspaceFileFactory fileFactory;
- public TestJobletContext(INCApplicationContext appContext, UUID jobId, int attempt) throws HyracksException {
+ public TestJobletContext(INCApplicationContext appContext, UUID 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
@@ -85,9 +88,4 @@
public UUID 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/TestStorageManagerComponentHolder.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStorageManagerComponentHolder.java
index 5338d5e..25e6f3f 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.btree.dataflow.BTreeRegistry;
import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
@@ -42,7 +42,7 @@
btreeRegistry = 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();
@@ -53,14 +53,14 @@
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 BTreeRegistry getBTreeRegistry(IHyracksStageletContext ctx) {
+ public synchronized static BTreeRegistry getBTreeRegistry(IHyracksTaskContext ctx) {
if (btreeRegistry == null) {
btreeRegistry = new BTreeRegistry();
}
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/TestStageletContext.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
similarity index 68%
rename from hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestStageletContext.java
rename to hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index 4d31326..589352a 100644
--- 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/TestTaskContext.java
@@ -15,10 +15,10 @@
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.context.IHyracksTaskContext;
+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;
@@ -26,17 +26,17 @@
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 TestStageletContext implements IHyracksStageletContext {
+public class TestTaskContext implements IHyracksTaskContext {
private final IHyracksJobletContext jobletContext;
- private UUID stageId;
- private ManagedWorkspaceFileFactory fileFactory;
+ private final TaskAttemptId taskId;
+ private WorkspaceFileFactory fileFactory;
- public TestStageletContext(IHyracksJobletContext jobletContext, UUID stageId) throws HyracksException {
+ public TestTaskContext(IHyracksJobletContext jobletContext, TaskAttemptId taskId) throws HyracksException {
this.jobletContext = jobletContext;
- this.stageId = stageId;
- fileFactory = new ManagedWorkspaceFileFactory(this, (IOManager) getIOManager());
+ this.taskId = taskId;
+ fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
}
@Override
@@ -55,8 +55,13 @@
}
@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
@@ -65,13 +70,8 @@
}
@Override
- public UUID getStageId() {
- return stageId;
- }
-
- @Override
public ICounterContext getCounterContext() {
- return new CounterContext(jobletContext.getJobId() + "." + jobletContext.getAttempt() + "." + stageId);
+ return new CounterContext(jobletContext.getJobId() + "." + taskId);
}
@Override
@@ -83,4 +83,9 @@
}
});
}
+
+ @Override
+ public TaskAttemptId getTaskId() {
+ return taskId;
+ }
}
\ 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 2edeaff..71c4a24 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
@@ -19,17 +19,23 @@
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.ActivityNodeId;
+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;
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);
- IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID(), 0);
- IHyracksStageletContext stageletCtx = new TestStageletContext(jobletCtx, UUID.randomUUID());
- return stageletCtx;
+ IHyracksJobletContext jobletCtx = new TestJobletContext(appCtx, UUID.randomUUID());
+ TaskAttemptId tid = new TaskAttemptId(new TaskId(new ActivityNodeId(new OperatorDescriptorId(
+ UUID.randomUUID()), UUID.randomUUID()), 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 e856296..8e3be9c 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,7 +59,7 @@
private static final int PAGE_SIZE = 32768; // 32K
private static final int NUM_PAGES = 40;
private static final int HYRACKS_FRAME_SIZE = 128;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
public class BufferAllocator implements ICacheMemoryAllocator {
@Override
@@ -72,7 +72,7 @@
}
}
- private ITupleReference createTuple(IHyracksStageletContext ctx, int f0,
+ private ITupleReference createTuple(IHyracksTaskContext ctx, int f0,
int f1, int f2, boolean print) throws HyracksDataException {
if (print)
System.out.println("CREATING: " + f0 + " " + f1 + " " + f2);
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 2985601..f9f8b89 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;
@@ -71,7 +71,7 @@
private static final int PAGE_SIZE = 256;
private static final int NUM_PAGES = 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);
public class BufferAllocator implements ICacheMemoryAllocator {
@Override
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 12371be..26a52b1 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;
@@ -102,7 +102,7 @@
IBTreeInteriorFrame interiorFrame = interiorFrameFactory.getFrame();
IBTreeMetaDataFrame metaFrame = metaFrameFactory.getFrame();
- 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 430a33c..709dd41 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,7 +22,7 @@
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;
@@ -36,7 +36,7 @@
public class StorageManagerTest extends AbstractBTreeTest {
private static final int PAGE_SIZE = 256;
private static final int NUM_PAGES = 10;
- private IHyracksStageletContext ctx = TestUtils.create(32768);
+ private IHyracksTaskContext ctx = TestUtils.create(32768);
public class PinnedLatchedPage {
public final ICachedPage page;
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
index f61249b..88b801f 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/searchers/SimpleConjunctiveSearcherTest.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;
@@ -79,7 +79,7 @@
private static final int PAGE_SIZE = 32768;
private static final int NUM_PAGES = 10;
private static final int HYRACKS_FRAME_SIZE = 32768;
- private IHyracksStageletContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
+ private IHyracksTaskContext ctx = TestUtils.create(HYRACKS_FRAME_SIZE);
public class BufferAllocator implements ICacheMemoryAllocator {
@Override
@@ -224,7 +224,7 @@
RecordDescriptor resultRecDesc = new RecordDescriptor(resultSerde);
FrameTupleAccessor resultAccessor = new FrameTupleAccessor(ctx.getFrameSize(), resultRecDesc);
FrameTupleReference resultTuple = new FrameTupleReference();
-
+
SimpleConjunctiveSearcher searcher = new SimpleConjunctiveSearcher(ctx, btree, btreeRecDesc, queryTokenizer,
numKeyFields, numValueFields);
@@ -241,8 +241,8 @@
for (int i = 0; i < resultAccessor.getTupleCount(); i++) {
resultTuple.reset(resultAccessor, i);
for (int j = 0; j < resultTuple.getFieldCount(); j++) {
- ByteArrayInputStream inStream = new ByteArrayInputStream(resultTuple.getFieldData(j), resultTuple
- .getFieldStart(j), resultTuple.getFieldLength(j));
+ ByteArrayInputStream inStream = new ByteArrayInputStream(resultTuple.getFieldData(j),
+ resultTuple.getFieldStart(j), resultTuple.getFieldLength(j));
DataInput dataIn = new DataInputStream(inStream);
Object o = resultSerde[j].deserialize(dataIn);
System.out.print(o + " ");