VariableSizeFrame(VSizeFrame) support for Hyracks.
This patch replaced Frame/Accessor/Appender with the new API which
supports BigObject.
The ExternalSorter/TopKSorter/ExternalGroupSorter
have been implemented to support big object.
The Groupby && Join should work with BigObject also. But it will break the
memory budget when it encounter a big object. I will fix the memory
problem later in a separate CR.
The design about the frame allocation is
here:https://docs.google.com/presentation/d/15h9iQf5OYsgGZoQTbGHkj1yS2G9q2fd0s1lDAD1EJq0/edit?usp=sharing
Suggest review order:
Patch 12: It includes all of the sorting operators.
Patch 13: It applys the new IFrame API to all Hyracks codes.
Patch 14: Some bug fixes to pass all Asterix's tests.
Patch 15: Skip it!
Patch 16: Some bug fixes to the Asterix's tests in small frame setting.
Later Patch: address the comments
Change-Id: I2e08692078683f6f2cf17387e39037ad851fc05b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/234
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java
new file mode 100644
index 0000000..ee92084
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FixedSizeFrame.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FixedSizeFrame implements IFrame{
+
+ private final ByteBuffer buffer;
+
+ public FixedSizeFrame(ByteBuffer buffer){
+ this.buffer = buffer;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public void ensureFrameSize(int frameSize) throws HyracksDataException {
+ throw new HyracksDataException("FixedSizeFrame doesn't support capacity changes");
+ }
+
+ @Override
+ public void resize(int frameSize) throws HyracksDataException {
+ throw new HyracksDataException("FixedSizeFrame doesn't support capacity changes");
+ }
+
+ @Override
+ public int getFrameSize() {
+ return buffer.capacity();
+ }
+
+ @Override
+ public int getMinSize() {
+ return buffer.capacity() / FrameHelper.deserializeNumOfMinFrame(buffer, 0);
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ buffer.clear();
+ }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
index 0dc97bc..176f23e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameConstants.java
@@ -15,9 +15,34 @@
package edu.uci.ics.hyracks.api.comm;
public interface FrameConstants {
- public static final int SIZE_LEN = 4;
+ /**
+ * We use 4bytes to store the tupleCount at the end of the Frame.
+ */
+ int SIZE_LEN = 4;
- public static final boolean DEBUG_FRAME_IO = false;
+ /**
+ * The offset of the frame_count which is one byte indicate how many initial_frames contained in current frame.
+ * The actual frameSize = frame_count * intitialFrameSize(given by user)
+ */
+ int META_DATA_FRAME_COUNT_OFFSET = 0;
- public static final int FRAME_FIELD_MAGIC = 0x12345678;
+ /**
+ * The start offset of the tuple data. The first byte is used to store the frame_count
+ */
+ int TUPLE_START_OFFSET = 1;
+
+ /**
+ * Since we use one byte to store the frame_count, the max frame_count is 255.
+ */
+ int MAX_NUM_MINFRAME = 255;
+
+ /**
+ * Indicate the total size of the meta data.
+ */
+ int META_DATA_LEN = SIZE_LEN + TUPLE_START_OFFSET;
+
+ boolean DEBUG_FRAME_IO = false;
+
+ int FRAME_FIELD_MAGIC = 0x12345678;
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
index a6774c7..2376d2e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/FrameHelper.java
@@ -14,8 +14,65 @@
*/
package edu.uci.ics.hyracks.api.comm;
+import java.nio.ByteBuffer;
+
public class FrameHelper {
public static int getTupleCountOffset(int frameSize) {
- return frameSize - 4;
+ return frameSize - FrameConstants.SIZE_LEN;
}
-}
\ No newline at end of file
+
+ /**
+ * The actual frameSize = frameCount * intitialFrameSize
+ * This method is used to put that frameCount into the first byte of the frame buffer.
+ * @param outputFrame
+ * @param numberOfMinFrame
+ */
+ public static void serializeFrameSize(ByteBuffer outputFrame, byte numberOfMinFrame) {
+ serializeFrameSize(outputFrame, 0, numberOfMinFrame);
+ }
+
+ public static void serializeFrameSize(ByteBuffer outputFrame, int start, byte numberOfMinFrame) {
+ outputFrame.array()[start + FrameConstants.META_DATA_FRAME_COUNT_OFFSET] = (byte) (numberOfMinFrame & 0xff);
+ }
+
+ public static byte deserializeNumOfMinFrame(ByteBuffer frame) {
+ return deserializeNumOfMinFrame(frame, 0);
+ }
+
+ public static byte deserializeNumOfMinFrame(ByteBuffer buffer, int start) {
+ return (byte) (buffer.array()[start + FrameConstants.META_DATA_FRAME_COUNT_OFFSET] & 0xff);
+ }
+
+ /**
+ * Add one tuple requires
+ * 4bytes to store the tuple offset
+ * 4bytes * |fields| to store the relative offset of each field
+ * nbytes the actual data.
+ * If the tupleLength includes the field slot, please set the fieldCount = 0
+ */
+ public static int calcSpaceInFrame(int fieldCount, int tupleLength) {
+ return 4 + fieldCount * 4 + tupleLength;
+ }
+
+ /**
+ * A faster way of calculating the ceiling
+ *
+ * @param fieldCount please set fieldCount to 0 if the tupleLength includes the fields' length
+ * @param tupleLength
+ * @param minFrameSize
+ * @return
+ */
+ public static int calcAlignedFrameSizeToStore(int fieldCount, int tupleLength, int minFrameSize) {
+ assert fieldCount >= 0 && tupleLength >= 0 && minFrameSize > 0;
+ return (1 + (calcSpaceInFrame(fieldCount, tupleLength) + FrameConstants.META_DATA_LEN - 1) / minFrameSize)
+ * minFrameSize;
+ }
+
+ public static void clearRemainingFrame(ByteBuffer buffer, int position) {
+ buffer.array()[position] = 0;
+ }
+
+ public static boolean hasBeenCleared(ByteBuffer buffer, int position) {
+ return deserializeNumOfMinFrame(buffer, position) == 0;
+ }
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java
new file mode 100644
index 0000000..ccbbb0d
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrame.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrame {
+
+ ByteBuffer getBuffer();
+
+ /**
+ * Make sure the frameSize is bigger or equal to the given size
+ *
+ * @param frameSize
+ * @throws HyracksDataException
+ */
+ void ensureFrameSize(int frameSize) throws HyracksDataException;
+
+ /**
+ *
+ * Expand of shrink the inner buffer to make the size exactly equal to {@code frameSize}
+ * @param frameSize
+ */
+ void resize(int frameSize) throws HyracksDataException;
+
+ /**
+ * Return the size of frame in bytes
+ *
+ * @return
+ */
+ int getFrameSize();
+
+ /**
+ * Return the minimum frame size which should read from the configuration file given by user
+ *
+ * @return
+ */
+ int getMinSize();
+
+ /**
+ * Reset the status of buffer, prepare to the next round of read/write
+ */
+ void reset() throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java
new file mode 100644
index 0000000..89b2bba
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameAppender.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameAppender {
+ /**
+ * Reset to attach to a new frame.
+ *
+ * @param frame the new frame
+ * @param clear indicate whether we need to clear this new frame
+ * @throws HyracksDataException
+ */
+ void reset(IFrame frame, boolean clear) throws HyracksDataException;
+
+ /**
+ * Get how many tuples in current frame.
+ *
+ * @return
+ */
+ int getTupleCount();
+
+ /**
+ * Get the ByteBuffer which contains the frame data.
+ *
+ * @return
+ */
+ ByteBuffer getBuffer();
+
+ /**
+ * Flush the frame content to the given writer.
+ * Clear the inner buffer after flush if {@code clear} is <code>true</code>.
+ *
+ * @param outWriter the output writer
+ * @param clear indicate whether to clear the inside frame after flushed or not.
+ * @throws HyracksDataException
+ */
+ void flush(IFrameWriter outWriter, boolean clear) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java
new file mode 100644
index 0000000..f66248f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameFieldAppender.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The IFrameFieldAppender is used to append the data into frame field by field.
+ */
+public interface IFrameFieldAppender extends IFrameAppender {
+
+ /**
+ * Append the field stored in {@code bytes} into the current frame.
+ *
+ * @param bytes the byte array that stores the field data
+ * @param offset the offset of the field data
+ * @param length the length of the field data
+ * @return true if the current frame has enough space to hold the field data, otherwise return false.
+ * @throws HyracksDataException
+ */
+ boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException;
+
+ /**
+ * Append the field of {@code fid} from the tuple {@code tid} whose information is stored in the {@code accessor}
+ * into the current frame.
+ *
+ * @param accessor tupleAccessor
+ * @param tid tuple id in tupleAccessor
+ * @param fid field id of the tuple {@code tid}
+ * @return true if the current frame has enough space to hold the field data, otherwise return false.
+ * @throws HyracksDataException
+ */
+ boolean appendField(IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
index c72782a..cd3c5ab 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameReader.java
@@ -14,14 +14,12 @@
*/
package edu.uci.ics.hyracks.api.comm;
-import java.nio.ByteBuffer;
-
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IFrameReader {
- public void open() throws HyracksDataException;
+ void open() throws HyracksDataException;
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException;
+ boolean nextFrame(IFrame frame) throws HyracksDataException;
- public void close() throws HyracksDataException;
+ void close() throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
index ee34add..130704f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAccessor.java
@@ -17,23 +17,28 @@
import java.nio.ByteBuffer;
public interface IFrameTupleAccessor {
- public int getFieldCount();
+ int getFieldCount();
- public int getFieldSlotsLength();
+ int getFieldSlotsLength();
- public int getFieldEndOffset(int tupleIndex, int fIdx);
+ int getFieldEndOffset(int tupleIndex, int fIdx);
- public int getFieldStartOffset(int tupleIndex, int fIdx);
+ int getFieldStartOffset(int tupleIndex, int fIdx);
- public int getFieldLength(int tupleIndex, int fIdx);
+ int getFieldLength(int tupleIndex, int fIdx);
- public int getTupleEndOffset(int tupleIndex);
+ int getTupleLength(int tupleIndex);
- public int getTupleStartOffset(int tupleIndex);
+ int getTupleEndOffset(int tupleIndex);
- public int getTupleCount();
+ int getTupleStartOffset(int tupleIndex);
- public ByteBuffer getBuffer();
+ int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx);
- public void reset(ByteBuffer buffer);
+ int getTupleCount();
+
+ ByteBuffer getBuffer();
+
+ void reset(ByteBuffer buffer);
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java
new file mode 100644
index 0000000..4da2afc
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameTupleAppender.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameTupleAppender extends IFrameAppender {
+
+ boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException;
+
+ boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException;
+
+ boolean append(byte[] bytes, int offset, int length) throws HyracksDataException;
+
+ boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException;
+
+ boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) throws HyracksDataException;
+
+ boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1,
+ int tIndex1) throws HyracksDataException;
+
+ boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
+ int offset1, int dataLen1) throws HyracksDataException;
+
+ boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException;
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
index 8e35dda..538759e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/IFrameWriter.java
@@ -45,7 +45,7 @@
* Note: If the call to {@link IFrameWriter#open()} failed, the {@link IFrameWriter#close()} is not called by the producer. So an exceptional
* return from the {@link IFrameWriter#open()} call must clean up all partially
* allocated resources.
- *
+ *
* @author vinayakb
*/
public interface IFrameWriter {
@@ -56,9 +56,8 @@
/**
* Provide data to the stream of this {@link IFrameWriter}.
- *
- * @param buffer
- * - Buffer containing data.
+ *
+ * @param buffer - Buffer containing data.
* @throws HyracksDataException
*/
public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
@@ -66,14 +65,14 @@
/**
* Indicate that a failure was encountered and the current stream is to be
* aborted.
- *
+ *
* @throws HyracksDataException
*/
public void fail() throws HyracksDataException;
/**
* Close this {@link IFrameWriter} and give up all resources.
- *
+ *
* @throws HyracksDataException
*/
public void close() throws HyracksDataException;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java
new file mode 100644
index 0000000..902ae75
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/NoShrinkVSizeFrame.java
@@ -0,0 +1,31 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class NoShrinkVSizeFrame extends VSizeFrame {
+ public NoShrinkVSizeFrame(IHyracksFrameMgrContext ctx) throws HyracksDataException {
+ super(ctx);
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ buffer.clear();
+ }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java
new file mode 100644
index 0000000..a5a7f19
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/comm/VSizeFrame.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.comm;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Variable size frame. The buffer inside can be resized.
+ */
+public class VSizeFrame implements IFrame {
+
+ protected final int minFrameSize;
+ protected IHyracksFrameMgrContext ctx;
+ protected ByteBuffer buffer;
+
+ public VSizeFrame(IHyracksFrameMgrContext ctx) throws HyracksDataException {
+ this(ctx, ctx.getInitialFrameSize());
+ }
+
+ public VSizeFrame(IHyracksFrameMgrContext ctx, int frameSize) throws HyracksDataException {
+ this.minFrameSize = ctx.getInitialFrameSize();
+ this.ctx = ctx;
+ buffer = ctx.allocateFrame(frameSize);
+ }
+
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public void ensureFrameSize(int newSize) throws HyracksDataException {
+ if (newSize > getFrameSize()) {
+ buffer = ctx.reallocateFrame(buffer, newSize, true);
+ }
+ }
+
+ @Override
+ public void resize(int frameSize) throws HyracksDataException {
+ if (getFrameSize() != frameSize) {
+ buffer = ctx.reallocateFrame(buffer, frameSize, false);
+ }
+ }
+
+ @Override
+ public int getFrameSize() {
+ return buffer.capacity();
+ }
+
+ @Override
+ public int getMinSize() {
+ return minFrameSize;
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ resize(minFrameSize);
+ buffer.clear();
+ }
+
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
index eddc4df..d60ff6e 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksCommonContext.java
@@ -14,17 +14,9 @@
*/
package edu.uci.ics.hyracks.api.context;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.IIOManager;
-public interface IHyracksCommonContext {
- public int getFrameSize();
+public interface IHyracksCommonContext extends IHyracksFrameMgrContext{
public IIOManager getIOManager();
-
- public ByteBuffer allocateFrame() throws HyracksDataException;
-
- public void deallocateFrames(int frameCount);
}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java
new file mode 100644
index 0000000..d7a18bc
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksFrameMgrContext.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.api.context;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IHyracksFrameMgrContext {
+ int getInitialFrameSize();
+
+ //TODO tobedeleted
+ ByteBuffer allocateFrame() throws HyracksDataException;
+
+ ByteBuffer allocateFrame(int bytes) throws HyracksDataException;
+
+ ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+ throws HyracksDataException;
+
+ /**
+ * The caller should call this method to return the pre-allocated frames.
+ *
+ * @param bytes
+ */
+ void deallocateFrames(int bytes);
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
index d2c2ee3..dec0bfd 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataset/IHyracksDatasetReader.java
@@ -14,13 +14,12 @@
*/
package edu.uci.ics.hyracks.api.dataset;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IHyracksDatasetReader {
public Status getResultStatus();
- public int read(ByteBuffer buffer) throws HyracksDataException;
+ public int read(IFrame frame) throws HyracksDataException;
}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
index 97ce664..9975fe3 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -14,36 +14,21 @@
*/
package edu.uci.ics.hyracks.client.dataset;
-import java.nio.ByteBuffer;
-
import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
-public class DatasetClientContext implements IHyracksCommonContext {
+public class DatasetClientContext extends FrameManager implements IHyracksCommonContext {
private final int frameSize;
public DatasetClientContext(int frameSize) {
+ super(frameSize);
this.frameSize = frameSize;
}
@Override
- public int getFrameSize() {
- return frameSize;
- }
-
- @Override
public IIOManager getIOManager() {
return null;
}
- @Override
- public ByteBuffer allocateFrame() {
- return ByteBuffer.allocate(frameSize);
- }
-
- @Override
- public void deallocateFrames(int frameCount) {
- // TODO Auto-generated method stub
-
- }
}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
index 062b5bf..34c4d6e 100644
--- a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -24,6 +24,8 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
import edu.uci.ics.hyracks.api.dataset.DatasetJobRecord.Status;
@@ -64,7 +66,8 @@
private static int NUM_READ_BUFFERS = 1;
public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
- ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+ ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId,
+ ResultSetId resultSetId)
throws Exception {
this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
this.netManager = netManager;
@@ -119,7 +122,8 @@
}
@Override
- public int read(ByteBuffer buffer) throws HyracksDataException {
+ public int read(IFrame frame) throws HyracksDataException {
+ frame.reset();
ByteBuffer readBuffer;
int readSize = 0;
@@ -129,7 +133,7 @@
}
}
- while (readSize <= 0
+ while (readSize < frame.getFrameSize()
&& !((lastReadPartition == knownRecords.length - 1) && isPartitionReadComplete(lastMonitor))) {
waitForNextFrame(lastMonitor);
if (isPartitionReadComplete(lastMonitor)) {
@@ -142,14 +146,23 @@
readBuffer = resultChannel.getNextBuffer();
lastMonitor.notifyFrameRead();
if (readBuffer != null) {
- buffer.put(readBuffer);
- buffer.flip();
- readSize = buffer.limit();
- resultChannel.recycleBuffer(readBuffer);
+ if (readSize <=0) {
+ int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
+ frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+ frame.getBuffer().clear();
+ frame.getBuffer().put(readBuffer);
+ resultChannel.recycleBuffer(readBuffer);
+ readSize = frame.getBuffer().position();
+ } else {
+ frame.getBuffer().put(readBuffer);
+ resultChannel.recycleBuffer(readBuffer);
+ readSize = frame.getBuffer().position();
+ }
}
}
}
+ frame.getBuffer().flip();
return readSize;
}
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index c5cb7d0..87fa23c 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -95,7 +95,7 @@
}
ccb.getReadInterface().setFullBufferAcceptor(new ReadFullBufferAcceptor());
ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
- ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers, ctx.getFrameSize());
+ ccb.getReadInterface().setBufferFactory(new ReadBufferFactory(nBuffers, ctx), nBuffers, ctx.getInitialFrameSize());
ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
writeBuffer.putLong(partitionId.getJobId().getId());
writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index b573b73..5ac0a47 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -54,32 +54,36 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
ByteBuffer destBuffer = null;
- synchronized (this) {
- while (true) {
- if (aborted) {
- throw new HyracksDataException("Connection has been aborted");
- }
- destBuffer = emptyStack.poll();
- if (destBuffer == null && allocateCounter < nBuffers) {
- destBuffer = ByteBuffer.allocateDirect(frameSize);
- allocateCounter++;
- }
- if (destBuffer != null) {
- break;
- }
- try {
- wait();
- } catch (InterruptedException e) {
- throw new HyracksDataException(e);
+ int startPos = 0;
+ do {
+ synchronized (this) {
+ while (true) {
+ if (aborted) {
+ throw new HyracksDataException("Connection has been aborted");
+ }
+ destBuffer = emptyStack.poll();
+ if (destBuffer == null && allocateCounter < nBuffers) {
+ destBuffer = ByteBuffer.allocateDirect(frameSize);
+ allocateCounter++;
+ }
+ 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();
- ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
+ buffer.position(startPos);
+ startPos = Math.min(startPos + destBuffer.capacity(), buffer.capacity());
+ buffer.limit(startPos);
+ destBuffer.clear();
+ destBuffer.put(buffer);
+ destBuffer.flip();
+ ccb.getWriteInterface().getFullBufferAcceptor().accept(destBuffer);
+ } while (startPos < buffer.capacity());
}
@Override
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
index c59398c..641b7bf 100644
--- a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/ReadBufferFactory.java
@@ -30,7 +30,7 @@
public ReadBufferFactory(int limit, IHyracksCommonContext ctx) {
this.limit = limit;
- this.frameSize = ctx.getFrameSize();
+ this.frameSize = ctx.getInitialFrameSize();
}
@Override
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index 89c5b75..c6e49e1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -56,6 +56,7 @@
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;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
public class Joblet implements IHyracksJobletContext, ICounterContext {
private static final Logger LOGGER = Logger.getLogger(Joblet.class.getName());
@@ -88,7 +89,7 @@
private final IJobletEventListener jobletEventListener;
- private final int frameSize;
+ private final FrameManager frameManager;
private final AtomicLong memoryAllocation;
@@ -102,7 +103,7 @@
this.appCtx = appCtx;
this.deploymentId = deploymentId;
this.jobId = jobId;
- this.frameSize = acg.getFrameSize();
+ this.frameManager = new FrameManager(acg.getFrameSize());
memoryAllocation = new AtomicLong();
this.acg = acg;
partitionRequestMap = new HashMap<PartitionId, IPartitionCollector>();
@@ -222,23 +223,33 @@
}
ByteBuffer allocateFrame() throws HyracksDataException {
- if (appCtx.getMemoryManager().allocate(frameSize)) {
- memoryAllocation.addAndGet(frameSize);
- return ByteBuffer.allocate(frameSize);
- }
+ return frameManager.allocateFrame();
+ }
+
+ ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ if (appCtx.getMemoryManager().allocate(bytes)) {
+ memoryAllocation.addAndGet(bytes);
+ return frameManager.allocateFrame(bytes);
+ }
throw new HyracksDataException("Unable to allocate frame: Not enough memory");
}
- public void deallocateFrames(int nFrames) {
- memoryAllocation.addAndGet(nFrames * frameSize);
- appCtx.getMemoryManager().deallocate(nFrames * frameSize);
+ ByteBuffer reallocateFrame(ByteBuffer usedBuffer, int newFrameSizeInBytes, boolean copyOldData)
+ throws HyracksDataException {
+ return frameManager.reallocateFrame(usedBuffer, newFrameSizeInBytes, copyOldData);
}
- final int getFrameSize() {
- return frameSize;
+ void deallocateFrames(int bytes) {
+ memoryAllocation.addAndGet(bytes);
+ appCtx.getMemoryManager().deallocate(bytes);
+ frameManager.deallocateFrames(bytes);
}
- IIOManager getIOManager() {
+ public final int getFrameSize() {
+ return frameManager.getInitialFrameSize();
+ }
+
+ public IIOManager getIOManager() {
return appCtx.getRootContext().getIOManager();
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index 3014024..387ab04 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -25,6 +25,7 @@
import java.util.concurrent.Executor;
import java.util.concurrent.Semaphore;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
@@ -120,12 +121,23 @@
}
@Override
- public void deallocateFrames(int frameCount) {
- joblet.deallocateFrames(frameCount);
+ public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ return joblet.allocateFrame(bytes);
}
@Override
- public int getFrameSize() {
+ public ByteBuffer reallocateFrame(ByteBuffer usedBuffer, int newSizeInBytes, boolean copyOldData)
+ throws HyracksDataException {
+ return joblet.reallocateFrame(usedBuffer, newSizeInBytes, copyOldData);
+ }
+
+ @Override
+ public void deallocateFrames(int bytes) {
+ joblet.deallocateFrames(bytes);
+ }
+
+ @Override
+ public int getInitialFrameSize() {
return joblet.getFrameSize();
}
@@ -317,12 +329,12 @@
try {
writer.open();
try {
- ByteBuffer buffer = allocateFrame();
- while (reader.nextFrame(buffer)) {
+ VSizeFrame frame = new VSizeFrame(this);
+ while( reader.nextFrame(frame)){
if (aborted) {
return;
}
- buffer.flip();
+ ByteBuffer buffer = frame.getBuffer();
writer.nextFrame(buffer);
buffer.compact();
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
index d3a6fb5..673a319 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -67,7 +67,7 @@
resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
resultState = new ResultState(resultSetPartitionId, asyncMode, ctx.getIOManager(), fileFactory,
- ctx.getFrameSize());
+ ctx.getInitialFrameSize());
}
public ResultState getResultState() {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
index c24034d..9ea6ddb 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -190,6 +190,9 @@
initReadFileHandle();
}
readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+ if (readSize < 0){
+ throw new HyracksDataException("Premature end of file");
+ }
}
if (readSize < buffer.capacity()) {
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
index 239b6aa..b3f3e2a 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/io/IOManager.java
@@ -103,6 +103,15 @@
}
}
+ /**
+ * Please do check the return value of this read!
+ *
+ * @param fHandle
+ * @param offset
+ * @param data
+ * @return The number of bytes read, possibly zero, or -1 if the given offset is greater than or equal to the file's current size
+ * @throws HyracksDataException
+ */
@Override
public int syncRead(IFileHandle fHandle, long offset, ByteBuffer data) throws HyracksDataException {
try {
@@ -111,7 +120,7 @@
while (remaining > 0) {
int len = ((FileHandle) fHandle).getFileChannel().read(data, offset);
if (len < 0) {
- return -1;
+ return n == 0 ? -1 : n;
}
remaining -= len;
offset += len;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 816e345..b52a8a1 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -114,6 +114,9 @@
ByteBuffer destFrame = emptyQueue.poll();
buffer.position(0);
buffer.limit(buffer.capacity());
+ if (destFrame.capacity() < buffer.capacity()){
+ throw new HyracksDataException("should never happen");
+ }
destFrame.clear();
destFrame.put(buffer);
fullQueue.add(destFrame);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index b209cc1..4f5d60f 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -64,7 +64,7 @@
*/
NetworkOutputChannel writer = partitionRequests.remove(pid);
if (writer != null) {
- writer.setFrameSize(partition.getTaskContext().getFrameSize());
+ writer.setFrameSize(partition.getTaskContext().getInitialFrameSize());
partition.writeTo(writer);
if (!partition.isReusable()) {
return;
@@ -116,7 +116,7 @@
List<IPartition> pList = availablePartitionMap.get(partitionId);
if (pList != null && !pList.isEmpty()) {
IPartition partition = pList.get(0);
- writer.setFrameSize(partition.getTaskContext().getFrameSize());
+ writer.setFrameSize(partition.getTaskContext().getInitialFrameSize());
partition.writeTo(writer);
if (!partition.isReusable()) {
availablePartitionMap.remove(partitionId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
index e7102e1..c9fc237 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/profiling/ConnectorReceiverProfilingFrameReader.java
@@ -14,8 +14,7 @@
*/
package edu.uci.ics.hyracks.control.nc.profiling;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
@@ -44,8 +43,8 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- boolean status = reader.nextFrame(buffer);
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
+ boolean status = reader.nextFrame(frame);
if (status) {
frameCounter.update(1);
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java
new file mode 100644
index 0000000..fd71d8b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/resources/memory/FrameManager.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.control.nc.resources.memory;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class FrameManager implements IHyracksFrameMgrContext {
+
+ private final int minFrameSize;
+
+ public FrameManager(int minFrameSize) {
+ this.minFrameSize = minFrameSize;
+ }
+
+ @Override
+ public int getInitialFrameSize() {
+ return minFrameSize;
+ }
+
+ @Override
+ public ByteBuffer allocateFrame() throws HyracksDataException {
+ return allocateFrame(minFrameSize);
+ }
+
+ @Override
+ public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ if (bytes % minFrameSize != 0) {
+ throw new HyracksDataException("The size should be an integral multiple of the default frame size");
+ }
+ ByteBuffer buffer = ByteBuffer.allocate(bytes);
+ if (bytes / minFrameSize > FrameConstants.MAX_NUM_MINFRAME) {
+ throw new HyracksDataException(
+ "Unable to allocate frame larger than:" + FrameConstants.MAX_NUM_MINFRAME + " bytes");
+ }
+ FrameHelper.serializeFrameSize(buffer, (byte) (bytes / minFrameSize));
+ return (ByteBuffer) buffer.clear();
+ }
+
+ @Override
+ public ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+ throws HyracksDataException {
+ if (!copyOldData) {
+ deallocateFrames(tobeDeallocate.capacity());
+ return allocateFrame(newSizeInBytes);
+ } else {
+ ByteBuffer buffer = allocateFrame(newSizeInBytes);
+ int limit = Math.min(newSizeInBytes, tobeDeallocate.capacity());
+ int pos = Math.min(limit, tobeDeallocate.position());
+ tobeDeallocate.position(0);
+ tobeDeallocate.limit(limit);
+ buffer.put(tobeDeallocate);
+ buffer.position(pos);
+
+ if (newSizeInBytes / minFrameSize > FrameConstants.MAX_NUM_MINFRAME) {
+ throw new HyracksDataException("Unable to allocate frame of size bigger than MinFrameSize * "
+ + FrameConstants.MAX_NUM_MINFRAME);
+ }
+ FrameHelper.serializeFrameSize(buffer, (byte) (newSizeInBytes / minFrameSize));
+ return buffer;
+ }
+ }
+
+ @Override
+ public void deallocateFrames(int bytes) {
+ //TODO make a global memory manager to allocate and deallocate the frames.
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/pom.xml b/hyracks/hyracks-dataflow-common/pom.xml
index 9f965a3..3f2936d 100644
--- a/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks/hyracks-dataflow-common/pom.xml
@@ -57,6 +57,12 @@
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-data-std</artifactId>
<version>0.2.16-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.16-SNAPSHOT</version>
+ <scope>test</scope>
</dependency>
</dependencies>
</project>
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
new file mode 100644
index 0000000..1f501aa
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/AbstractFrameAppender.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class AbstractFrameAppender implements IFrameAppender {
+ protected IFrame frame;
+ protected byte[] array; // cached the getBuffer().array to speed up byte array access a little
+
+ protected int tupleCount;
+ protected int tupleDataEndOffset;
+
+ @Override
+ public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+ this.frame = frame;
+ if (clear) {
+ this.frame.reset();
+ }
+ reset(getBuffer(), clear);
+ }
+
+ protected boolean hasEnoughSpace(int fieldCount, int tupleLength) {
+ return tupleDataEndOffset + FrameHelper.calcSpaceInFrame(fieldCount, tupleLength)
+ + tupleCount * FrameConstants.SIZE_LEN
+ <= FrameHelper.getTupleCountOffset(frame.getFrameSize());
+ }
+
+ private void reset(ByteBuffer buffer, boolean clear) {
+ array = buffer.array();
+ if (clear) {
+ IntSerDeUtils.putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), 0);
+ tupleCount = 0;
+ tupleDataEndOffset = FrameConstants.TUPLE_START_OFFSET;
+ } else {
+ tupleCount = IntSerDeUtils.getInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()));
+ tupleDataEndOffset = tupleCount == 0 ?
+ FrameConstants.TUPLE_START_OFFSET :
+ IntSerDeUtils.getInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize())
+ - tupleCount * FrameConstants.SIZE_LEN);
+ }
+ }
+
+ @Override
+ public int getTupleCount() {
+ return tupleCount;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return frame.getBuffer();
+ }
+
+ @Override
+ public void flush(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
+ getBuffer().clear();
+ if (getTupleCount() > 0) {
+ outWriter.nextFrame(getBuffer());
+ }
+ if (clearFrame) {
+ frame.reset();
+ reset(getBuffer(), true);
+ }
+ }
+
+ protected boolean canHoldNewTuple(int fieldCount, int dataLength) throws HyracksDataException {
+ if (hasEnoughSpace(fieldCount, dataLength)) {
+ return true;
+ }
+ if (tupleCount == 0) {
+ frame.ensureFrameSize(FrameHelper.calcAlignedFrameSizeToStore(fieldCount, dataLength, frame.getMinSize()));
+ reset(frame.getBuffer(), true);
+ return true;
+ }
+ return false;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
index 9223740..53c5eb3 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ArrayTupleBuilder.java
@@ -17,6 +17,7 @@
import java.io.DataOutput;
import java.io.IOException;
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
index 556ecbb..b1052e7 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializer.java
@@ -20,6 +20,7 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -41,11 +42,11 @@
private ByteBuffer buffer;
- public FrameDeserializer(int frameSize, RecordDescriptor recordDescriptor) {
+ public FrameDeserializer(RecordDescriptor recordDescriptor) {
this.bbis = new ByteBufferInputStream();
this.di = new DataInputStream(bbis);
this.recordDescriptor = recordDescriptor;
- frameTupleAccessor = new FrameTupleAccessor(frameSize, recordDescriptor);
+ frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
}
public void reset(ByteBuffer buffer) {
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
index b94a219..2dcf84c 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataReader.java
@@ -14,16 +14,16 @@
*/
package edu.uci.ics.hyracks.dataflow.common.comm.io;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
public class FrameDeserializingDataReader implements IOpenableDataReader<Object[]> {
- private final ByteBuffer buffer;
+ private final IFrame frame;
private boolean eos;
@@ -35,16 +35,15 @@
public FrameDeserializingDataReader(IHyracksTaskContext ctx, IFrameReader frameReader,
RecordDescriptor recordDescriptor) throws HyracksDataException {
- buffer = ctx.allocateFrame();
+ this.frame = new VSizeFrame(ctx);
this.frameReader = frameReader;
- this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
+ this.frameDeserializer = new FrameDeserializer(recordDescriptor);
}
@Override
public void open() throws HyracksDataException {
frameReader.open();
- buffer.clear();
- buffer.flip();
+ frame.reset();
eos = false;
first = true;
}
@@ -64,11 +63,11 @@
if (!first && !frameDeserializer.done()) {
return frameDeserializer.deserializeRecord();
}
- buffer.clear();
- if (!frameReader.nextFrame(buffer)) {
+ frame.reset();
+ if (!frameReader.nextFrame(frame)) {
eos = true;
} else {
- frameDeserializer.reset(buffer);
+ frameDeserializer.reset(frame.getBuffer());
}
first = false;
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
index b3a72d7..8855de5 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameDeserializingDataWriter.java
@@ -29,7 +29,7 @@
public FrameDeserializingDataWriter(IHyracksTaskContext ctx, IOpenableDataWriter<Object[]> writer,
RecordDescriptor recordDescriptor) {
this.writer = writer;
- this.frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), recordDescriptor);
+ this.frameDeserializer = new FrameDeserializer(recordDescriptor);
}
@Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java
new file mode 100644
index 0000000..2db3e44
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldAppender.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class FrameFixedFieldAppender extends AbstractFrameAppender implements IFrameFieldAppender {
+ private final int fieldCount;
+ private int lastFieldEndOffset;
+ private int currentField;
+ private int leftOverSize;
+ private byte[] cachedLeftOverFields;
+
+ public FrameFixedFieldAppender(int numberFields) {
+ this.fieldCount = numberFields;
+ this.lastFieldEndOffset = 0;
+ this.currentField = 0;
+ this.leftOverSize = 0;
+ }
+
+ @Override
+ public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+ super.reset(frame, clear);
+ lastFieldEndOffset = 0;
+ currentField = 0;
+ leftOverSize = 0;
+ }
+
+ @Override
+ public void flush(IFrameWriter outWriter, boolean clearFrame) throws HyracksDataException {
+ super.flush(outWriter, clearFrame);
+ if (clearFrame) {
+ if (leftOverSize > 0) {
+ if (!canHoldNewTuple(0, leftOverSize)) {
+ throw new HyracksDataException(
+ "The given frame can not be extended to insert the leftover data from the last record");
+ }
+ System.arraycopy(cachedLeftOverFields, 0, array, tupleDataEndOffset, leftOverSize);
+ leftOverSize = 0;
+ }
+ }
+ }
+
+ public boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException {
+ if (canHoldNewTuple(fieldCount, lastFieldEndOffset + length)) {
+ int currentFieldDataStart = tupleDataEndOffset + fieldCount * 4 + lastFieldEndOffset;
+ System.arraycopy(bytes, offset, array, currentFieldDataStart, length);
+ lastFieldEndOffset = lastFieldEndOffset + length;
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + currentField * 4, lastFieldEndOffset);
+ if (++currentField == fieldCount) {
+ tupleDataEndOffset += fieldCount * 4 + lastFieldEndOffset;
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+ tupleDataEndOffset);
+ ++tupleCount;
+ IntSerDeUtils.putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
+
+ //reset for the next tuple
+ currentField = 0;
+ lastFieldEndOffset = 0;
+ }
+ return true;
+ } else {
+ if (currentField > 0) {
+ copyLeftOverData();
+ }
+ return false;
+ }
+ }
+
+ private void copyLeftOverData() {
+ leftOverSize = lastFieldEndOffset + fieldCount * 4;
+ if (cachedLeftOverFields == null || cachedLeftOverFields.length < leftOverSize) {
+ cachedLeftOverFields = new byte[leftOverSize];
+ }
+ System.arraycopy(array, tupleDataEndOffset, cachedLeftOverFields, 0, leftOverSize);
+ }
+
+ public boolean appendField(IFrameTupleAccessor fta, int tIndex, int fIndex) throws HyracksDataException {
+ int startOffset = fta.getTupleStartOffset(tIndex);
+ int fStartOffset = fta.getFieldStartOffset(tIndex, fIndex);
+ int fLen = fta.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
+ return appendField(fta.getBuffer().array(), startOffset + fta.getFieldSlotsLength() + fStartOffset, fLen);
+ }
+
+ public boolean hasLeftOverFields() {
+ return currentField != 0;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java
new file mode 100644
index 0000000..74289c6
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameFixedFieldTupleAppender.java
@@ -0,0 +1,130 @@
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * This appender can appendTuple and appendField but at the expense of additional checks.
+ * Please use this Field/Tuple mixed appender only if you don't know the sequence of append functions.
+ * Try using {@link FrameFixedFieldAppender} if you only want to appendFields.
+ * and using {@link FrameTupleAppender} if you only want to appendTuples.
+ */
+public class FrameFixedFieldTupleAppender implements IFrameTupleAppender, IFrameFieldAppender {
+
+ private FrameFixedFieldAppender fieldAppender;
+ private FrameTupleAppender tupleAppender;
+ private IFrame sharedFrame;
+ private IFrameAppender lastAppender;
+
+ public FrameFixedFieldTupleAppender(int numFields) {
+ tupleAppender = new FrameTupleAppender();
+ fieldAppender = new FrameFixedFieldAppender(numFields);
+ lastAppender = tupleAppender;
+ }
+
+ private void resetAppenderIfNecessary(IFrameAppender appender) throws HyracksDataException {
+ if (lastAppender != appender) {
+ if (lastAppender == fieldAppender) {
+ if (fieldAppender.hasLeftOverFields()) {
+ throw new HyracksDataException("The previous appended fields haven't been flushed yet.");
+ }
+ }
+ appender.reset(sharedFrame, false);
+ lastAppender = appender;
+ }
+ }
+
+ @Override
+ public boolean appendField(byte[] bytes, int offset, int length) throws HyracksDataException {
+ resetAppenderIfNecessary(fieldAppender);
+ return fieldAppender.appendField(bytes, offset, length);
+ }
+
+ @Override
+ public boolean appendField(IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException {
+ resetAppenderIfNecessary(fieldAppender);
+ return fieldAppender.appendField(accessor, tid, fid);
+ }
+
+ @Override
+ public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.append(tupleAccessor, tIndex);
+ }
+
+ @Override
+ public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.append(fieldSlots, bytes, offset, length);
+ }
+
+ @Override
+ public boolean append(byte[] bytes, int offset, int length) throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.append(bytes, offset, length);
+ }
+
+ @Override
+ public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
+ throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length);
+ }
+
+ @Override
+ public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+ throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.append(tupleAccessor, tStartOffset, tEndOffset);
+ }
+
+ @Override
+ public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+ throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1);
+ }
+
+ @Override
+ public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
+ int offset1, int dataLen1) throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1);
+ }
+
+ @Override
+ public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields)
+ throws HyracksDataException {
+ resetAppenderIfNecessary(tupleAppender);
+ return tupleAppender.appendProjection(accessor, tIndex, fields);
+ }
+
+ @Override
+ public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+ sharedFrame = frame;
+ tupleAppender.reset(sharedFrame, clear);
+ fieldAppender.reset(sharedFrame, clear);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return lastAppender.getTupleCount();
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return lastAppender.getBuffer();
+ }
+
+ @Override
+ public void flush(IFrameWriter outWriter, boolean clear) throws HyracksDataException {
+ lastAppender.flush(outWriter, clear);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
index 1810674..90f3d25 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -14,28 +14,26 @@
*/
package edu.uci.ics.hyracks.dataflow.common.comm.io;
-import java.nio.ByteBuffer;
import java.util.logging.Level;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
private final FrameTupleAppender frameTupleAppender;
- public FrameOutputStream(int frameSize) {
- super(frameSize);
- this.frameTupleAppender = new FrameTupleAppender(frameSize);
+ public FrameOutputStream(int initialStreamCapaciy) {
+ super(initialStreamCapaciy);
+ this.frameTupleAppender = new FrameTupleAppender();
}
- public void reset(ByteBuffer buffer, boolean clear) {
- if (clear) {
- buffer.clear();
- }
- frameTupleAppender.reset(buffer, clear);
+ public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+ frameTupleAppender.reset(frame, clear);
}
public int getTupleCount() {
@@ -46,7 +44,7 @@
return tupleCount;
}
- public boolean appendTuple() {
+ public boolean appendTuple() throws HyracksDataException {
if (LOGGER.isLoggable(Level.FINEST)) {
LOGGER.finest("appendTuple(): tuple size: " + count);
}
@@ -54,4 +52,8 @@
count = 0;
return appended;
}
+
+ public void flush(IFrameWriter writer) throws HyracksDataException {
+ frameTupleAppender.flush(writer, true);
+ }
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index 6edd647..600e0b4 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -17,6 +17,7 @@
import java.io.DataInputStream;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
import edu.uci.ics.hyracks.api.comm.FrameHelper;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -26,29 +27,33 @@
/**
* FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
- * formatted with tuple data concatenated starting at offset 0, one tuple after
- * another. Offset FS - 4 holds an int indicating the number of tuples (N) in
+ * formatted with tuple data concatenated starting at offset 1, one tuple after
+ * another. The first byte is used to notify how big the frame is, so the maximum frame size is 255 * initialFrameSetting.
+ * Offset FS - 4 holds an int indicating the number of tuples (N) in
* the frame. FS - ((i + 1) * 4) for i from 0 to N - 1 holds an int indicating
* the offset of the (i + 1)^th tuple. Every tuple is organized as a sequence of
* ints indicating the end of each field in the tuple relative to the end of the
* field slots.
- *
- * @author vinayakb
*/
-public final class FrameTupleAccessor implements IFrameTupleAccessor {
- private final int frameSize;
+public class FrameTupleAccessor implements IFrameTupleAccessor {
+ private int tupleCountOffset;
private final RecordDescriptor recordDescriptor;
-
private ByteBuffer buffer;
+ private int start;
- public FrameTupleAccessor(int frameSize, RecordDescriptor recordDescriptor) {
- this.frameSize = frameSize;
+ public FrameTupleAccessor(RecordDescriptor recordDescriptor) {
this.recordDescriptor = recordDescriptor;
}
@Override
public void reset(ByteBuffer buffer) {
+ reset(buffer, 0, buffer.limit());
+ }
+
+ public void reset(ByteBuffer buffer, int start, int length) {
this.buffer = buffer;
+ this.start = start;
+ this.tupleCountOffset = start + FrameHelper.getTupleCountOffset(length);
}
@Override
@@ -58,28 +63,39 @@
@Override
public int getTupleCount() {
- return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize));
+ return IntSerDeUtils.getInt(buffer.array(), tupleCountOffset);
}
@Override
public int getTupleStartOffset(int tupleIndex) {
- return tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize)
- - 4 * tupleIndex);
+ int offset = tupleIndex == 0 ?
+ FrameConstants.TUPLE_START_OFFSET :
+ IntSerDeUtils.getInt(buffer.array(), tupleCountOffset - 4 * tupleIndex);
+ return start + offset;
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
}
@Override
public int getTupleEndOffset(int tupleIndex) {
- return IntSerDeUtils.getInt(buffer.array(), FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+ return start + IntSerDeUtils
+ .getInt(buffer.array(), tupleCountOffset - FrameConstants.SIZE_LEN * (tupleIndex + 1));
}
@Override
public int getFieldStartOffset(int tupleIndex, int fIdx) {
- return fIdx == 0 ? 0 : IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ return fIdx == 0 ?
+ 0 :
+ IntSerDeUtils
+ .getInt(buffer.array(), getTupleStartOffset(tupleIndex) + (fIdx - 1) * FrameConstants.SIZE_LEN);
}
@Override
public int getFieldEndOffset(int tupleIndex, int fIdx) {
- return IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + fIdx * 4);
+ return IntSerDeUtils.getInt(buffer.array(), getTupleStartOffset(tupleIndex) + fIdx * FrameConstants.SIZE_LEN);
}
@Override
@@ -88,34 +104,57 @@
}
@Override
- public int getFieldSlotsLength() {
- return getFieldCount() * 4;
+ public int getTupleLength(int tupleIndex) {
+ return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
}
- public void prettyPrint() {
+ @Override
+ public int getFieldSlotsLength() {
+ return getFieldCount() * FrameConstants.SIZE_LEN;
+ }
+
+ public void prettyPrint(String prefix) {
ByteBufferInputStream bbis = new ByteBufferInputStream();
DataInputStream dis = new DataInputStream(bbis);
int tc = getTupleCount();
- System.err.println("TC: " + tc);
+ StringBuilder sb = new StringBuilder();
+ sb.append(prefix).append("TC: " + tc).append("\n");
for (int i = 0; i < tc; ++i) {
- System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
- for (int j = 0; j < getFieldCount(); ++j) {
- System.err.print(j + ":(" + getFieldStartOffset(i, j) + ", " + getFieldEndOffset(i, j) + ") ");
- System.err.print("{");
- bbis.setByteBuffer(buffer, getTupleStartOffset(i) + getFieldSlotsLength() + getFieldStartOffset(i, j));
- try {
- System.err.print(recordDescriptor.getFields()[j].deserialize(dis));
- } catch (HyracksDataException e) {
- e.printStackTrace();
- }
- System.err.print("}");
- }
- System.err.println("]");
+ prettyPrint(i, bbis, dis, sb);
}
+ System.err.println(sb.toString());
+ }
+
+ public void prettyPrint() {
+ prettyPrint("");
+ }
+
+ protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
+ sb.append(" tid" + tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
+ for (int j = 0; j < getFieldCount(); ++j) {
+ sb.append("f" + j + ":(" + getFieldStartOffset(tid, j) + ", " + getFieldEndOffset(tid, j) + ") ");
+ sb.append("{");
+ bbis.setByteBuffer(buffer, getTupleStartOffset(tid) + getFieldSlotsLength() + getFieldStartOffset(tid, j));
+ try {
+ sb.append(recordDescriptor.getFields()[j].deserialize(dis));
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ sb.append("}");
+ }
+ sb.append("\n");
+ }
+
+ public void prettyPrint(int tid) {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ StringBuilder sb = new StringBuilder();
+ prettyPrint(tid, bbis, dis, sb);
+ System.err.println(sb.toString());
}
@Override
public int getFieldCount() {
return recordDescriptor.getFieldCount();
}
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
index 98b8e79..8a3c35d 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppender.java
@@ -1,164 +1,121 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
+ * 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.
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.common.comm.io;
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
-public class FrameTupleAppender {
- private final int frameSize;
+public class FrameTupleAppender extends AbstractFrameAppender implements IFrameTupleAppender {
- private ByteBuffer buffer;
-
- private int tupleCount;
-
- private int tupleDataEndOffset;
-
- private int numberOfFields = -1;
- private int currentField = 0;
- private int lastFieldEndOffset = 0;
-
- public FrameTupleAppender(int frameSize) {
- this.frameSize = frameSize;
+ public FrameTupleAppender() {
}
- public FrameTupleAppender(int frameSize, int numberOfFields) {
- this.frameSize = frameSize;
- this.numberOfFields = numberOfFields;
+ public FrameTupleAppender(IFrame frame) throws HyracksDataException {
+ reset(frame, true);
}
- public void reset(ByteBuffer buffer, boolean clear) {
- this.buffer = buffer;
- if (clear) {
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), 0);
- tupleCount = 0;
- tupleDataEndOffset = 0;
- } else {
- tupleCount = buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
- tupleDataEndOffset = tupleCount == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize)
- - tupleCount * 4);
- }
+ public FrameTupleAppender(IFrame frame, boolean clear) throws HyracksDataException {
+ reset(frame, clear);
}
- public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) {
- if (tupleDataEndOffset + fieldSlots.length * 4 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ public boolean append(int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+ if (canHoldNewTuple(fieldSlots.length, length)) {
for (int i = 0; i < fieldSlots.length; ++i) {
- buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots[i]);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots[i]);
}
- System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + fieldSlots.length * 4, length);
+ System.arraycopy(bytes, offset, array, tupleDataEndOffset + fieldSlots.length * 4, length);
tupleDataEndOffset += fieldSlots.length * 4 + length;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(getBuffer().array(),
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(getBuffer().array(), FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public boolean append(byte[] bytes, int offset, int length) {
- if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
- System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset, length);
+ public boolean append(byte[] bytes, int offset, int length) throws HyracksDataException {
+ if (canHoldNewTuple(0, length)) {
+ System.arraycopy(bytes, offset, getBuffer().array(), tupleDataEndOffset, length);
tupleDataEndOffset += length;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length) {
- if (tupleDataEndOffset + fieldSlots.length * 4 + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ public boolean appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
+ throws HyracksDataException {
+ if (canHoldNewTuple(fieldSlots.length, length)) {
int effectiveSlots = 0;
for (int i = 0; i < fieldSlots.length; ++i) {
if (fieldSlots[i] > 0) {
- buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots[i]);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots[i]);
effectiveSlots++;
}
}
- System.arraycopy(bytes, offset, buffer.array(), tupleDataEndOffset + effectiveSlots * 4, length);
+ System.arraycopy(bytes, offset, array, tupleDataEndOffset + effectiveSlots * 4, length);
tupleDataEndOffset += effectiveSlots * 4 + length;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+ tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public boolean appendField(byte[] bytes, int offset, int length) {
- if (numberOfFields < 0) {
- throw new IllegalStateException("unintialized number of fields " + numberOfFields);
- }
- int currentTupleDataStart = tupleDataEndOffset + numberOfFields * 4 + lastFieldEndOffset;
- if (currentTupleDataStart + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
- System.arraycopy(bytes, offset, buffer.array(), currentTupleDataStart, length);
- lastFieldEndOffset = lastFieldEndOffset + length;
- buffer.putInt(tupleDataEndOffset + currentField * 4, lastFieldEndOffset);
- if (++currentField == numberOfFields) {
- tupleDataEndOffset += numberOfFields * 4 + lastFieldEndOffset;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
- ++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
-
- //reset for the next tuple
- currentField = 0;
- lastFieldEndOffset = 0;
- }
- return true;
- } else {
- //reset for the next tuple
- currentField = 0;
- lastFieldEndOffset = 0;
- return false;
- }
- }
-
- public boolean appendField(IFrameTupleAccessor fta, int tIndex, int fIndex) {
- if (numberOfFields < 0) {
- throw new IllegalStateException("unintialized number of fields " + numberOfFields);
- }
- int startOffset = fta.getTupleStartOffset(tIndex);
- int fStartOffset = fta.getFieldStartOffset(tIndex, fIndex);
- int fLen = fta.getFieldEndOffset(tIndex, fIndex) - fStartOffset;
- return appendField(fta.getBuffer().array(), startOffset + fta.getFieldSlotsLength() + fStartOffset, fLen);
- }
-
- public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) {
+ public boolean append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+ throws HyracksDataException {
int length = tEndOffset - tStartOffset;
- if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ if (canHoldNewTuple(0, length)) {
ByteBuffer src = tupleAccessor.getBuffer();
- System.arraycopy(src.array(), tStartOffset, buffer.array(), tupleDataEndOffset, length);
+ System.arraycopy(src.array(), tStartOffset, array, tupleDataEndOffset, length);
tupleDataEndOffset += length;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1),
+ tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) {
+ public boolean append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
int tStartOffset = tupleAccessor.getTupleStartOffset(tIndex);
int tEndOffset = tupleAccessor.getTupleEndOffset(tIndex);
return append(tupleAccessor, tStartOffset, tEndOffset);
}
- public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1) {
+ public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1,
+ int tIndex1) throws HyracksDataException {
int startOffset0 = accessor0.getTupleStartOffset(tIndex0);
int endOffset0 = accessor0.getTupleEndOffset(tIndex0);
int length0 = endOffset0 - startOffset0;
@@ -167,7 +124,7 @@
int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
int length1 = endOffset1 - startOffset1;
- if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ if (canHoldNewTuple(0, length0 + length1)) {
ByteBuffer src0 = accessor0.getBuffer();
ByteBuffer src1 = accessor1.getBuffer();
int slotsLen0 = accessor0.getFieldSlotsLength();
@@ -175,28 +132,31 @@
int dataLen0 = length0 - slotsLen0;
int dataLen1 = length1 - slotsLen1;
// Copy slots from accessor0 verbatim
- System.arraycopy(src0.array(), startOffset0, buffer.array(), tupleDataEndOffset, slotsLen0);
+ System.arraycopy(src0.array(), startOffset0, array, tupleDataEndOffset, slotsLen0);
// Copy slots from accessor1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
for (int i = 0; i < slotsLen1 / 4; ++i) {
- buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, src1.getInt(startOffset1 + i * 4) + dataLen0);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+ src1.getInt(startOffset1 + i * 4) + dataLen0);
}
// Copy data0
- System.arraycopy(src0.array(), startOffset0 + slotsLen0, buffer.array(), tupleDataEndOffset + slotsLen0
+ System.arraycopy(src0.array(), startOffset0 + slotsLen0, array, tupleDataEndOffset + slotsLen0
+ slotsLen1, dataLen0);
// Copy data1
- System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
+ System.arraycopy(src1.array(), startOffset1 + slotsLen1, array, tupleDataEndOffset + slotsLen0
+ slotsLen1 + dataLen0, dataLen1);
tupleDataEndOffset += (length0 + length1);
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
public boolean appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1,
- int offset1, int dataLen1) {
+ int offset1, int dataLen1) throws HyracksDataException {
int startOffset0 = accessor0.getTupleStartOffset(tIndex0);
int endOffset0 = accessor0.getTupleEndOffset(tIndex0);
int length0 = endOffset0 - startOffset0;
@@ -204,33 +164,36 @@
int slotsLen1 = fieldSlots1.length * 4;
int length1 = slotsLen1 + dataLen1;
- if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ if (canHoldNewTuple(0, length0 + length1)) {
ByteBuffer src0 = accessor0.getBuffer();
int slotsLen0 = accessor0.getFieldSlotsLength();
int dataLen0 = length0 - slotsLen0;
// Copy slots from accessor0 verbatim
- System.arraycopy(src0.array(), startOffset0, buffer.array(), tupleDataEndOffset, slotsLen0);
+ System.arraycopy(src0.array(), startOffset0, array, tupleDataEndOffset, slotsLen0);
// Copy fieldSlots1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
for (int i = 0; i < fieldSlots1.length; ++i) {
- buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, (fieldSlots1[i] + dataLen0));
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+ (fieldSlots1[i] + dataLen0));
}
// Copy data0
- System.arraycopy(src0.array(), startOffset0 + slotsLen0, buffer.array(), tupleDataEndOffset + slotsLen0
+ System.arraycopy(src0.array(), startOffset0 + slotsLen0, array, tupleDataEndOffset + slotsLen0
+ slotsLen1, dataLen0);
// Copy bytes1
- System.arraycopy(bytes1, offset1, buffer.array(), tupleDataEndOffset + slotsLen0 + fieldSlots1.length * 4
- + dataLen0, dataLen1);
+ System.arraycopy(bytes1, offset1, array,
+ tupleDataEndOffset + slotsLen0 + fieldSlots1.length * 4 + dataLen0, dataLen1);
tupleDataEndOffset += (length0 + length1);
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
public boolean appendConcat(int[] fieldSlots0, byte[] bytes0, int offset0, int dataLen0,
- IFrameTupleAccessor accessor1, int tIndex1) {
+ IFrameTupleAccessor accessor1, int tIndex1) throws HyracksDataException {
int slotsLen0 = fieldSlots0.length * 4;
int length0 = slotsLen0 + dataLen0;
@@ -238,40 +201,45 @@
int endOffset1 = accessor1.getTupleEndOffset(tIndex1);
int length1 = endOffset1 - startOffset1;
- if (tupleDataEndOffset + length0 + length1 + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ if (canHoldNewTuple(0, length0 + length1)) {
ByteBuffer src1 = accessor1.getBuffer();
int slotsLen1 = accessor1.getFieldSlotsLength();
int dataLen1 = length1 - slotsLen1;
// Copy fieldSlots0 verbatim
for (int i = 0; i < fieldSlots0.length; ++i) {
- buffer.putInt(tupleDataEndOffset + i * 4, fieldSlots0[i]);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fieldSlots0[i]);
}
// Copy slots from accessor1 with the following transformation: newSlotIdx = oldSlotIdx + dataLen0
for (int i = 0; i < slotsLen1 / 4; ++i) {
- buffer.putInt(tupleDataEndOffset + slotsLen0 + i * 4, src1.getInt(startOffset1 + i * 4) + dataLen0);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + slotsLen0 + i * 4,
+ src1.getInt(startOffset1 + i * 4) + dataLen0);
}
// Copy bytes0
- System.arraycopy(bytes0, offset0, buffer.array(), tupleDataEndOffset + slotsLen0 + slotsLen1, dataLen0);
+ System.arraycopy(bytes0, offset0, array, tupleDataEndOffset + slotsLen0 + slotsLen1,
+ dataLen0);
// Copy data1
- System.arraycopy(src1.array(), startOffset1 + slotsLen1, buffer.array(), tupleDataEndOffset + slotsLen0
+ System.arraycopy(src1.array(), startOffset1 + slotsLen1, array, tupleDataEndOffset + slotsLen0
+ slotsLen1 + dataLen0, dataLen1);
tupleDataEndOffset += (length0 + length1);
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) {
+ public boolean appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields)
+ throws HyracksDataException {
int fTargetSlotsLength = fields.length * 4;
int length = fTargetSlotsLength;
for (int i = 0; i < fields.length; ++i) {
length += (accessor.getFieldEndOffset(tIndex, fields[i]) - accessor.getFieldStartOffset(tIndex, fields[i]));
}
- if (tupleDataEndOffset + length + 4 + (tupleCount + 1) * 4 <= frameSize) {
+ if (canHoldNewTuple(0, length)) {
int fSrcSlotsLength = accessor.getFieldSlotsLength();
int tStartOffset = accessor.getTupleStartOffset(tIndex);
@@ -281,26 +249,21 @@
int fSrcStart = tStartOffset + fSrcSlotsLength + accessor.getFieldStartOffset(tIndex, fields[i]);
int fLen = accessor.getFieldEndOffset(tIndex, fields[i])
- accessor.getFieldStartOffset(tIndex, fields[i]);
- System.arraycopy(accessor.getBuffer().array(), fSrcStart, buffer.array(), tupleDataEndOffset
+ System.arraycopy(accessor.getBuffer().array(), fSrcStart, array, tupleDataEndOffset
+ fTargetSlotsLength + fStartOffset, fLen);
fEndOffset += fLen;
- buffer.putInt(tupleDataEndOffset + i * 4, fEndOffset);
+ IntSerDeUtils.putInt(array, tupleDataEndOffset + i * 4, fEndOffset);
fStartOffset = fEndOffset;
}
tupleDataEndOffset += length;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleCount + 1), tupleDataEndOffset);
+ IntSerDeUtils.putInt(array,
+ FrameHelper.getTupleCountOffset(frame.getFrameSize()) - 4 * (tupleCount + 1), tupleDataEndOffset);
++tupleCount;
- buffer.putInt(FrameHelper.getTupleCountOffset(frameSize), tupleCount);
+ IntSerDeUtils
+ .putInt(array, FrameHelper.getTupleCountOffset(frame.getFrameSize()), tupleCount);
return true;
}
return false;
}
- public int getTupleCount() {
- return tupleCount;
- }
-
- public ByteBuffer getBuffer() {
- return buffer;
- }
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java
new file mode 100644
index 0000000..b29c8d1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderAccessor.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameConstants;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class FrameTupleAppenderAccessor extends FrameTupleAppender implements IFrameTupleAccessor {
+ private int tupleCountOffset;
+ private final RecordDescriptor recordDescriptor;
+
+ public FrameTupleAppenderAccessor(RecordDescriptor recordDescriptor) {
+ super();
+ this.recordDescriptor = recordDescriptor;
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ throw new IllegalAccessError("should not call this function");
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ int offset = tupleIndex == 0 ?
+ FrameConstants.TUPLE_START_OFFSET :
+ IntSerDeUtils.getInt(getBuffer().array(), tupleCountOffset - 4 * tupleIndex);
+ return offset;
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return IntSerDeUtils.getInt(getBuffer().array(), tupleCountOffset - 4 * (tupleIndex + 1));
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return fIdx == 0 ? 0 : IntSerDeUtils.getInt(getBuffer().array(),
+ getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return IntSerDeUtils.getInt(getBuffer().array(), getTupleStartOffset(tupleIndex) + fIdx * 4);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return getFieldCount() * 4;
+ }
+
+ public void prettyPrint() {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ int tc = getTupleCount();
+ System.err.println("TC: " + tc);
+ for (int i = 0; i < tc; ++i) {
+ prettyPrint(i, bbis, dis);
+ }
+ }
+
+ protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis) {
+ System.err.print("tid" + tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
+ for (int j = 0; j < getFieldCount(); ++j) {
+ System.err.print("f" + j + ":(" + getFieldStartOffset(tid, j) + ", " + getFieldEndOffset(tid, j) + ") ");
+ System.err.print("{");
+ bbis.setByteBuffer(getBuffer(),
+ getTupleStartOffset(tid) + getFieldSlotsLength() + getFieldStartOffset(tid, j));
+ try {
+ System.err.print(recordDescriptor.getFields()[j].deserialize(dis));
+ } catch (HyracksDataException e) {
+ e.printStackTrace();
+ }
+ System.err.print("}");
+ }
+ System.err.println();
+ }
+
+ public void prettyPrint(int tid) {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ prettyPrint(tid, bbis, dis);
+ }
+
+ @Override
+ public int getFieldCount() {
+ return recordDescriptor.getFieldCount();
+ }
+
+ @Override
+ public void reset(IFrame frame, boolean clear) throws HyracksDataException {
+ super.reset(frame, clear);
+ this.tupleCountOffset = FrameHelper.getTupleCountOffset(frame.getFrameSize());
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
index 2de4256..b8e7450 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameTupleAppenderWrapper.java
@@ -15,9 +15,9 @@
package edu.uci.ics.hyracks.dataflow.common.comm.io;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -26,18 +26,15 @@
* This class wraps the calls of FrameTupleAppender and
* allows user to not worry about flushing full frames.
* TODO(yingyib): cleanup existing usage of FrameTupleAppender.
- *
+ *
* @author yingyib
*/
public class FrameTupleAppenderWrapper {
- private final FrameTupleAppender frameTupleAppender;
- private final ByteBuffer outputFrame;
+ private final IFrameTupleAppender frameTupleAppender;
private final IFrameWriter outputWriter;
- public FrameTupleAppenderWrapper(FrameTupleAppender frameTupleAppender, ByteBuffer outputFrame,
- IFrameWriter outputWriter) {
+ public FrameTupleAppenderWrapper(IFrameTupleAppender frameTupleAppender, IFrameWriter outputWriter) {
this.frameTupleAppender = frameTupleAppender;
- this.outputFrame = outputFrame;
this.outputWriter = outputWriter;
}
@@ -46,9 +43,7 @@
}
public void flush() throws HyracksDataException {
- if (frameTupleAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- }
+ frameTupleAppender.flush(outputWriter, true);
}
public void close() throws HyracksDataException {
@@ -59,81 +54,42 @@
outputWriter.fail();
}
- public void reset(ByteBuffer buffer, boolean clear) {
+ public void reset(IFrame buffer, boolean clear) throws HyracksDataException {
frameTupleAppender.reset(buffer, clear);
}
public void appendSkipEmptyField(int[] fieldSlots, byte[] bytes, int offset, int length)
throws HyracksDataException {
- if (!frameTupleAppender.append(fieldSlots, bytes, offset, length)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendSkipEmptyFieldToWriter(outputWriter, frameTupleAppender,
+ fieldSlots, bytes, offset, length);
}
public void append(byte[] bytes, int offset, int length) throws HyracksDataException {
- if (!frameTupleAppender.append(bytes, offset, length)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.append(bytes, offset, length)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendToWriter(outputWriter, frameTupleAppender, bytes, offset, length);
}
- public void append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset) throws HyracksDataException {
- if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ public void append(IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+ throws HyracksDataException {
+ FrameUtils.appendToWriter(outputWriter, frameTupleAppender, tupleAccessor, tStartOffset, tEndOffset);
}
public void append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
- if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendToWriter(outputWriter, frameTupleAppender, tupleAccessor, tIndex);
}
public void appendConcat(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
throws HyracksDataException {
- if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendConcatToWriter(outputWriter, frameTupleAppender, accessor0, tIndex0, accessor1, tIndex1);
}
public void appendConcat(IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1, int offset1,
int dataLen1) throws HyracksDataException {
- if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendConcatToWriter(outputWriter, frameTupleAppender, accessor0, tIndex0,
+ fieldSlots1, bytes1, offset1, dataLen1);
}
public void appendProjection(IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException {
- if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
- FrameUtils.flushFrame(outputFrame, outputWriter);
- frameTupleAppender.reset(outputFrame, true);
- if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
- throw new HyracksDataException("The output cannot be fit into a frame.");
- }
- }
+ FrameUtils.appendProjectionToWriter(outputWriter, frameTupleAppender, accessor, tIndex, fields);
}
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
index 471b1ef..0995564 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
@@ -15,79 +15,23 @@
package edu.uci.ics.hyracks.dataflow.common.comm.io;
import java.io.DataInputStream;
-import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+public class ResultFrameTupleAccessor extends FrameTupleAccessor {
- private final int frameSize;
- private ByteBuffer buffer;
-
- public ResultFrameTupleAccessor(int frameSize) {
- this.frameSize = frameSize;
+ public ResultFrameTupleAccessor() {
+ super(null);
}
@Override
- public void reset(ByteBuffer buffer) {
- this.buffer = buffer;
- }
+ protected void prettyPrint(int tid, ByteBufferInputStream bbis, DataInputStream dis, StringBuilder sb) {
+ sb.append(tid + ":(" + getTupleStartOffset(tid) + ", " + getTupleEndOffset(tid) + ")[");
- @Override
- public ByteBuffer getBuffer() {
- return buffer;
- }
+ bbis.setByteBuffer(getBuffer(), getTupleStartOffset(tid));
+ sb.append(dis);
- @Override
- public int getTupleCount() {
- return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
- }
-
- @Override
- public int getTupleStartOffset(int tupleIndex) {
- return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
- }
-
- @Override
- public int getTupleEndOffset(int tupleIndex) {
- return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
- }
-
- @Override
- public int getFieldStartOffset(int tupleIndex, int fIdx) {
- return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
- }
-
- @Override
- public int getFieldEndOffset(int tupleIndex, int fIdx) {
- return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
- }
-
- @Override
- public int getFieldLength(int tupleIndex, int fIdx) {
- return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
- }
-
- @Override
- public int getFieldSlotsLength() {
- return getFieldCount() * 4;
- }
-
- public void prettyPrint() {
- ByteBufferInputStream bbis = new ByteBufferInputStream();
- DataInputStream dis = new DataInputStream(bbis);
- int tc = getTupleCount();
- System.err.println("TC: " + tc);
- for (int i = 0; i < tc; ++i) {
- System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
-
- bbis.setByteBuffer(buffer, getTupleStartOffset(i));
- System.err.print(dis);
-
- System.err.println("]");
- }
+ sb.append("]\n");
}
@Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
index 4392fd6..3feeb58 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/SerializingDataWriter.java
@@ -14,21 +14,20 @@
*/
package edu.uci.ics.hyracks.dataflow.common.comm.io;
-import java.nio.ByteBuffer;
import java.util.logging.Level;
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class SerializingDataWriter implements IOpenableDataWriter<Object[]> {
private static final Logger LOGGER = Logger.getLogger(SerializingDataWriter.class.getName());
- private final ByteBuffer buffer;
-
private final ArrayTupleBuilder tb;
private final RecordDescriptor recordDescriptor;
@@ -41,20 +40,17 @@
public SerializingDataWriter(IHyracksTaskContext ctx, RecordDescriptor recordDescriptor, IFrameWriter frameWriter)
throws HyracksDataException {
- buffer = ctx.allocateFrame();
tb = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
this.recordDescriptor = recordDescriptor;
this.frameWriter = frameWriter;
- tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
+ tupleAppender = new FrameTupleAppender(new VSizeFrame(ctx));
open = false;
}
@Override
public void open() throws HyracksDataException {
frameWriter.open();
- buffer.clear();
open = true;
- tupleAppender.reset(buffer, true);
}
@Override
@@ -62,9 +58,7 @@
if (!open) {
throw new HyracksDataException("Closing SerializingDataWriter that has not been opened");
}
- if (tupleAppender.getTupleCount() > 0) {
- flushFrame();
- }
+ tupleAppender.flush(frameWriter, true);
frameWriter.close();
open = false;
}
@@ -82,22 +76,8 @@
}
tb.addField(recordDescriptor.getFields()[i], instance);
}
- if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- if (LOGGER.isLoggable(Level.FINEST)) {
- LOGGER.finest("Flushing: position = " + buffer.position());
- }
- flushFrame();
- tupleAppender.reset(buffer, true);
- if (!tupleAppender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + buffer.capacity() + ")");
- }
- }
- }
-
- private void flushFrame() throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
- frameWriter.nextFrame(buffer);
+ FrameUtils.appendToWriter(frameWriter, tupleAppender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
@Override
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
index 4387e68..e2a9e5f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/util/FrameUtils.java
@@ -16,32 +16,268 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrameFieldAppender;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public class FrameUtils {
- public static void copy(ByteBuffer srcFrame, ByteBuffer destFrame) {
- makeReadable(srcFrame);
+
+ public static void copyWholeFrame(ByteBuffer srcFrame, ByteBuffer destFrame) {
+ srcFrame.clear();
destFrame.clear();
destFrame.put(srcFrame);
}
- public static void makeReadable(ByteBuffer frame) {
- frame.position(0);
- frame.limit(frame.capacity());
+ public static void copyAndFlip(ByteBuffer srcFrame, ByteBuffer destFrame) {
+ srcFrame.position(0);
+ destFrame.clear();
+ destFrame.put(srcFrame);
+ destFrame.flip();
}
public static void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
writer.nextFrame(buffer);
- buffer.position(0);
- buffer.limit(buffer.capacity());
+ buffer.clear();
}
- public static int getAbsoluteFieldStartOffset(IFrameTupleAccessor accessor, int tuple, int field) {
- return accessor.getTupleStartOffset(tuple) + accessor.getFieldSlotsLength()
- + accessor.getFieldStartOffset(tuple, field);
+ /**
+ * A util function to append the data to appender. If the appender buffer is full, it will directly flush
+ * to the given writer, which saves the detecting logic in the caller.
+ * It will return the bytes that have been flushed.
+ *
+ * @param writer
+ * @param frameTupleAppender
+ * @param fieldSlots
+ * @param bytes
+ * @param offset
+ * @param length
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendSkipEmptyFieldToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ int[] fieldSlots, byte[] bytes, int offset, int length) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.appendSkipEmptyField(fieldSlots, bytes, offset, length)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
}
+
+ /**
+ * A util function to append the data to appender. If the appender buffer is full, it will directly flush
+ * to the given writer, which saves the detecting logic in the caller.
+ * It will return the bytes that have been flushed.
+ *
+ * @param writer
+ * @param frameTupleAppender
+ * @param bytes
+ * @param offset
+ * @param length
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender, byte[] bytes,
+ int offset, int length) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.append(bytes, offset, length)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.append(bytes, offset, length)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param frameTupleAppender
+ * @param tupleAccessor
+ * @param tStartOffset
+ * @param tEndOffset
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ IFrameTupleAccessor tupleAccessor, int tStartOffset, int tEndOffset)
+ throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.append(tupleAccessor, tStartOffset, tEndOffset)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param frameTupleAppender
+ * @param tupleAccessor
+ * @param tIndex
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.append(tupleAccessor, tIndex)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param tupleAppender
+ * @param fieldEndOffsets
+ * @param byteArray
+ * @param start
+ * @param size
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendToWriter(IFrameWriter writer, IFrameTupleAppender tupleAppender,
+ int[] fieldEndOffsets, byte[] byteArray, int start, int size) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!tupleAppender.append(fieldEndOffsets, byteArray, start, size)) {
+
+ flushedBytes = tupleAppender.getBuffer().capacity();
+ tupleAppender.flush(writer, true);
+
+ if (!tupleAppender.append(fieldEndOffsets, byteArray, start, size)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param frameTupleAppender
+ * @param accessor0
+ * @param tIndex0
+ * @param accessor1
+ * @param tIndex1
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendConcatToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
+ throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.appendConcat(accessor0, tIndex0, accessor1, tIndex1)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param frameTupleAppender
+ * @param accessor0
+ * @param tIndex0
+ * @param fieldSlots1
+ * @param bytes1
+ * @param offset1
+ * @param dataLen1
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendConcatToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ IFrameTupleAccessor accessor0, int tIndex0, int[] fieldSlots1, byte[] bytes1, int offset1,
+ int dataLen1) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.appendConcat(accessor0, tIndex0, fieldSlots1, bytes1, offset1, dataLen1)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param frameTupleAppender
+ * @param accessor
+ * @param tIndex
+ * @param fields
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendProjectionToWriter(IFrameWriter writer, IFrameTupleAppender frameTupleAppender,
+ IFrameTupleAccessor accessor, int tIndex, int[] fields) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
+ flushedBytes = frameTupleAppender.getBuffer().capacity();
+ frameTupleAppender.flush(writer, true);
+ if (!frameTupleAppender.appendProjection(accessor, tIndex, fields)) {
+ throw new HyracksDataException("The output cannot be fit into a frame.");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param appender
+ * @param array
+ * @param start
+ * @param length
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendFieldToWriter(IFrameWriter writer, IFrameFieldAppender appender, byte[] array,
+ int start, int length) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!appender.appendField(array, start, length)) {
+ flushedBytes = appender.getBuffer().capacity();
+ appender.flush(writer, true);
+ if (!appender.appendField(array, start, length)) {
+ throw new HyracksDataException("Could not write frame: the size of the tuple is too long");
+ }
+ }
+ return flushedBytes;
+ }
+
+ /**
+ * @param writer
+ * @param appender
+ * @param accessor
+ * @param tid
+ * @param fid
+ * @return the number of bytes that have been flushed, 0 if not get flushed.
+ * @throws HyracksDataException
+ */
+ public static int appendFieldToWriter(IFrameWriter writer, IFrameFieldAppender appender,
+ IFrameTupleAccessor accessor, int tid, int fid) throws HyracksDataException {
+ int flushedBytes = 0;
+ if (!appender.appendField(accessor, tid, fid)) {
+ flushedBytes = appender.getBuffer().capacity();
+ appender.flush(writer, true);
+ if (!appender.appendField(accessor, tid, fid)) {
+ throw new HyracksDataException("Could not write frame: the size of the tuple is too long");
+ }
+ }
+ return flushedBytes;
+ }
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
index 31cd25c..be40d9f 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/io/RunFileReader.java
@@ -14,8 +14,8 @@
*/
package edu.uci.ics.hyracks.dataflow.common.io;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -43,12 +43,30 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- buffer.clear();
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
if (readPtr >= size) {
return false;
}
- readPtr += ioManager.syncRead(handle, readPtr, buffer);
+ frame.reset();
+ int readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
+ if (readLength <= 0) {
+ throw new HyracksDataException("Premature end of file");
+ }
+ readPtr += readLength;
+ frame.ensureFrameSize(frame.getMinSize() * FrameHelper.deserializeNumOfMinFrame(frame.getBuffer()));
+ if (frame.getBuffer().hasRemaining()) {
+ if (readPtr < size) {
+ readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
+ if (readLength < 0) {
+ throw new HyracksDataException("Premature end of file");
+ }
+ readPtr += readLength;
+ }
+ if (frame.getBuffer().hasRemaining()) { // file is vanished.
+ FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
+ }
+ }
+ frame.getBuffer().flip();
return true;
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
index 9faef09..4e2a985 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/util/IntSerDeUtils.java
@@ -22,4 +22,10 @@
+ ((bytes[offset + 3] & 0xff) << 0);
}
+ public static void putInt(byte[] bytes, int offset, int value) {
+ bytes[offset++] = (byte) (value >> 24);
+ bytes[offset++] = (byte) (value >> 16);
+ bytes[offset++] = (byte) (value >> 8);
+ bytes[offset++] = (byte) (value);
+ }
}
diff --git a/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java b/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java
new file mode 100644
index 0000000..ab9333c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/test/java/edu/uci/ics/hyracks/dataflow/common/comm/io/largeobject/FrameFixedFieldTupleAppenderTest.java
@@ -0,0 +1,215 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.common.comm.io.largeobject;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameFixedFieldAppender;
+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.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+
+public class FrameFixedFieldTupleAppenderTest {
+
+ static final int INPUT_BUFFER_SIZE = 4096;
+ static final int TEST_FRAME_SIZE = 256;
+
+ FrameFixedFieldAppender appender;
+ static ISerializerDeserializer[] fields = new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE,
+ };
+ static RecordDescriptor recordDescriptor = new RecordDescriptor(fields);
+ static ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+
+ class SequetialDataVerifier implements IFrameWriter {
+
+ private final IFrameTupleAccessor accessor;
+ private IFrameTupleAccessor innerAccessor;
+ private int tid;
+
+ public SequetialDataVerifier(IFrameTupleAccessor accessor) {
+ this.accessor = accessor;
+ this.innerAccessor = new FrameTupleAccessor(recordDescriptor);
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ this.tid = 0;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ innerAccessor.reset(buffer);
+ for (int i = 0; i < innerAccessor.getTupleCount(); ++i) {
+ validate(innerAccessor, i);
+ }
+ }
+
+ private void validate(IFrameTupleAccessor innerAccessor, int i) {
+ assertTrue(tid < accessor.getTupleCount());
+ assertEquals(accessor.getTupleLength(tid), innerAccessor.getTupleLength(i));
+ assertArrayEquals(Arrays.copyOfRange(accessor.getBuffer().array(), accessor.getTupleStartOffset(tid),
+ accessor.getTupleEndOffset(tid)),
+ Arrays.copyOfRange(innerAccessor.getBuffer().array(), innerAccessor.getTupleStartOffset(i),
+ innerAccessor.getTupleEndOffset(i)));
+ tid++;
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ assert false;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ assertEquals(accessor.getTupleCount(), tid);
+ }
+ }
+
+ @Before
+ public void createAppender() throws HyracksDataException {
+ appender = new FrameFixedFieldAppender(fields.length);
+ FrameManager manager = new FrameManager(TEST_FRAME_SIZE);
+ IFrame frame = new VSizeFrame(manager);
+ appender.reset(frame, true);
+ }
+
+ private void testProcess(IFrameTupleAccessor accessor) throws HyracksDataException {
+ IFrameWriter writer = prepareValidator(accessor);
+ writer.open();
+ for (int tid = 0; tid < accessor.getTupleCount(); tid++) {
+ for (int fid = 0; fid < fields.length; fid++) {
+ if (!appender.appendField(accessor, tid, fid)) {
+ appender.flush(writer, true);
+ if (!appender.appendField(accessor, tid, fid)) {
+ }
+ }
+ }
+ }
+ appender.flush(writer, true);
+ writer.close();
+ }
+
+ @Test
+ public void testAppendFieldShouldSucceed() throws HyracksDataException {
+ IFrameTupleAccessor accessor = prepareData(DATA_TYPE.NORMAL_RECORD);
+ testProcess(accessor);
+ }
+
+ @Test
+ public void testResetShouldWork() throws HyracksDataException {
+ testAppendFieldShouldSucceed();
+ appender.reset(new VSizeFrame(new FrameManager(TEST_FRAME_SIZE)), true);
+ testAppendFieldShouldSucceed();
+ }
+
+ private IFrameWriter prepareValidator(IFrameTupleAccessor accessor) throws HyracksDataException {
+ return new SequetialDataVerifier(accessor);
+ }
+
+ enum DATA_TYPE {
+ NORMAL_RECORD,
+ ONE_FIELD_LONG,
+ ONE_RECORD_LONG,
+ }
+
+ private IFrameTupleAccessor prepareData(DATA_TYPE type) throws HyracksDataException {
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(recordDescriptor);
+ IFrameTupleAppender appender = new FrameTupleAppender(
+ new VSizeFrame(new FrameManager(INPUT_BUFFER_SIZE)), true);
+ int i = 0;
+ do {
+ switch (type) {
+ case NORMAL_RECORD:
+ makeATuple(tupleBuilder, i++);
+ break;
+ case ONE_FIELD_LONG:
+ makeASizeUpTuple(tupleBuilder, i++);
+ break;
+ case ONE_RECORD_LONG:
+ makeABigObjectTuple(tupleBuilder, i++);
+ break;
+ }
+ } while (appender
+ .append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize()));
+ accessor.reset(appender.getBuffer());
+ return accessor;
+ }
+
+ private void makeATuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+ tupleBuilder.reset();
+ tupleBuilder.addField(fields[0], i);
+ tupleBuilder.addField(fields[1], String.valueOf(i));
+ tupleBuilder.addField(fields[2], -i);
+ tupleBuilder.addField(fields[3], String.valueOf(-i));
+ }
+
+ private String makeALongString(int length, char ch) {
+ char[] array = new char[length];
+ Arrays.fill(array, ch);
+ return new String(array);
+ }
+
+ private void makeASizeUpTuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+ tupleBuilder.reset();
+ tupleBuilder.addField(fields[0], i);
+ tupleBuilder.addField(fields[1], makeALongString(Math.min(Math.abs(1 << i), INPUT_BUFFER_SIZE), (char) i));
+ tupleBuilder.addField(fields[2], -i);
+ tupleBuilder.addField(fields[3], String.valueOf(-i));
+ }
+
+ private void makeABigObjectTuple(ArrayTupleBuilder tupleBuilder, int i) throws HyracksDataException {
+ tupleBuilder.reset();
+ tupleBuilder.addField(fields[0], i);
+ tupleBuilder.addField(fields[1], makeALongString(Math.min(i * 20, TEST_FRAME_SIZE), (char) i));
+ tupleBuilder.addField(fields[2], -i);
+ tupleBuilder.addField(fields[3], makeALongString(Math.min(i * 20, TEST_FRAME_SIZE), (char) i));
+ }
+
+ @Test
+ public void testAppendLargeFieldShouldSucceed() throws HyracksDataException {
+ IFrameTupleAccessor accessor = prepareData(DATA_TYPE.ONE_FIELD_LONG);
+ testProcess(accessor);
+ }
+
+ @Test
+ public void testAppendSmallFieldButLargeObjectWithShouldSucceed() throws HyracksDataException {
+ IFrameTupleAccessor accessor = prepareData(DATA_TYPE.ONE_RECORD_LONG);
+ testProcess(accessor);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
index 9bce999..0c06769 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
@@ -29,7 +29,6 @@
package edu.uci.ics.hyracks.dataflow.hadoop;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
@@ -45,6 +44,7 @@
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.util.ReflectionUtils;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
@@ -205,9 +205,7 @@
key = hadoopRecordReader.createKey();
value = hadoopRecordReader.createValue();
- ByteBuffer outBuffer = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(outBuffer, true);
+ FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
RecordDescriptor outputRecordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor(
(Class<? extends Writable>) hadoopRecordReader.createKey().getClass(),
(Class<? extends Writable>) hadoopRecordReader.createValue().getClass());
@@ -223,18 +221,11 @@
case 1:
tb.addField(outputRecordDescriptor.getFields()[1], value);
}
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize()
- + ") larger than frame size (" + outBuffer.capacity() + ")");
- }
- }
+ FrameUtils
+ .appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(),
+ 0, tb.getSize());
}
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outBuffer, writer);
- }
+ appender.flush(writer, true);
} catch (Exception e) {
writer.fail();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
index 5ac55ff..95f53df 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/HadoopHelper.java
@@ -203,7 +203,7 @@
public int getSortFrameLimit(IHyracksCommonContext ctx) {
int sortMemory = job.getConfiguration().getInt("io.sort.mb", 100);
- return (int) (((long) sortMemory * 1024 * 1024) / ctx.getFrameSize());
+ return (int) (((long) sortMemory * 1024 * 1024) / ctx.getInitialFrameSize());
}
public Job getJob() {
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
index 0bbb21f..070fc88 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/KVIterator.java
@@ -15,44 +15,42 @@
package edu.uci.ics.hyracks.dataflow.hadoop.mapreduce;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.List;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.mapred.RawKeyValueIterator;
import org.apache.hadoop.util.Progress;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class KVIterator implements RawKeyValueIterator {
private final HadoopHelper helper;
private FrameTupleAccessor accessor;
private DataInputBuffer kBuffer;
private DataInputBuffer vBuffer;
- private List<ByteBuffer> buffers;
+ private List<IFrame> buffers;
private int bSize;
private int bPtr;
private int tIdx;
private boolean eog;
- public KVIterator(IHyracksTaskContext ctx, HadoopHelper helper, RecordDescriptor recordDescriptor) {
+ public KVIterator(HadoopHelper helper, RecordDescriptor recordDescriptor) {
this.helper = helper;
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+ accessor = new FrameTupleAccessor(recordDescriptor);
kBuffer = new DataInputBuffer();
vBuffer = new DataInputBuffer();
}
- void reset(List<ByteBuffer> buffers, int bSize) {
+ void reset(List<IFrame> buffers, int bSize) {
this.buffers = buffers;
this.bSize = bSize;
bPtr = 0;
tIdx = 0;
eog = false;
if (bSize > 0) {
- accessor.reset(buffers.get(0));
+ accessor.reset(buffers.get(0).getBuffer());
tIdx = -1;
} else {
eog = true;
@@ -83,14 +81,14 @@
continue;
}
tIdx = -1;
- accessor.reset(buffers.get(bPtr));
+ accessor.reset(buffers.get(bPtr).getBuffer());
continue;
}
kBuffer.reset(accessor.getBuffer().array(),
- FrameUtils.getAbsoluteFieldStartOffset(accessor, tIdx, helper.KEY_FIELD_INDEX),
+ accessor.getAbsoluteFieldStartOffset(tIdx, helper.KEY_FIELD_INDEX),
accessor.getFieldLength(tIdx, helper.KEY_FIELD_INDEX));
vBuffer.reset(accessor.getBuffer().array(),
- FrameUtils.getAbsoluteFieldStartOffset(accessor, tIdx, helper.VALUE_FIELD_INDEX),
+ accessor.getAbsoluteFieldStartOffset(tIdx, helper.KEY_FIELD_INDEX),
accessor.getFieldLength(tIdx, helper.VALUE_FIELD_INDEX));
break;
}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
index 2bef21a..bf6e2cf 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
@@ -29,7 +29,9 @@
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -40,13 +42,12 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.hadoop.util.MRContextUtil;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
-import edu.uci.ics.hyracks.dataflow.hadoop.util.MRContextUtil;
public class MapperOperatorDescriptor<K1 extends Writable, V1 extends Writable, K2 extends Writable, V2 extends Writable>
extends AbstractSingleActivityOperatorDescriptor {
@@ -82,16 +83,15 @@
class SortingRecordWriter extends RecordWriter<K2, V2> {
private final ArrayTupleBuilder tb;
- private final ByteBuffer frame;
+ private final IFrame frame;
private final FrameTupleAppender fta;
private ExternalSortRunGenerator runGen;
private int blockId;
public SortingRecordWriter() throws HyracksDataException {
tb = new ArrayTupleBuilder(2);
- frame = ctx.allocateFrame();
- fta = new FrameTupleAppender(ctx.getFrameSize());
- fta.reset(frame, true);
+ frame = new VSizeFrame(ctx);
+ fta = new FrameTupleAppender(frame);
}
public void initBlock(int blockId) throws HyracksDataException {
@@ -113,31 +113,29 @@
value.write(dos);
tb.addFieldEndOffset();
if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- runGen.nextFrame(frame);
+ runGen.nextFrame(frame.getBuffer());
fta.reset(frame, true);
if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
- + frame.capacity() + ")");
+ + frame.getBuffer().capacity() + ")");
}
}
}
public void sortAndFlushBlock(final IFrameWriter writer) throws HyracksDataException {
if (fta.getTupleCount() > 0) {
- runGen.nextFrame(frame);
+ runGen.nextFrame(frame.getBuffer());
fta.reset(frame, true);
}
runGen.close();
IFrameWriter delegatingWriter = new IFrameWriter() {
- private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- private final ByteBuffer outFrame = ctx.allocateFrame();
- private final FrameTupleAccessor fta = new FrameTupleAccessor(ctx.getFrameSize(),
+ private final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
+ private final FrameTupleAccessor fta = new FrameTupleAccessor(
helper.getMapOutputRecordDescriptorWithoutExtraFields());
private final ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
@Override
public void open() throws HyracksDataException {
- appender.reset(outFrame, true);
}
@Override
@@ -155,8 +153,7 @@
}
tb.addFieldEndOffset();
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(outFrame, writer);
- appender.reset(outFrame, true);
+ appender.flush(writer, true);
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
throw new IllegalStateException();
}
@@ -166,9 +163,7 @@
@Override
public void close() throws HyracksDataException {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- }
+ appender.flush(writer, true);
}
@Override
@@ -183,12 +178,10 @@
TaskAttemptContext ctaskAttemptContext = helper.createTaskAttemptContext(taId);
final IFrameWriter outputWriter = delegatingWriter;
RecordWriter<K2, V2> recordWriter = new RecordWriter<K2, V2>() {
- private final FrameTupleAppender fta = new FrameTupleAppender(ctx.getFrameSize());
- private final ByteBuffer buffer = ctx.allocateFrame();
+ private final FrameTupleAppender fta = new FrameTupleAppender(new VSizeFrame(ctx));
private final ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
{
- fta.reset(buffer, true);
outputWriter.open();
}
@@ -201,8 +194,7 @@
value.write(dos);
tb.addFieldEndOffset();
if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(buffer, outputWriter);
- fta.reset(buffer, true);
+ fta.flush(outputWriter, true);
if (!fta.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
throw new IllegalStateException();
}
@@ -211,10 +203,7 @@
@Override
public void close(TaskAttemptContext context) throws IOException, InterruptedException {
- if (fta.getTupleCount() > 0) {
- FrameUtils.flushFrame(buffer, outputWriter);
- outputWriter.close();
- }
+ fta.flush(outputWriter, true);
}
};
delegatingWriter = new ReduceWriter<K2, V2, K2, V2>(ctx, helper,
@@ -226,7 +215,7 @@
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
- ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getFrameSorter(),
+ ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, runGen.getSorter(),
runGen.getRuns(), new int[] { 0 }, comparators, null,
helper.getMapOutputRecordDescriptorWithoutExtraFields(), framesLimit, delegatingWriter);
merger.process();
@@ -253,8 +242,9 @@
Thread.currentThread().setContextClassLoader(ctxCL);
}
recordWriter.initBlock(blockId);
- Mapper<K1, V1, K2, V2>.Context mCtx = new MRContextUtil().createMapContext(conf, taId, recordReader,
- recordWriter, null, null, split);
+ Mapper<K1, V1, K2, V2>.Context mCtx = new MRContextUtil()
+ .createMapContext(conf, taId, recordReader,
+ recordWriter, null, null, split);
mapper.run(mCtx);
recordReader.close();
recordWriter.sortAndFlushBlock(writer);
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
index 8877df4..ccf7276 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ReduceWriter.java
@@ -26,7 +26,9 @@
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.counters.GenericCounter;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -43,7 +45,7 @@
private final int[] groupFields;
private final FrameTupleAccessor accessor0;
private final FrameTupleAccessor accessor1;
- private final ByteBuffer copyFrame;
+ private final IFrame copyFrame;
private final IBinaryComparator[] comparators;
private final KVIterator kvi;
private final Reducer<K2, V2, K3, V3> reducer;
@@ -53,7 +55,7 @@
private boolean first;
private boolean groupStarted;
- private List<ByteBuffer> group;
+ private List<IFrame> group;
private int bPtr;
private FrameTupleAppender fta;
private Counter keyCounter;
@@ -66,10 +68,10 @@
this.ctx = ctx;
this.helper = helper;
this.groupFields = groupFields;
- accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- accessor1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- copyFrame = ctx.allocateFrame();
- accessor1.reset(copyFrame);
+ accessor0 = new FrameTupleAccessor(recordDescriptor);
+ accessor1 = new FrameTupleAccessor(recordDescriptor);
+ copyFrame = new VSizeFrame(ctx);
+ accessor1.reset(copyFrame.getBuffer());
comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -79,17 +81,17 @@
this.taId = taId;
this.taskAttemptContext = taskAttemptContext;
- kvi = new KVIterator(ctx, helper, recordDescriptor);
+ kvi = new KVIterator(helper, recordDescriptor);
}
@Override
public void open() throws HyracksDataException {
first = true;
groupStarted = false;
- group = new ArrayList<ByteBuffer>();
+ group = new ArrayList<>();
bPtr = 0;
- group.add(ctx.allocateFrame());
- fta = new FrameTupleAppender(ctx.getFrameSize());
+ group.add(new VSizeFrame(ctx));
+ fta = new FrameTupleAppender();
keyCounter = new GenericCounter();
valueCounter = new GenericCounter();
}
@@ -104,6 +106,7 @@
first = false;
} else {
if (i == 0) {
+ accessor1.reset(copyFrame.getBuffer());
switchGroupIfRequired(accessor1, accessor1.getTupleCount() - 1, accessor0, i);
} else {
switchGroupIfRequired(accessor0, i - 1, accessor0, i);
@@ -111,20 +114,21 @@
}
accumulate(accessor0, i);
}
- FrameUtils.copy(buffer, copyFrame);
+ copyFrame.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer());
}
private void accumulate(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
if (!fta.append(accessor, tIndex)) {
++bPtr;
if (group.size() <= bPtr) {
- group.add(ctx.allocateFrame());
+ group.add(new VSizeFrame(ctx));
}
fta.reset(group.get(bPtr), true);
if (!fta.append(accessor, tIndex)) {
throw new HyracksDataException("Record size ("
+ (accessor.getTupleEndOffset(tIndex) - accessor.getTupleStartOffset(tIndex))
- + ") larger than frame size (" + group.get(bPtr).capacity() + ")");
+ + ") larger than frame size (" + group.get(bPtr).getBuffer().capacity() + ")");
}
}
}
@@ -137,7 +141,7 @@
}
}
- private void groupInit() {
+ private void groupInit() throws HyracksDataException {
groupStarted = true;
bPtr = 0;
fta.reset(group.get(0), true);
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
index 295fe1f..6708e17 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/mapreduce/ShuffleFrameReader.java
@@ -22,8 +22,11 @@
import org.apache.hadoop.conf.Configuration;
-import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.NoShrinkVSizeFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -34,26 +37,29 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
public class ShuffleFrameReader implements IFrameReader {
private final IHyracksTaskContext ctx;
private final NonDeterministicChannelReader channelReader;
private final HadoopHelper helper;
private final RecordDescriptor recordDescriptor;
+ private final IFrame vframe;
private List<RunFileWriter> runFileWriters;
+ private List<Integer> runFileMaxFrameSize;
private RunFileReader reader;
public ShuffleFrameReader(IHyracksTaskContext ctx, NonDeterministicChannelReader channelReader,
MarshalledWritable<Configuration> mConfig) throws HyracksDataException {
this.ctx = ctx;
this.channelReader = channelReader;
- helper = new HadoopHelper(mConfig);
+ this.helper = new HadoopHelper(mConfig);
this.recordDescriptor = helper.getMapOutputRecordDescriptor();
+ this.vframe = new NoShrinkVSizeFrame(ctx);
}
@Override
@@ -61,21 +67,28 @@
channelReader.open();
int nSenders = channelReader.getSenderPartitionCount();
runFileWriters = new ArrayList<RunFileWriter>();
+ runFileMaxFrameSize = new ArrayList<>();
RunInfo[] infos = new RunInfo[nSenders];
- FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- IInputChannel[] channels = channelReader.getChannels();
+ FrameTupleAccessor accessor = new FrameTupleAccessor(recordDescriptor);
while (true) {
int entry = channelReader.findNextSender();
if (entry < 0) {
break;
}
RunInfo info = infos[entry];
- IInputChannel channel = channels[entry];
- ByteBuffer netBuffer = channel.getNextBuffer();
- accessor.reset(netBuffer);
+ ByteBuffer netBuffer = channelReader.getNextBuffer(entry);
+ netBuffer.clear();
+ int nBlocks = FrameHelper.deserializeNumOfMinFrame(netBuffer);
+
+ if (nBlocks > 1) {
+ netBuffer = getCompleteBuffer(nBlocks, netBuffer, entry);
+ }
+
+ accessor.reset(netBuffer, 0, netBuffer.limit());
int nTuples = accessor.getTupleCount();
for (int i = 0; i < nTuples; ++i) {
- int tBlockId = IntegerPointable.getInteger(accessor.getBuffer().array(), FrameUtils.getAbsoluteFieldStartOffset(accessor, i, HadoopHelper.BLOCKID_FIELD_INDEX));
+ int tBlockId = IntegerPointable.getInteger(accessor.getBuffer().array(),
+ accessor.getAbsoluteFieldStartOffset(i, HadoopHelper.BLOCKID_FIELD_INDEX));
if (info == null) {
info = new RunInfo();
info.reset(tBlockId);
@@ -86,7 +99,10 @@
}
info.write(accessor, i);
}
- channel.recycleBuffer(netBuffer);
+
+ if (nBlocks == 1) {
+ channelReader.recycleBuffer(entry, netBuffer);
+ }
}
for (int i = 0; i < infos.length; ++i) {
RunInfo info = infos[i];
@@ -94,7 +110,6 @@
info.close();
}
}
- infos = null;
FileReference outFile = ctx.createManagedWorkspaceFile(ShuffleFrameReader.class.getName() + ".run");
int framesLimit = helper.getSortFrameLimit(ctx);
@@ -103,22 +118,40 @@
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
- List<IFrameReader> runs = new LinkedList<IFrameReader>();
- for (RunFileWriter rfw : runFileWriters) {
- runs.add(rfw.createReader());
+ List<RunAndMaxFrameSizePair> runs = new LinkedList<>();
+ for (int i = 0; i < runFileWriters.size(); i++) {
+ runs.add(new RunAndMaxFrameSizePair(runFileWriters.get(i).createReader(), runFileMaxFrameSize.get(i)));
}
RunFileWriter rfw = new RunFileWriter(outFile, ctx.getIOManager());
- ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 }, comparators, null,
- recordDescriptor, framesLimit, rfw);
+ ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, null, runs, new int[] { 0 },
+ comparators, null, recordDescriptor, framesLimit, rfw);
merger.process();
reader = rfw.createReader();
reader.open();
}
+ private ByteBuffer getCompleteBuffer(int nBlocks, ByteBuffer netBuffer, int entry) throws HyracksDataException {
+ vframe.reset();
+ vframe.ensureFrameSize(vframe.getMinSize() * nBlocks);
+ FrameUtils.copyWholeFrame(netBuffer, vframe.getBuffer());
+ channelReader.recycleBuffer(entry, netBuffer);
+ for (int i = 1; i < nBlocks; ++i) {
+ netBuffer = channelReader.getNextBuffer(entry);
+ netBuffer.clear();
+ vframe.getBuffer().put(netBuffer);
+ channelReader.recycleBuffer(entry, netBuffer);
+ }
+ if (vframe.getBuffer().hasRemaining()) { // bigger frame
+ FrameHelper.clearRemainingFrame(vframe.getBuffer(), vframe.getBuffer().position());
+ }
+ vframe.getBuffer().flip();
+ return vframe.getBuffer();
+ }
+
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- return reader.nextFrame(buffer);
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
+ return reader.nextFrame(frame);
}
@Override
@@ -127,20 +160,22 @@
}
private class RunInfo {
- private final ByteBuffer buffer;
+ private final IFrame buffer;
private final FrameTupleAppender fta;
private FileReference file;
private RunFileWriter rfw;
private int blockId;
+ private int maxFrameSize = ctx.getInitialFrameSize();
public RunInfo() throws HyracksDataException {
- buffer = ctx.allocateFrame();
- fta = new FrameTupleAppender(ctx.getFrameSize());
+ buffer = new VSizeFrame(ctx);
+ fta = new FrameTupleAppender();
}
public void reset(int blockId) throws HyracksDataException {
this.blockId = blockId;
+ this.maxFrameSize = ctx.getInitialFrameSize();
fta.reset(buffer, true);
try {
file = ctx.createManagedWorkspaceFile(ShuffleFrameReader.class.getName() + ".run");
@@ -165,15 +200,15 @@
flush();
rfw.close();
runFileWriters.add(rfw);
+ runFileMaxFrameSize.add(maxFrameSize);
}
private void flush() throws HyracksDataException {
if (fta.getTupleCount() <= 0) {
return;
}
- buffer.limit(buffer.capacity());
- buffer.position(0);
- rfw.nextFrame(buffer);
+ maxFrameSize = buffer.getFrameSize() > maxFrameSize ? buffer.getFrameSize() : maxFrameSize;
+ rfw.nextFrame((ByteBuffer) buffer.getBuffer().clear());
fta.reset(buffer, true);
}
}
diff --git a/hyracks/hyracks-dataflow-std/pom.xml b/hyracks/hyracks-dataflow-std/pom.xml
index c4b23fd..5eadbca 100644
--- a/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks/hyracks-dataflow-std/pom.xml
@@ -66,5 +66,11 @@
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-control-nc</artifactId>
+ <version>0.2.16-SNAPSHOT</version>
+ <scope>test</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
index 607a817..12ff8e8 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/InputChannelFrameReader.java
@@ -18,6 +18,8 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -42,27 +44,58 @@
public void open() throws HyracksDataException {
}
- @Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- synchronized (this) {
- while (!failed && !eos && availableFrames <= 0) {
- try {
- wait();
- } catch (InterruptedException e) {
- throw new HyracksDataException(e);
- }
+ private synchronized boolean canGetNextBuffer() throws HyracksDataException {
+ while (!failed && !eos && availableFrames <= 0) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
}
- if (failed) {
- throw new HyracksDataException("Failure occurred on input");
- }
- if (availableFrames <= 0 && eos) {
- return false;
- }
- --availableFrames;
}
- ByteBuffer srcBuffer = channel.getNextBuffer();
- FrameUtils.copy(srcBuffer, buffer);
- channel.recycleBuffer(srcBuffer);
+ if (failed) {
+ throw new HyracksDataException("Failure occurred on input");
+ }
+ if (availableFrames <= 0 && eos) {
+ return false;
+ }
+ --availableFrames;
+ return true;
+ }
+
+ /**
+ * This implementation works under the truth that one Channel is never shared by two readers.
+ * More precisely, one channel only has exact one reader and one writer side.
+ *
+ * @param frame outputFrame
+ * @return {@code true} if succeed to read the data from the channel to the {@code frame}.
+ * Otherwise return {@code false} if the end of stream is reached.
+ * @throws HyracksDataException
+ */
+ @Override
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
+ if (!canGetNextBuffer()) {
+ return false;
+ }
+ frame.reset();
+ ByteBuffer srcFrame = channel.getNextBuffer();
+ int nBlocks = FrameHelper.deserializeNumOfMinFrame(srcFrame);
+ frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+ FrameUtils.copyWholeFrame(srcFrame, frame.getBuffer());
+ channel.recycleBuffer(srcFrame);
+
+ for (int i = 1; i < nBlocks; ++i) {
+ if (!canGetNextBuffer()) {
+ throw new HyracksDataException(
+ "InputChannelReader is waiting for the new frames, but the input stream is finished");
+ }
+ srcFrame = channel.getNextBuffer();
+ frame.getBuffer().put(srcFrame);
+ channel.recycleBuffer(srcFrame);
+ }
+ if (frame.getBuffer().hasRemaining()) { // bigger frame
+ FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
+ }
+ frame.getBuffer().flip();
return true;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
index 7f447c6..0c25d54 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicChannelReader.java
@@ -14,6 +14,7 @@
*/
package edu.uci.ics.hyracks.dataflow.std.collectors;
+import java.nio.ByteBuffer;
import java.util.BitSet;
import java.util.logging.Level;
import java.util.logging.Logger;
@@ -67,12 +68,27 @@
return nSenderPartitions;
}
- public void open() throws HyracksDataException {
- lastReadSender = -1;
+ public synchronized ByteBuffer getNextBuffer(int index) throws HyracksDataException {
+ while ((availableFrameCounts[index] <= 0)) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ if (--availableFrameCounts[index] == 0) {
+ frameAvailability.clear(index);
+ }
+ return channels[index].getNextBuffer();
+
}
- public IInputChannel[] getChannels() {
- return channels;
+ public void recycleBuffer(int index, ByteBuffer frame) {
+ channels[index].recycleBuffer(frame);
+ }
+
+ public void open() throws HyracksDataException {
+ lastReadSender = -1;
}
public synchronized int findNextSender() throws HyracksDataException {
@@ -83,9 +99,6 @@
}
if (lastReadSender >= 0) {
assert availableFrameCounts[lastReadSender] > 0;
- if (--availableFrameCounts[lastReadSender] == 0) {
- frameAvailability.clear(lastReadSender);
- }
return lastReadSender;
}
if (!failSenders.isEmpty()) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
index e107cfa..6dd6972 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/NonDeterministicFrameReader.java
@@ -16,7 +16,8 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
@@ -34,16 +35,27 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
int index = channelReader.findNextSender();
- if (index >= 0) {
- IInputChannel[] channels = channelReader.getChannels();
- ByteBuffer srcFrame = channels[index].getNextBuffer();
- FrameUtils.copy(srcFrame, buffer);
- channels[index].recycleBuffer(srcFrame);
- return true;
+ if (index < 0) {
+ return false;
}
- return false;
+ frame.reset();
+ ByteBuffer srcFrame = channelReader.getNextBuffer(index);
+ int nBlocks = FrameHelper.deserializeNumOfMinFrame(srcFrame);
+ frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+ FrameUtils.copyWholeFrame(srcFrame, frame.getBuffer());
+ channelReader.recycleBuffer(index, srcFrame);
+ for (int i = 1; i < nBlocks; ++i) {
+ srcFrame = channelReader.getNextBuffer(index);
+ frame.getBuffer().put(srcFrame);
+ channelReader.recycleBuffer(index, srcFrame);
+ }
+ if (frame.getBuffer().hasRemaining()) { // bigger frame
+ FrameHelper.clearRemainingFrame(frame.getBuffer(), frame.getBuffer().position());
+ }
+ frame.getBuffer().flip();
+ return true;
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
index 2dda9cc..125d07a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/collectors/SortMergeFrameReader.java
@@ -14,11 +14,12 @@
*/
package edu.uci.ics.hyracks.dataflow.std.collectors;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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.INormalizedKeyComputer;
@@ -54,13 +55,13 @@
@Override
public void open() throws HyracksDataException {
if (maxConcurrentMerges >= nSenders) {
- List<ByteBuffer> inFrames = new ArrayList<ByteBuffer>();
+ List<IFrame> inFrames = new ArrayList<>(nSenders);
for (int i = 0; i < nSenders; ++i) {
- inFrames.add(ByteBuffer.allocate(ctx.getFrameSize()));
+ inFrames.add(new VSizeFrame(ctx));
}
- List<IFrameReader> batch = new ArrayList<IFrameReader>();
+ List<IFrameReader> batch = new ArrayList<IFrameReader>(nSenders);
pbm.getNextBatch(batch, nSenders);
- merger = new RunMergingFrameReader(ctx, batch.toArray(new IFrameReader[nSenders]), inFrames, sortFields,
+ merger = new RunMergingFrameReader(ctx, batch, inFrames, sortFields,
comparators, nmkComputer, recordDescriptor);
} else {
// multi level merge.
@@ -70,10 +71,8 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- buffer.position(buffer.capacity());
- buffer.limit(buffer.capacity());
- return merger.nextFrame(buffer);
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
+ return merger.nextFrame(frame);
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
index dcca28e..edf4cc9 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
@@ -17,19 +17,22 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class LocalityAwarePartitionDataWriter implements IFrameWriter {
private final IFrameWriter[] pWriters;
- private final FrameTupleAppender[] appenders;
+ private final IFrameTupleAppender[] appenders;
private final FrameTupleAccessor tupleAccessor;
private final ITuplePartitionComputer tpc;
@@ -38,17 +41,17 @@
ILocalityMap localityMap, int senderIndex) throws HyracksDataException {
int[] consumerPartitions = localityMap.getConsumers(senderIndex, nConsumerPartitions);
pWriters = new IFrameWriter[consumerPartitions.length];
- appenders = new FrameTupleAppender[consumerPartitions.length];
+ appenders = new IFrameTupleAppender[consumerPartitions.length];
for (int i = 0; i < consumerPartitions.length; ++i) {
try {
pWriters[i] = pwFactory.createFrameWriter(consumerPartitions[i]);
- appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
- appenders[i].reset(ctx.allocateFrame(), true);
+ appenders[i] = new FrameTupleAppender();
+ appenders[i].reset(new VSizeFrame(ctx), true);
} catch (IOException e) {
throw new HyracksDataException(e);
}
}
- tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+ tupleAccessor = new FrameTupleAccessor(recordDescriptor);
this.tpc = tpc;
}
@@ -61,7 +64,6 @@
public void open() throws HyracksDataException {
for (int i = 0; i < pWriters.length; ++i) {
pWriters[i].open();
- appenders[i].reset(appenders[i].getBuffer(), true);
}
}
@@ -77,15 +79,7 @@
int tupleCount = tupleAccessor.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
int h = pWriters.length == 1 ? 0 : tpc.partition(tupleAccessor, i, pWriters.length);
- FrameTupleAppender appender = appenders[h];
- if (!appender.append(tupleAccessor, i)) {
- ByteBuffer appenderBuffer = appender.getBuffer();
- flushFrame(appenderBuffer, pWriters[h]);
- appender.reset(appenderBuffer, true);
- if (!appender.append(tupleAccessor, i)) {
- throw new HyracksDataException("Record size (" + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i)) + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
}
}
@@ -101,12 +95,6 @@
}
}
- private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
- frameWriter.nextFrame(buffer);
- }
-
/*
* (non-Javadoc)
*
@@ -115,9 +103,7 @@
@Override
public void close() throws HyracksDataException {
for (int i = 0; i < pWriters.length; ++i) {
- if (appenders[i].getTupleCount() > 0) {
- flushFrame(appenders[i].getBuffer(), pWriters[i]);
- }
+ appenders[i].flush(pWriters[i], true);
pWriters[i].close();
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index ea586fc..74f16d1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -19,12 +19,14 @@
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
public class PartitionDataWriter implements IFrameWriter {
private final int consumerPartitionCount;
@@ -33,7 +35,7 @@
private final FrameTupleAccessor tupleAccessor;
private final ITuplePartitionComputer tpc;
private final IHyracksTaskContext ctx;
- private boolean allocated = false;
+ private boolean allocatedFrame = false;
public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
@@ -43,12 +45,12 @@
for (int i = 0; i < consumerPartitionCount; ++i) {
try {
pWriters[i] = pwFactory.createFrameWriter(i);
- appenders[i] = new FrameTupleAppender(ctx.getFrameSize());
+ appenders[i] = new FrameTupleAppender();
} catch (IOException e) {
throw new HyracksDataException(e);
}
}
- tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
+ tupleAccessor = new FrameTupleAccessor(recordDescriptor);
this.tpc = tpc;
this.ctx = ctx;
}
@@ -56,21 +58,13 @@
@Override
public void close() throws HyracksDataException {
for (int i = 0; i < pWriters.length; ++i) {
- if (allocated) {
- if (appenders[i].getTupleCount() > 0) {
- flushFrame(appenders[i].getBuffer(), pWriters[i]);
- }
+ if (allocatedFrame) {
+ appenders[i].flush(pWriters[i], true);
}
pWriters[i].close();
}
}
- private void flushFrame(ByteBuffer buffer, IFrameWriter frameWriter) throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
- frameWriter.nextFrame(buffer);
- }
-
@Override
public void open() throws HyracksDataException {
for (int i = 0; i < pWriters.length; ++i) {
@@ -80,34 +74,22 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (!allocated) {
+ if (!allocatedFrame) {
allocateFrames();
- allocated = true;
+ allocatedFrame = true;
}
tupleAccessor.reset(buffer);
int tupleCount = tupleAccessor.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
int h = tpc.partition(tupleAccessor, i, consumerPartitionCount);
- FrameTupleAppender appender = appenders[h];
- if (!appender.append(tupleAccessor, i)) {
- ByteBuffer appenderBuffer = appender.getBuffer();
- flushFrame(appenderBuffer, pWriters[h]);
- appender.reset(appenderBuffer, true);
- if (!appender.append(tupleAccessor, i)) {
- throw new HyracksDataException("Record size ("
- + (tupleAccessor.getTupleEndOffset(i) - tupleAccessor.getTupleStartOffset(i))
- + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
+
}
}
- /**
- * @throws HyracksDataException
- */
private void allocateFrames() throws HyracksDataException {
for (int i = 0; i < appenders.length; ++i) {
- appenders[i].reset(ctx.allocateFrame(), true);
+ appenders[i].reset(new VSizeFrame(ctx), true);
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 5be1eab..765e223 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -18,11 +18,10 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
-import java.io.Reader;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -41,7 +40,8 @@
this(fieldParserFactories, fieldDelimiter, '\"');
}
- public DelimitedDataTupleParserFactory(IValueParserFactory[] fieldParserFactories, char fieldDelimiter, char quote) {
+ public DelimitedDataTupleParserFactory(IValueParserFactory[] fieldParserFactories, char fieldDelimiter,
+ char quote) {
this.valueParserFactories = fieldParserFactories;
this.fieldDelimiter = fieldDelimiter;
this.quote = quote;
@@ -57,8 +57,8 @@
for (int i = 0; i < valueParserFactories.length; ++i) {
valueParsers[i] = valueParserFactories[i].createValueParser();
}
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ IFrame frame = new VSizeFrame(ctx);
+ FrameTupleAppender appender = new FrameTupleAppender();
appender.reset(frame, true);
ArrayTupleBuilder tb = new ArrayTupleBuilder(valueParsers.length);
DataOutput dos = tb.getDataOutput();
@@ -80,18 +80,10 @@
valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
tb.addFieldEndOffset();
}
- 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 HyracksDataException("Record size (" + tb.getSize()
- + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
+ appender.flush(writer, true);
} catch (IOException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index 99f5a5f..7b2e8a0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -37,7 +37,7 @@
public class PlainFileWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
/**
- *
+ *
*/
private static final long serialVersionUID = 1L;
@@ -45,11 +45,6 @@
private String delim;
- /**
- * @param spec
- * @param inputArity
- * @param outputArity
- */
public PlainFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider,
String delim) {
super(spec, 1, 0);
@@ -74,7 +69,7 @@
// Output files
final FileSplit[] splits = fileSplitProvider.getFileSplits();
// Frame accessor
- final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
+ final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
// Record descriptor
final RecordDescriptor recordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index a4970ea..4d62fa0 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -14,11 +14,12 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -33,7 +34,6 @@
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;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
@@ -76,10 +76,8 @@
}
RecordDescriptor internalRecordDescriptor = outRecordDescriptor;
- final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(ctx.getFrameSize(),
- internalRecordDescriptor);
- final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(ctx.getFrameSize(),
- internalRecordDescriptor);
+ final FrameTupleAccessor storedKeysAccessor1 = new FrameTupleAccessor(internalRecordDescriptor);
+ final FrameTupleAccessor storedKeysAccessor2 = new FrameTupleAccessor(internalRecordDescriptor);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
@@ -118,14 +116,14 @@
private int lastBufIndex;
- private ByteBuffer outputFrame;
+ private IFrame outputFrame;
private FrameTupleAppender outputAppender;
- private FrameTupleAppender stateAppender = new FrameTupleAppender(ctx.getFrameSize());
+ private FrameTupleAppender stateAppender = new FrameTupleAppender();
private final ISerializableTable table = new SerializableHashTable(tableSize, ctx);
private final TuplePointer storedTuplePointer = new TuplePointer();
- private final List<ByteBuffer> frames = new ArrayList<ByteBuffer>();
+ private final List<IFrame> frames = new ArrayList<>();
/**
* A tuple is "pointed" to by 3 entries in the tPointers array. [0]
@@ -153,7 +151,7 @@
table.getTuplePointer(entry, offset, storedTuplePointer);
int fIndex = storedTuplePointer.frameIndex;
int tIndex = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(fIndex));
+ storedKeysAccessor1.reset(frames.get(fIndex).getBuffer());
int tStart = storedKeysAccessor1.getTupleStartOffset(tIndex);
int f0StartRel = storedKeysAccessor1.getFieldStartOffset(tIndex, sfIdx);
int f0EndRel = storedKeysAccessor1.getFieldEndOffset(tIndex, sfIdx);
@@ -191,7 +189,7 @@
table.getTuplePointer(entry, offset++, storedTuplePointer);
if (storedTuplePointer.frameIndex < 0)
break;
- storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex));
+ storedKeysAccessor1.reset(frames.get(storedTuplePointer.frameIndex).getBuffer());
int c = ftpcPartial.compare(accessor, tIndex, storedKeysAccessor1, storedTuplePointer.tupleIndex);
if (c == 0) {
foundGroup = true;
@@ -232,7 +230,7 @@
}
@Override
- public List<ByteBuffer> getFrames() {
+ public List<IFrame> getFrames() {
return frames;
}
@@ -244,11 +242,11 @@
@Override
public void flushFrames(IFrameWriter writer, boolean isPartial) throws HyracksDataException {
if (outputFrame == null) {
- outputFrame = ctx.allocateFrame();
+ outputFrame = new VSizeFrame(ctx);
}
if (outputAppender == null) {
- outputAppender = new FrameTupleAppender(outputFrame.capacity());
+ outputAppender = new FrameTupleAppender();
}
outputAppender.reset(outputFrame, true);
@@ -265,7 +263,7 @@
int bIndex = storedTuplePointer.frameIndex;
int tIndex = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(bIndex));
+ storedKeysAccessor1.reset(frames.get(bIndex).getBuffer());
outputTupleBuilder.reset();
for (int k = 0; k < storedKeys.length; k++) {
@@ -285,8 +283,7 @@
if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
- FrameUtils.flushFrame(outputFrame, writer);
- outputAppender.reset(outputFrame, true);
+ outputAppender.flush(writer, true);
if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
throw new HyracksDataException(
@@ -296,10 +293,7 @@
} while (true);
}
- if (outputAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outputFrame, writer);
- outputAppender.reset(outputFrame, true);
- }
+ outputAppender.flush(writer, true);
aggregator.close();
return;
}
@@ -311,8 +305,8 @@
int frameIndex = storedTuplePointer.frameIndex;
int tupleIndex = storedTuplePointer.tupleIndex;
// Get the frame containing the value
- ByteBuffer buffer = frames.get(frameIndex);
- storedKeysAccessor1.reset(buffer);
+ IFrame buffer = frames.get(frameIndex);
+ storedKeysAccessor1.reset(buffer.getBuffer());
outputTupleBuilder.reset();
for (int k = 0; k < storedKeys.length; k++) {
@@ -332,18 +326,14 @@
if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
- FrameUtils.flushFrame(outputFrame, writer);
- outputAppender.reset(outputFrame, true);
+ outputAppender.flush(writer, true);
if (!outputAppender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
throw new HyracksDataException("The output item is too large to be fit into a frame.");
}
}
}
- if (outputAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outputFrame, writer);
- outputAppender.reset(outputFrame, true);
- }
+ outputAppender.flush(writer, true);
aggregator.close();
}
@@ -372,19 +362,14 @@
if (frames.size() < framesLimit) {
// Insert a new frame
- ByteBuffer frame = ctx.allocateFrame();
- frame.position(0);
- frame.limit(frame.capacity());
+ IFrame frame = new VSizeFrame(ctx);
frames.add(frame);
stateAppender.reset(frame, true);
lastBufIndex = frames.size() - 1;
} else {
// Reuse an old frame
lastBufIndex++;
- ByteBuffer frame = frames.get(lastBufIndex);
- frame.position(0);
- frame.limit(frame.capacity());
- stateAppender.reset(frame, true);
+ stateAppender.reset(frames.get(lastBufIndex), true);
}
return true;
}
@@ -398,7 +383,7 @@
table.getTuplePointer(mTable, mRow, storedTuplePointer);
int mFrame = storedTuplePointer.frameIndex;
int mTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor1.reset(frames.get(mFrame));
+ storedKeysAccessor1.reset(frames.get(mFrame).getBuffer());
int a = offset;
int b = a;
@@ -416,7 +401,7 @@
table.getTuplePointer(bTable, bRow, storedTuplePointer);
int bFrame = storedTuplePointer.frameIndex;
int bTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor2.reset(frames.get(bFrame));
+ storedKeysAccessor2.reset(frames.get(bFrame).getBuffer());
cmp = ftpcTuple.compare(storedKeysAccessor2, bTuple, storedKeysAccessor1, mTuple);
}
if (cmp > 0) {
@@ -438,7 +423,7 @@
table.getTuplePointer(cTable, cRow, storedTuplePointer);
int cFrame = storedTuplePointer.frameIndex;
int cTuple = storedTuplePointer.tupleIndex;
- storedKeysAccessor2.reset(frames.get(cFrame));
+ storedKeysAccessor2.reset(frames.get(cFrame).getBuffer());
cmp = ftpcTuple.compare(storedKeysAccessor2, cTuple, storedKeysAccessor1, mTuple);
}
if (cmp < 0) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
index 6ac2a6d..b0c7e4d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/ISpillableTable.java
@@ -14,9 +14,9 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group;
-import java.nio.ByteBuffer;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
@@ -29,7 +29,7 @@
public int getFrameCount();
- public List<ByteBuffer> getFrames();
+ public List<IFrame> getFrames();
public void sortFrames() throws HyracksDataException;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
index 8e9e8b8..e683e47 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupBuildOperatorNodePushable.java
@@ -61,7 +61,7 @@
this.spillableTableFactory = spillableTableFactory;
this.inRecordDescriptor = inRecordDescriptor;
this.outRecordDescriptor = outRecordDescriptor;
- this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+ this.accessor = new FrameTupleAccessor(inRecordDescriptor);
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
index a55443c..9e3d4fc 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupMergeOperatorNodePushable.java
@@ -14,14 +14,15 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group.external;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Comparator;
import java.util.LinkedList;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -33,7 +34,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppenderAccessor;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -60,12 +61,12 @@
/**
* Input frames, one for each run file.
*/
- private List<ByteBuffer> inFrames;
+ private List<IFrame> inFrames;
/**
* Output frame.
*/
- private ByteBuffer outFrame, writerFrame;
- private final FrameTupleAppender outAppender;
+ private IFrame outFrame, writerFrame;
+ private final FrameTupleAppenderAccessor outAppender;
private FrameTupleAppender writerAppender;
private LinkedList<RunFileReader> runs;
private ExternalGroupState aggState;
@@ -76,7 +77,6 @@
private int runFrameLimit = 1;
private int[] currentFrameIndexInRun;
private int[] currentRunFrames;
- private final FrameTupleAccessor outFrameAccessor;
ExternalGroupMergeOperatorNodePushable(IHyracksTaskContext ctx, Object stateId,
IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nmkFactory, int[] keyFields,
@@ -108,8 +108,7 @@
tupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
this.ctx = ctx;
- outAppender = new FrameTupleAppender(ctx.getFrameSize());
- outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
+ outAppender = new FrameTupleAppenderAccessor(outRecordDescriptor);
this.isOutputSorted = isOutputSorted;
this.framesLimit = framesLimit;
this.outRecordDescriptor = outRecordDescriptor;
@@ -132,10 +131,9 @@
} else {
aggState = null;
runs = new LinkedList<RunFileReader>(runs);
- inFrames = new ArrayList<ByteBuffer>();
- outFrame = ctx.allocateFrame();
+ inFrames = new ArrayList<>();
+ outFrame = new VSizeFrame(ctx);
outAppender.reset(outFrame, true);
- outFrameAccessor.reset(outFrame);
while (runs.size() > 0) {
try {
doPass(runs);
@@ -160,7 +158,7 @@
boolean finalPass = false;
while (inFrames.size() + 2 < framesLimit) {
- inFrames.add(ctx.allocateFrame());
+ inFrames.add(new VSizeFrame(ctx));
}
int runNumber;
if (runs.size() + 2 <= framesLimit) {
@@ -184,8 +182,8 @@
RunFileReader[] runFileReaders = new RunFileReader[runNumber];
FrameTupleAccessor[] tupleAccessors = new FrameTupleAccessor[inFrames.size()];
Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
- ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), outRecordDescriptor,
- runNumber, comparator, keyFields, nmkComputer);
+ ReferencedPriorityQueue topTuples = new ReferencedPriorityQueue(runNumber, comparator, keyFields,
+ nmkComputer);
/**
* current tuple index in each run
*/
@@ -203,8 +201,8 @@
for (int j = 0; j < runFrameLimit; j++) {
int frameIndex = currentFrameIndexInRun[runIndex] + j;
if (runFileReaders[runIndex].nextFrame(inFrames.get(frameIndex))) {
- tupleAccessors[frameIndex] = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDescriptor);
- tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+ tupleAccessors[frameIndex] = new FrameTupleAccessor(outRecordDescriptor);
+ tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
currentRunFrames[runIndex]++;
if (j == 0)
setNextTopTuple(runIndex, tupleIndices, runFileReaders, tupleAccessors, topTuples);
@@ -224,11 +222,11 @@
ReferenceEntry top = topTuples.peek();
int tupleIndex = top.getTupleIndex();
int runIndex = topTuples.peek().getRunid();
- FrameTupleAccessor fta = top.getAccessor();
+ IFrameTupleAccessor fta = top.getAccessor();
- int currentTupleInOutFrame = outFrameAccessor.getTupleCount() - 1;
+ int currentTupleInOutFrame = outAppender.getTupleCount() - 1;
if (currentTupleInOutFrame < 0
- || compareFrameTuples(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame) != 0) {
+ || compareFrameTuples(fta, tupleIndex, outAppender, currentTupleInOutFrame) != 0) {
/**
* Initialize the first output record Reset the
* tuple builder
@@ -259,7 +257,7 @@
* outFrame
*/
- aggregator.aggregate(fta, tupleIndex, outFrameAccessor, currentTupleInOutFrame, aggregateState);
+ aggregator.aggregate(fta, tupleIndex, outAppender, currentTupleInOutFrame, aggregateState);
}
tupleIndices[runIndex]++;
@@ -295,49 +293,42 @@
}
if (writerFrame == null) {
- writerFrame = ctx.allocateFrame();
+ writerFrame = new VSizeFrame(ctx);
}
if (writerAppender == null) {
- writerAppender = new FrameTupleAppender(ctx.getFrameSize());
+ writerAppender = new FrameTupleAppender();
writerAppender.reset(writerFrame, true);
}
- outFrameAccessor.reset(outFrame);
-
- for (int i = 0; i < outFrameAccessor.getTupleCount(); i++) {
+ for (int i = 0; i < outAppender.getTupleCount(); i++) {
finalTupleBuilder.reset();
for (int k = 0; k < storedKeys.length; k++) {
- finalTupleBuilder.addField(outFrameAccessor, i, storedKeys[k]);
+ finalTupleBuilder.addField(outAppender, i, storedKeys[k]);
}
if (isFinal) {
- aggregator.outputFinalResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+ aggregator.outputFinalResult(finalTupleBuilder, outAppender, i, aggregateState);
} else {
- aggregator.outputPartialResult(finalTupleBuilder, outFrameAccessor, i, aggregateState);
+ aggregator.outputPartialResult(finalTupleBuilder, outAppender, i, aggregateState);
}
if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
- FrameUtils.flushFrame(writerFrame, writer);
- writerAppender.reset(writerFrame, true);
+ writerAppender.flush(writer, true);
if (!writerAppender.appendSkipEmptyField(finalTupleBuilder.getFieldEndOffsets(),
finalTupleBuilder.getByteArray(), 0, finalTupleBuilder.getSize())) {
throw new HyracksDataException("Aggregation output is too large to be fit into a frame.");
}
}
}
- if (writerAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(writerFrame, writer);
- writerAppender.reset(writerFrame, true);
- }
+ writerAppender.flush(writer, true);
- outAppender.reset(outFrame, true);
}
private void setNextTopTuple(int runIndex, int[] tupleIndices, RunFileReader[] runCursors,
@@ -377,7 +368,7 @@
for (int j = 0; j < runFrameLimit; j++) {
int frameIndex = currentFrameIndexInRun[runIndex] + j;
if (runCursors[runIndex].nextFrame(inFrames.get(frameIndex))) {
- tupleAccessors[frameIndex].reset(inFrames.get(frameIndex));
+ tupleAccessors[frameIndex].reset(inFrames.get(frameIndex).getBuffer());
existNext = true;
currentRunFrames[runIndex]++;
} else {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index 3c0eb2b..0102e65 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -19,7 +19,9 @@
import java.util.Arrays;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -32,6 +34,7 @@
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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.group.AggregateState;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
@@ -40,7 +43,7 @@
/**
* The pointers in the link store 3 int values for each entry in the
* hashtable: (bufferIdx, tIndex, accumulatorIdx).
- *
+ *
* @author vinayakb
*/
private static class Link {
@@ -67,7 +70,7 @@
private static final int INIT_AGG_STATE_SIZE = 8;
private final IHyracksTaskContext ctx;
- private final List<ByteBuffer> buffers;
+ private final List<IFrame> buffers;
private final Link[] table;
/**
* Aggregate states: a list of states for all groups maintained in the main
@@ -84,6 +87,7 @@
private final ITuplePartitionComputer tpc;
private final IAggregatorDescriptor aggregator;
+ private final IFrame outputFrame;
private final FrameTupleAppender appender;
private final FrameTupleAccessor storedKeysAccessor;
@@ -96,7 +100,7 @@
throws HyracksDataException {
this.ctx = ctx;
- buffers = new ArrayList<ByteBuffer>();
+ buffers = new ArrayList<>();
table = new Link[tableSize];
keys = fields;
@@ -127,10 +131,10 @@
accumulatorSize = 0;
RecordDescriptor storedKeysRecordDescriptor = new RecordDescriptor(storedKeySerDeser);
- storedKeysAccessor = new FrameTupleAccessor(ctx.getFrameSize(), storedKeysRecordDescriptor);
+ storedKeysAccessor = new FrameTupleAccessor(storedKeysRecordDescriptor);
lastBIndex = -1;
- appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender = new FrameTupleAppender();
addNewBuffer();
@@ -140,14 +144,13 @@
stateTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length + 1);
}
outputTupleBuilder = new ArrayTupleBuilder(outRecordDescriptor.getFields().length);
+ outputFrame = new VSizeFrame(ctx);
}
private void addNewBuffer() throws HyracksDataException {
- ByteBuffer buffer = ctx.allocateFrame();
- buffer.position(0);
- buffer.limit(buffer.capacity());
- buffers.add(buffer);
- appender.reset(buffer, true);
+ VSizeFrame frame = new VSizeFrame(ctx);
+ buffers.add(frame);
+ appender.reset(frame, true);
++lastBIndex;
}
@@ -161,7 +164,7 @@
for (int i = 0; i < link.size; i += 3) {
int sbIndex = link.pointers[i];
int stIndex = link.pointers[i + 1];
- storedKeysAccessor.reset(buffers.get(sbIndex));
+ storedKeysAccessor.reset(buffers.get(sbIndex).getBuffer());
int c = ftpc.compare(accessor, tIndex, storedKeysAccessor, stIndex);
if (c == 0) {
saIndex = link.pointers[i + 2];
@@ -206,8 +209,7 @@
}
void write(IFrameWriter writer) throws HyracksDataException {
- ByteBuffer buffer = ctx.allocateFrame();
- appender.reset(buffer, true);
+ appender.reset(outputFrame, true);
for (int i = 0; i < table.length; ++i) {
Link link = table[i];
@@ -216,7 +218,7 @@
int bIndex = link.pointers[j];
int tIndex = link.pointers[j + 1];
int aIndex = link.pointers[j + 2];
- ByteBuffer keyBuffer = buffers.get(bIndex);
+ ByteBuffer keyBuffer = buffers.get(bIndex).getBuffer();
storedKeysAccessor.reset(keyBuffer);
// copy keys
@@ -228,22 +230,13 @@
aggregator.outputFinalResult(outputTupleBuilder, storedKeysAccessor, tIndex,
aggregateStates[aIndex]);
- if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
- outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
- writer.nextFrame(buffer);
- appender.reset(buffer, true);
- if (!appender.appendSkipEmptyField(outputTupleBuilder.getFieldEndOffsets(),
- outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize())) {
- throw new HyracksDataException("Cannot write the aggregation output into a frame.");
- }
- }
+ FrameUtils.appendSkipEmptyFieldToWriter(writer, appender, outputTupleBuilder.getFieldEndOffsets(),
+ outputTupleBuilder.getByteArray(), 0, outputTupleBuilder.getSize());
}
}
}
- if (appender.getTupleCount() != 0) {
- writer.nextFrame(buffer);
- }
+ appender.flush(writer, true);
}
void close() throws HyracksDataException {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
index 8e49a9a..998d882 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupBuildOperatorNodePushable.java
@@ -44,7 +44,7 @@
IAggregatorDescriptorFactory aggregatorFactory, int tableSize, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor) {
this.ctx = ctx;
- this.accessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
+ this.accessor = new FrameTupleAccessor(inRecordDescriptor);
this.stateId = stateId;
this.keys = keys;
this.tpcf = tpcf;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
index 9ce70c1..4dbf03b 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorNodePushable.java
@@ -21,8 +21,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
@@ -52,12 +50,6 @@
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
- final ByteBuffer copyFrame = ctx.allocateFrame();
- final FrameTupleAccessor copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
- copyFrameAccessor.reset(copyFrame);
- ByteBuffer outFrame = ctx.allocateFrame();
- final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(outFrame, true);
pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory, inRecordDescriptor,
outRecordDescriptor, writer);
pgw.open();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
index 45f0488..559dec4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java
@@ -16,15 +16,17 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppenderWrapper;
+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.group.AggregateState;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptor;
@@ -35,7 +37,7 @@
private final IBinaryComparator[] comparators;
private final IAggregatorDescriptor aggregator;
private final AggregateState aggregateState;
- private final ByteBuffer copyFrame;
+ private final IFrame copyFrame;
private final FrameTupleAccessor inFrameAccessor;
private final FrameTupleAccessor copyFrameAccessor;
@@ -62,15 +64,15 @@
this.aggregator = aggregatorFactory.createAggregator(ctx, inRecordDesc, outRecordDesc, groupFields,
groupFields, writer);
this.aggregateState = aggregator.createAggregateStates();
- copyFrame = ctx.allocateFrame();
- inFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
- copyFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDesc);
- copyFrameAccessor.reset(copyFrame);
+ copyFrame = new VSizeFrame(ctx);
+ inFrameAccessor = new FrameTupleAccessor(inRecordDesc);
+ copyFrameAccessor = new FrameTupleAccessor(inRecordDesc);
+ copyFrameAccessor.reset(copyFrame.getBuffer());
- ByteBuffer outFrame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ VSizeFrame outFrame = new VSizeFrame(ctx);
+ FrameTupleAppender appender = new FrameTupleAppender();
appender.reset(outFrame, true);
- appenderWrapper = new FrameTupleAppenderWrapper(appender, outFrame, writer);
+ appenderWrapper = new FrameTupleAppenderWrapper(appender, writer);
tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length);
}
@@ -105,7 +107,9 @@
}
}
- FrameUtils.copy(buffer, copyFrame);
+ copyFrame.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer());
+ copyFrameAccessor.reset(copyFrame.getBuffer());
}
private void switchGroupIfRequired(FrameTupleAccessor prevTupleAccessor, int prevTupleIndex,
@@ -145,9 +149,9 @@
throws HyracksDataException {
for (int i = 0; i < comparators.length; ++i) {
int fIdx = groupFields[i];
- int s1 = a1.getTupleStartOffset(t1Idx) + a1.getFieldSlotsLength() + a1.getFieldStartOffset(t1Idx, fIdx);
+ int s1 = a1.getAbsoluteFieldStartOffset(t1Idx, fIdx);
int l1 = a1.getFieldLength(t1Idx, fIdx);
- int s2 = a2.getTupleStartOffset(t2Idx) + a2.getFieldSlotsLength() + a2.getFieldStartOffset(t2Idx, fIdx);
+ int s2 = a2.getAbsoluteFieldStartOffset(t2Idx, fIdx);
int l2 = a2.getFieldLength(t2Idx, fIdx);
if (comparators[i].compare(a1.getBuffer().array(), s1, l1, a2.getBuffer().array(), s2, l2) != 0) {
return false;
@@ -165,6 +169,7 @@
@Override
public void close() throws HyracksDataException {
if (!isFailed && !first) {
+ assert(copyFrameAccessor.getTupleCount() > 0);
writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1);
appenderWrapper.flush();
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
index e695828..d0a1146 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunGenerator.java
@@ -14,11 +14,6 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group.sort;
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -31,20 +26,15 @@
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterMergeSort;
-import edu.uci.ics.hyracks.dataflow.std.sort.FrameSorterQuickSort;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
/**
* Group-by aggregation is pushed before run file generation.
- *
+ *
* @author yingyib
*/
-public class ExternalSortGroupByRunGenerator implements IFrameWriter {
- private final IHyracksTaskContext ctx;
- private final IFrameSorter frameSorter;
- private final List<IFrameReader> runs;
- private final int maxSortFrames;
+public class ExternalSortGroupByRunGenerator extends ExternalSortRunGenerator {
private final int[] groupFields;
private final IBinaryComparatorFactory[] comparatorFactories;
@@ -52,86 +42,44 @@
private final RecordDescriptor inRecordDesc;
private final RecordDescriptor outRecordDesc;
- public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor recordDesc,
+ public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor inputRecordDesc,
int framesLimit, int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
RecordDescriptor outRecordDesc, Algorithm alg) throws HyracksDataException {
- this.ctx = ctx;
- if (alg == Algorithm.MERGE_SORT) {
- frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
- recordDesc);
- } else {
- frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
- recordDesc);
- }
- this.runs = new LinkedList<IFrameReader>();
- this.maxSortFrames = framesLimit - 1;
+ this(ctx, sortFields, inputRecordDesc, framesLimit, groupFields, firstKeyNormalizerFactory, comparatorFactories,
+ aggregatorFactory, outRecordDesc, alg, EnumFreeSlotPolicy.LAST_FIT);
+ }
+
+ public ExternalSortGroupByRunGenerator(IHyracksTaskContext ctx, int[] sortFields, RecordDescriptor inputRecordDesc,
+ int framesLimit, int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
+ IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
+ RecordDescriptor outRecordDesc, Algorithm alg, EnumFreeSlotPolicy policy) throws HyracksDataException {
+ super(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, inputRecordDesc, alg, policy,
+ framesLimit);
+
this.groupFields = groupFields;
this.comparatorFactories = comparatorFactories;
this.aggregatorFactory = aggregatorFactory;
- this.inRecordDesc = recordDesc;
+ this.inRecordDesc = inputRecordDesc;
this.outRecordDesc = outRecordDesc;
}
@Override
- public void open() throws HyracksDataException {
- runs.clear();
- frameSorter.reset();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (frameSorter.getFrameCount() >= maxSortFrames) {
- flushFramesToRun();
- }
- frameSorter.insertFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- if (frameSorter.getFrameCount() > 0) {
- if (runs.size() <= 0) {
- frameSorter.sortFrames();
- } else {
- flushFramesToRun();
- }
- }
- }
-
- private void flushFramesToRun() throws HyracksDataException {
- frameSorter.sortFrames();
+ protected RunFileWriter getRunFileWriter() throws HyracksDataException {
FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalSortGroupByRunGenerator.class.getSimpleName());
- RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
+ return new RunFileWriter(file, ctx.getIOManager());
+ }
+ @Override
+ protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
//create group-by comparators
IBinaryComparator[] comparators = new IBinaryComparator[Math
.min(groupFields.length, comparatorFactories.length)];
for (int i = 0; i < comparators.length; i++) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
- PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory,
+ return new PreclusteredGroupWriter(ctx, groupFields, comparators, aggregatorFactory,
this.inRecordDesc, this.outRecordDesc, writer, true);
- pgw.open();
-
- try {
- frameSorter.flushFrames(pgw);
- } finally {
- pgw.close();
- }
- frameSorter.reset();
- runs.add(writer.createReader());
- }
-
- @Override
- public void fail() throws HyracksDataException {
- }
-
- public IFrameSorter getFrameSorter() {
- return frameSorter;
- }
-
- public List<IFrameReader> getRuns() {
- return runs;
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
index 2a580d3..7de400d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/ExternalSortGroupByRunMerger.java
@@ -14,12 +14,8 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group.sort;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.LinkedList;
import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
@@ -27,39 +23,25 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupWriter;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
-import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.ISorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
/**
* Group-by aggregation is pushed into multi-pass merge of external sort.
- *
+ *
* @author yingyib
*/
-public class ExternalSortGroupByRunMerger {
+public class ExternalSortGroupByRunMerger extends ExternalSortRunMerger {
- private final IHyracksTaskContext ctx;
- private final List<IFrameReader> runs;
private final RecordDescriptor inputRecordDesc;
private final RecordDescriptor partialAggRecordDesc;
private final RecordDescriptor outRecordDesc;
- private final int framesLimit;
- private final IFrameWriter writer;
- private List<ByteBuffer> inFrames;
- private ByteBuffer outFrame;
- private FrameTupleAppender outFrameAppender;
-
- private final IFrameSorter frameSorter; // Used in External sort, no replacement
- // selection
private final int[] groupFields;
- private final INormalizedKeyComputer firstKeyNkc;
- private final IBinaryComparator[] comparators;
private final IAggregatorDescriptorFactory mergeAggregatorFactory;
private final IAggregatorDescriptorFactory partialAggregatorFactory;
private final boolean localSide;
@@ -68,25 +50,19 @@
private final int[] mergeGroupFields;
private final IBinaryComparator[] groupByComparators;
- // Constructor for external sort, no replacement selection
- public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+ public ExternalSortGroupByRunMerger(IHyracksTaskContext ctx, ISorter frameSorter, List<RunAndMaxFrameSizePair> runs,
int[] sortFields, RecordDescriptor inRecordDesc, RecordDescriptor partialAggRecordDesc,
RecordDescriptor outRecordDesc, int framesLimit, IFrameWriter writer, int[] groupFields,
INormalizedKeyComputer nmk, IBinaryComparator[] comparators,
IAggregatorDescriptorFactory partialAggregatorFactory, IAggregatorDescriptorFactory aggregatorFactory,
boolean localStage) {
- this.ctx = ctx;
- this.frameSorter = frameSorter;
- this.runs = new LinkedList<IFrameReader>(runs);
+ super(ctx, frameSorter, runs, sortFields, comparators, nmk, partialAggRecordDesc, framesLimit,
+ writer);
this.inputRecordDesc = inRecordDesc;
this.partialAggRecordDesc = partialAggRecordDesc;
this.outRecordDesc = outRecordDesc;
- this.framesLimit = framesLimit;
- this.writer = writer;
this.groupFields = groupFields;
- this.firstKeyNkc = nmk;
- this.comparators = comparators;
this.mergeAggregatorFactory = aggregatorFactory;
this.partialAggregatorFactory = partialAggregatorFactory;
this.localSide = localStage;
@@ -112,82 +88,38 @@
}
}
- public void process() throws HyracksDataException {
+ @Override
+ protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
IAggregatorDescriptorFactory aggregatorFactory = localSide ? partialAggregatorFactory : mergeAggregatorFactory;
- PreclusteredGroupWriter pgw = new PreclusteredGroupWriter(ctx, groupFields, groupByComparators,
- aggregatorFactory, inputRecordDesc, outRecordDesc, writer, false);
- try {
- if (runs.size() <= 0) {
- pgw.open();
- if (frameSorter != null && frameSorter.getFrameCount() > 0) {
- frameSorter.flushFrames(pgw);
- }
- /** recycle sort buffer */
- frameSorter.close();
- } else {
- /** recycle sort buffer */
- frameSorter.close();
-
- inFrames = new ArrayList<ByteBuffer>();
- outFrame = ctx.allocateFrame();
- outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
- outFrameAppender.reset(outFrame, true);
- for (int i = 0; i < framesLimit - 1; ++i) {
- inFrames.add(ctx.allocateFrame());
- }
- int maxMergeWidth = framesLimit - 1;
- while (runs.size() > maxMergeWidth) {
- int generationSeparator = 0;
- while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
- int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
- runs.size() - maxMergeWidth + 1);
- FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortGroupByRunMerger.class
- .getSimpleName());
- IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
-
- aggregatorFactory = localSide ? mergeAggregatorFactory : partialAggregatorFactory;
- pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregatorFactory,
- partialAggRecordDesc, partialAggRecordDesc, mergeResultWriter, true);
- pgw.open();
-
- IFrameReader[] runCursors = new RunFileReader[mergeWidth];
- for (int i = 0; i < mergeWidth; i++) {
- runCursors[i] = runs.get(generationSeparator + i);
- }
- merge(pgw, runCursors);
- pgw.close();
- runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
- runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
- }
- }
- if (!runs.isEmpty()) {
- pgw = new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators,
- mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, writer, false);
- pgw.open();
- IFrameReader[] runCursors = new RunFileReader[runs.size()];
- for (int i = 0; i < runCursors.length; i++) {
- runCursors[i] = runs.get(i);
- }
- merge(pgw, runCursors);
- }
- }
- } catch (Exception e) {
- pgw.fail();
- } finally {
- pgw.close();
- }
+ boolean outputPartial = false;
+ return new PreclusteredGroupWriter(ctx, groupFields, groupByComparators,
+ aggregatorFactory, inputRecordDesc, outRecordDesc, nextWriter, outputPartial);
}
- private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
- RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, mergeSortFields,
- comparators, firstKeyNkc, partialAggRecordDesc);
- merger.open();
- try {
- while (merger.nextFrame(outFrame)) {
- FrameUtils.flushFrame(outFrame, mergeResultWriter);
- }
- } finally {
- merger.close();
- }
+ @Override
+ protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException {
+ FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortGroupByRunMerger.class.getSimpleName());
+ return new RunFileWriter(newRun, ctx.getIOManager());
+ }
+
+ @Override
+ protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
+ throws HyracksDataException {
+ IAggregatorDescriptorFactory aggregatorFactory = localSide ? mergeAggregatorFactory : partialAggregatorFactory;
+ boolean outputPartial = true;
+ return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators, aggregatorFactory,
+ partialAggRecordDesc, partialAggRecordDesc, mergeFileWriter, outputPartial);
+ }
+
+ @Override
+ protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+ boolean outputPartial = false;
+ return new PreclusteredGroupWriter(ctx, mergeGroupFields, groupByComparators,
+ mergeAggregatorFactory, partialAggRecordDesc, outRecordDesc, nextWriter, outputPartial);
+ }
+
+ @Override
+ protected int[] getSortFields() {
+ return mergeSortFields;
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
index cee105b..95cfb09 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/sort/SortGroupByOperatorDescriptor.java
@@ -14,18 +14,11 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group.sort;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
@@ -34,34 +27,24 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
import edu.uci.ics.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
-import edu.uci.ics.hyracks.dataflow.std.sort.IFrameSorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunMerger;
+import edu.uci.ics.hyracks.dataflow.std.sort.ISorter;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
/**
* This Operator pushes group-by aggregation into the external sort.
* After the in-memory sort, it aggregates the sorted data before writing it to a run file.
* During the merge phase, it does an aggregation over sorted results.
- *
+ *
* @author yingyib
*/
-public class SortGroupByOperatorDescriptor extends AbstractOperatorDescriptor {
- private static final long serialVersionUID = 1L;
+public class SortGroupByOperatorDescriptor extends AbstractSorterOperatorDescriptor {
- private static final int SORT_ACTIVITY_ID = 0;
- private static final int MERGE_ACTIVITY_ID = 1;
-
- private final int framesLimit;
- private final int[] sortFields;
private final int[] groupFields;
- private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
- private final IBinaryComparatorFactory[] comparatorFactories;
private final IAggregatorDescriptorFactory mergeAggregatorFactory;
private final IAggregatorDescriptorFactory partialAggregatorFactory;
private final RecordDescriptor partialAggRecordDesc;
@@ -69,46 +52,31 @@
private final boolean finalStage;
private Algorithm alg = Algorithm.MERGE_SORT;
- /***
- * @param spec
- * , the Hyracks job specification
- * @param framesLimit
- * , the frame limit for this operator
- * @param sortFields
- * , the fields to sort
- * @param groupFields
- * , the fields to group, which can be a prefix subset of sortFields
- * @param firstKeyNormalizerFactory
- * , the normalized key computer factory of the first key
- * @param comparatorFactories
- * , the comparator factories of sort keys
- * @param partialAggregatorFactory
- * , for aggregating the input of this operator
- * @param mergeAggregatorFactory
- * , for aggregating the intermediate data of this operator
- * @param partialAggRecordDesc
- * , the record descriptor of intermediate data
- * @param outRecordDesc
- * , the record descriptor of output data
- * @param finalStage
- * , whether the operator is used for final stage aggregation
+ /**
+ * @param spec , the Hyracks job specification
+ * @param framesLimit , the frame limit for this operator
+ * @param sortFields , the fields to sort
+ * @param groupFields , the fields to group, which can be a prefix subset of sortFields
+ * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
+ * @param comparatorFactories , the comparator factories of sort keys
+ * @param partialAggregatorFactory , for aggregating the input of this operator
+ * @param mergeAggregatorFactory , for aggregating the intermediate data of this operator
+ * @param partialAggRecordDesc , the record descriptor of intermediate data
+ * @param outRecordDesc , the record descriptor of output data
+ * @param finalStage , whether the operator is used for final stage aggregation
*/
public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory partialAggregatorFactory,
IAggregatorDescriptorFactory mergeAggregatorFactory, RecordDescriptor partialAggRecordDesc,
RecordDescriptor outRecordDesc, boolean finalStage) {
- super(spec, 1, 1);
- this.framesLimit = framesLimit;
- this.sortFields = sortFields;
+
+ super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, outRecordDesc);
if (framesLimit <= 1) {
throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
}
- this.recordDescriptors[0] = outRecordDesc;
this.groupFields = groupFields;
- this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
- this.comparatorFactories = comparatorFactories;
this.mergeAggregatorFactory = mergeAggregatorFactory;
this.partialAggregatorFactory = partialAggregatorFactory;
this.partialAggRecordDesc = partialAggRecordDesc;
@@ -116,31 +84,19 @@
this.finalStage = finalStage;
}
- /***
- * @param spec
- * , the Hyracks job specification
- * @param framesLimit
- * , the frame limit for this operator
- * @param sortFields
- * , the fields to sort
- * @param groupFields
- * , the fields to group, which can be a prefix subset of sortFields
- * @param firstKeyNormalizerFactory
- * , the normalized key computer factory of the first key
- * @param comparatorFactories
- * , the comparator factories of sort keys
- * @param partialAggregatorFactory
- * , for aggregating the input of this operator
- * @param mergeAggregatorFactory
- * , for aggregating the intermediate data of this operator
- * @param partialAggRecordDesc
- * , the record descriptor of intermediate data
- * @param outRecordDesc
- * , the record descriptor of output data
- * @param finalStage
- * , whether the operator is used for final stage aggregation
- * @param alg
- * , the in-memory sort algorithm
+ /**
+ * @param spec , the Hyracks job specification
+ * @param framesLimit , the frame limit for this operator
+ * @param sortFields , the fields to sort
+ * @param groupFields , the fields to group, which can be a prefix subset of sortFields
+ * @param firstKeyNormalizerFactory , the normalized key computer factory of the first key
+ * @param comparatorFactories , the comparator factories of sort keys
+ * @param partialAggregatorFactory , for aggregating the input of this operator
+ * @param mergeAggregatorFactory , for aggregating the intermediate data of this operator
+ * @param partialAggRecordDesc , the record descriptor of intermediate data
+ * @param outRecordDesc , the record descriptor of output data
+ * @param finalStage , whether the operator is used for final stage aggregation
+ * @param alg , the in-memory sort algorithm
*/
public SortGroupByOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
int[] groupFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
@@ -153,122 +109,33 @@
}
@Override
- public void contributeActivities(IActivityGraphBuilder builder) {
- SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
- MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
- builder.addActivity(this, sa);
- builder.addSourceEdge(0, sa, 0);
-
- builder.addActivity(this, ma);
- builder.addTargetEdge(0, ma, 0);
-
- builder.addBlockingEdge(sa, ma);
+ public AbstractSorterOperatorDescriptor.SortActivity getSortActivity(ActivityId id) {
+ return new AbstractSorterOperatorDescriptor.SortActivity(id) {
+ @Override
+ protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescriptorProvider) throws HyracksDataException {
+ return new ExternalSortGroupByRunGenerator(ctx, sortFields,
+ recordDescriptorProvider.getInputRecordDescriptor(this.getActivityId(), 0), framesLimit,
+ groupFields, firstKeyNormalizerFactory, comparatorFactories, partialAggregatorFactory,
+ partialAggRecordDesc, alg);
+ }
+ };
}
- public static class SortTaskState extends AbstractStateObject {
- private List<IFrameReader> runs;
- private IFrameSorter frameSorter;
+ @Override
+ public AbstractSorterOperatorDescriptor.MergeActivity getMergeActivity(ActivityId id) {
+ return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
- public SortTaskState() {
- }
-
- private SortTaskState(JobId jobId, TaskId taskId) {
- super(jobId, taskId);
- }
-
- @Override
- public void toBytes(DataOutput out) throws IOException {
-
- }
-
- @Override
- public void fromBytes(DataInput in) throws IOException {
-
- }
- }
-
- private class SortActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public SortActivity(ActivityId id) {
- super(id);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
- private ExternalSortGroupByRunGenerator runGen;
-
- @Override
- public void open() throws HyracksDataException {
- runGen = new ExternalSortGroupByRunGenerator(ctx, sortFields,
- recordDescProvider.getInputRecordDescriptor(SortActivity.this.getActivityId(), 0),
- framesLimit, groupFields, firstKeyNormalizerFactory, comparatorFactories,
- partialAggregatorFactory, partialAggRecordDesc, alg);
- runGen.open();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- runGen.nextFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
- getActivityId(), partition));
- runGen.close();
- state.runs = runGen.getRuns();
- state.frameSorter = runGen.getFrameSorter();
- ctx.setStateObject(state);
- }
-
- @Override
- public void fail() throws HyracksDataException {
- runGen.fail();
- }
- };
- return op;
- }
- }
-
- private class MergeActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public MergeActivity(ActivityId id) {
- super(id);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
- @Override
- public void initialize() throws HyracksDataException {
- SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
- SORT_ACTIVITY_ID), partition));
- List<IFrameReader> runs = state.runs;
- IFrameSorter frameSorter = state.frameSorter;
- int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
-
- IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparators.length; i++) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
- .createNormalizedKeyComputer();
-
- ExternalSortGroupByRunMerger merger = new ExternalSortGroupByRunMerger(ctx, frameSorter, runs,
- sortFields, recordDescProvider.getInputRecordDescriptor(new ActivityId(odId,
- SORT_ACTIVITY_ID), 0), partialAggRecordDesc, outputRecordDesc, necessaryFrames,
- writer, groupFields, nkc, comparators, partialAggregatorFactory, mergeAggregatorFactory,
- !finalStage);
- merger.process();
- }
- };
- return op;
- }
+ @Override
+ protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+ List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+ INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+ return new ExternalSortGroupByRunMerger(ctx, sorter, runs, sortFields,
+ recordDescProvider.getInputRecordDescriptor(new ActivityId(odId, SORT_ACTIVITY_ID), 0),
+ partialAggRecordDesc, outputRecordDesc, necessaryFrames, writer, groupFields, nmkComputer,
+ comparators, partialAggregatorFactory, mergeAggregatorFactory, !finalStage);
+ }
+ };
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
index 8375521..9c7fcf4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java
@@ -16,6 +16,8 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -103,7 +105,7 @@
try {
- ByteBuffer buffer = ctx.allocateFrame();// input
+ IFrame buffer = new VSizeFrame(ctx);
// buffer
int tableSize = (int) (numPartitions * recordsPerFrame * factor);
ISerializableTable table = new SerializableHashTable(tableSize, ctx);
@@ -115,19 +117,19 @@
continue;
}
table.reset();
- InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
- ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1), hpcRep1,
- new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table, predEvaluator);
+ InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(rd0), hpcRep0,
+ new FrameTupleAccessor(rd1), hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators),
+ isLeftOuter, nullWriters1, table, predEvaluator);
// build
if (buildWriter != null) {
RunFileReader buildReader = buildWriter.createReader();
buildReader.open();
while (buildReader.nextFrame(buffer)) {
- ByteBuffer copyBuffer = ctx.allocateFrame();
- FrameUtils.copy(buffer, copyBuffer);
+ ByteBuffer copyBuffer = ctx.allocateFrame(buffer.getFrameSize());
+ FrameUtils.copyAndFlip(buffer.getBuffer(), copyBuffer);
joiner.build(copyBuffer);
- buffer.clear();
+ buffer.reset();
}
buildReader.close();
}
@@ -136,8 +138,8 @@
RunFileReader probeReader = probeWriter.createReader();
probeReader.open();
while (probeReader.nextFrame(buffer)) {
- joiner.join(buffer, writer);
- buffer.clear();
+ joiner.join(buffer.getBuffer(), writer);
+ buffer.reset();
}
probeReader.close();
joiner.closeJoin(writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
index c6dbe61..70f28da 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
@@ -16,6 +16,8 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -39,7 +41,7 @@
private final FrameTupleAccessor accessor0;
private final ITuplePartitionComputer hpc;
private final FrameTupleAppender appender;
- private ByteBuffer[] outbufs;
+ private IFrame[] outbufs;
private GraceHashJoinPartitionState state;
GraceHashJoinPartitionBuildOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] keys,
@@ -48,8 +50,8 @@
this.ctx = ctx;
this.stateId = stateId;
this.numPartitions = numPartitions;
- accessor0 = new FrameTupleAccessor(ctx.getFrameSize(), inRecordDescriptor);
- appender = new FrameTupleAppender(ctx.getFrameSize());
+ accessor0 = new FrameTupleAccessor(inRecordDescriptor);
+ appender = new FrameTupleAppender();
hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner();
comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
@@ -60,7 +62,7 @@
@Override
public void close() throws HyracksDataException {
for (int i = 0; i < numPartitions; i++) {
- ByteBuffer head = outbufs[i];
+ ByteBuffer head = outbufs[i].getBuffer();
accessor0.reset(head);
if (accessor0.getTupleCount() > 0) {
write(i, head);
@@ -97,13 +99,13 @@
for (int i = 0; i < tCount; ++i) {
int entry = hpc.partition(accessor0, i, numPartitions);
- ByteBuffer outbuf = outbufs[entry];
+ IFrame outbuf = outbufs[entry];
appender.reset(outbuf, false);
if (!appender.append(accessor0, i)) {
// buffer is full, ie. we cannot fit the tuple
// into the buffer -- write it to disk
- write(entry, outbuf);
- outbuf.clear();
+ write(entry, outbuf.getBuffer());
+ outbuf.reset();
appender.reset(outbuf, true);
if (!appender.append(accessor0, i)) {
throw new HyracksDataException("Item too big to fit in frame");
@@ -115,10 +117,10 @@
@Override
public void open() throws HyracksDataException {
state = new GraceHashJoinPartitionState(ctx.getJobletContext().getJobId(), stateId);
- outbufs = new ByteBuffer[numPartitions];
+ outbufs = new IFrame[numPartitions];
state.setRunWriters(new RunFileWriter[numPartitions]);
for (int i = 0; i < numPartitions; i++) {
- outbufs[i] = ctx.allocateFrame();
+ outbufs[i] = new VSizeFrame(ctx);
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 910edc7..0915ff9 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -19,6 +19,8 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -74,10 +76,8 @@
/**
* @param spec
- * @param memsize
- * in frames
- * @param inputsize0
- * in frames
+ * @param memsize in frames
+ * @param inputsize0 in frames
* @param recordsPerFrame
* @param factor
* @param keys0
@@ -201,21 +201,21 @@
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
.getJobId(), new TaskId(getActivityId(), partition));
- private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), rd1);
+ private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1);
private final ITuplePartitionComputer hpcBuild = new FieldHashPartitionComputerFactory(keys1,
hashFunctionFactories).createPartitioner();
- private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- private final FrameTupleAppender ftappender = new FrameTupleAppender(ctx.getFrameSize());
- private ByteBuffer[] bufferForPartitions;
- private final ByteBuffer inBuffer = ctx.allocateFrame();
+ private final FrameTupleAppender appender = new FrameTupleAppender();
+ private final FrameTupleAppender ftappender = new FrameTupleAppender();
+ private IFrame[] bufferForPartitions;
+ private final IFrame inBuffer = new VSizeFrame(ctx);
@Override
public void close() throws HyracksDataException {
if (state.memoryForHashtable != 0)
- build(inBuffer);
+ build(inBuffer.getBuffer());
for (int i = 0; i < state.nPartitions; i++) {
- ByteBuffer buf = bufferForPartitions[i];
+ ByteBuffer buf = bufferForPartitions[i].getBuffer();
accessorBuild.reset(buf);
if (accessorBuild.getTupleCount() > 0) {
write(i, buf);
@@ -233,18 +233,18 @@
accessorBuild.reset(buffer);
int tCount = accessorBuild.getTupleCount();
for (int i = 0; i < tCount; ++i) {
- int entry = -1;
+ int entry;
if (state.memoryForHashtable == 0) {
entry = hpcBuild.partition(accessorBuild, i, state.nPartitions);
boolean newBuffer = false;
- ByteBuffer bufBi = bufferForPartitions[entry];
+ IFrame bufBi = bufferForPartitions[entry];
while (true) {
appender.reset(bufBi, newBuffer);
if (appender.append(accessorBuild, i)) {
break;
} else {
- write(entry, bufBi);
- bufBi.clear();
+ write(entry, bufBi.getBuffer());
+ bufBi.reset();
newBuffer = true;
}
}
@@ -253,7 +253,7 @@
if (entry < state.memoryForHashtable) {
while (true) {
if (!ftappender.append(accessorBuild, i)) {
- build(inBuffer);
+ build(inBuffer.getBuffer());
ftappender.reset(inBuffer, true);
} else {
@@ -263,14 +263,14 @@
} else {
entry %= state.nPartitions;
boolean newBuffer = false;
- ByteBuffer bufBi = bufferForPartitions[entry];
+ IFrame bufBi = bufferForPartitions[entry];
while (true) {
appender.reset(bufBi, newBuffer);
if (appender.append(accessorBuild, i)) {
break;
} else {
- write(entry, bufBi);
- bufBi.clear();
+ write(entry, bufBi.getBuffer());
+ bufBi.reset();
newBuffer = true;
}
}
@@ -285,8 +285,8 @@
}
private void build(ByteBuffer inBuffer) throws HyracksDataException {
- ByteBuffer copyBuffer = ctx.allocateFrame();
- FrameUtils.copy(inBuffer, copyBuffer);
+ ByteBuffer copyBuffer = ctx.allocateFrame(inBuffer.capacity());
+ FrameUtils.copyAndFlip(inBuffer, copyBuffer);
state.joiner.build(copyBuffer);
}
@@ -321,13 +321,13 @@
int tableSize = (int) (state.memoryForHashtable * recordsPerFrame * factor);
ISerializableTable table = new SerializableHashTable(tableSize, ctx);
state.joiner = new InMemoryHashJoin(ctx, tableSize,
- new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
- ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
- comparators), isLeftOuter, nullWriters1, table, predEvaluator);
- bufferForPartitions = new ByteBuffer[state.nPartitions];
+ new FrameTupleAccessor(rd0), hpc0, new FrameTupleAccessor(rd1), hpc1,
+ new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1, table,
+ predEvaluator);
+ bufferForPartitions = new IFrame[state.nPartitions];
state.fWriters = new RunFileWriter[state.nPartitions];
for (int i = 0; i < state.nPartitions; i++) {
- bufferForPartitions[i] = ctx.allocateFrame();
+ bufferForPartitions[i] = new VSizeFrame(ctx);
}
ftappender.reset(inBuffer, true);
@@ -391,20 +391,20 @@
IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private BuildAndPartitionTaskState state;
- private final FrameTupleAccessor accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), rd0);
+ private final FrameTupleAccessor accessorProbe = new FrameTupleAccessor(rd0);
private final ITuplePartitionComputerFactory hpcf0 = new FieldHashPartitionComputerFactory(keys0,
hashFunctionFactories);
private final ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1,
hashFunctionFactories);
private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner();
- private final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- private final FrameTupleAppender ftap = new FrameTupleAppender(ctx.getFrameSize());
- private final ByteBuffer inBuffer = ctx.allocateFrame();
- private final ByteBuffer outBuffer = ctx.allocateFrame();
+ private final FrameTupleAppender appender = new FrameTupleAppender();
+ private final FrameTupleAppender ftap = new FrameTupleAppender();
+ private final IFrame inBuffer = new VSizeFrame(ctx);
+ private final IFrame outBuffer = new VSizeFrame(ctx);
private RunFileWriter[] buildWriters;
private RunFileWriter[] probeWriters;
- private ByteBuffer[] bufferForPartitions;
+ private IFrame[] bufferForPartitions;
@Override
public void open() throws HyracksDataException {
@@ -413,9 +413,9 @@
writer.open();
buildWriters = state.fWriters;
probeWriters = new RunFileWriter[state.nPartitions];
- bufferForPartitions = new ByteBuffer[state.nPartitions];
+ bufferForPartitions = new IFrame[state.nPartitions];
for (int i = 0; i < state.nPartitions; i++) {
- bufferForPartitions[i] = ctx.allocateFrame();
+ bufferForPartitions[i] = new VSizeFrame(ctx);
}
appender.reset(outBuffer, true);
ftap.reset(inBuffer, true);
@@ -428,18 +428,18 @@
int tupleCount0 = accessorProbe.getTupleCount();
for (int i = 0; i < tupleCount0; ++i) {
- int entry = -1;
+ int entry ;
if (state.memoryForHashtable == 0) {
entry = hpcProbe.partition(accessorProbe, i, state.nPartitions);
boolean newBuffer = false;
- ByteBuffer outbuf = bufferForPartitions[entry];
+ IFrame outbuf = bufferForPartitions[entry];
while (true) {
appender.reset(outbuf, newBuffer);
if (appender.append(accessorProbe, i)) {
break;
} else {
- write(entry, outbuf);
- outbuf.clear();
+ write(entry, outbuf.getBuffer());
+ outbuf.reset();
newBuffer = true;
}
}
@@ -448,7 +448,7 @@
if (entry < state.memoryForHashtable) {
while (true) {
if (!ftap.append(accessorProbe, i)) {
- state.joiner.join(inBuffer, writer);
+ state.joiner.join(inBuffer.getBuffer(), writer);
ftap.reset(inBuffer, true);
} else
break;
@@ -457,14 +457,14 @@
} else {
entry %= state.nPartitions;
boolean newBuffer = false;
- ByteBuffer outbuf = bufferForPartitions[entry];
+ IFrame outbuf = bufferForPartitions[entry];
while (true) {
appender.reset(outbuf, newBuffer);
if (appender.append(accessorProbe, i)) {
break;
} else {
- write(entry, outbuf);
- outbuf.clear();
+ write(entry, outbuf.getBuffer());
+ outbuf.reset();
newBuffer = true;
}
}
@@ -478,7 +478,7 @@
@Override
public void close() throws HyracksDataException {
- state.joiner.join(inBuffer, writer);
+ state.joiner.join(inBuffer.getBuffer(), writer);
state.joiner.closeJoin(writer);
ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(state.nPartitions, hpcf0)
.createPartitioner();
@@ -486,7 +486,7 @@
.createPartitioner();
if (state.memoryForHashtable != memsize - 2) {
for (int i = 0; i < state.nPartitions; i++) {
- ByteBuffer buf = bufferForPartitions[i];
+ ByteBuffer buf = bufferForPartitions[i].getBuffer();
accessorProbe.reset(buf);
if (accessorProbe.getTupleCount() > 0) {
write(i, buf);
@@ -494,7 +494,7 @@
closeWriter(i);
}
- inBuffer.clear();
+ inBuffer.reset();
int tableSize = -1;
if (state.memoryForHashtable == 0) {
tableSize = (int) (state.nPartitions * recordsPerFrame * factor);
@@ -510,18 +510,18 @@
}
table.reset();
InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(
- ctx.getFrameSize(), rd0), hpcRep0, new FrameTupleAccessor(ctx.getFrameSize(), rd1),
- hpcRep1, new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter,
- nullWriters1, table, predEvaluator);
+ rd0), hpcRep0, new FrameTupleAccessor(rd1), hpcRep1,
+ new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1,
+ table, predEvaluator);
if (buildWriter != null) {
RunFileReader buildReader = buildWriter.createReader();
buildReader.open();
while (buildReader.nextFrame(inBuffer)) {
- ByteBuffer copyBuffer = ctx.allocateFrame();
- FrameUtils.copy(inBuffer, copyBuffer);
+ ByteBuffer copyBuffer = ctx.allocateFrame(inBuffer.getFrameSize());
+ FrameUtils.copyAndFlip(inBuffer.getBuffer(), copyBuffer);
joiner.build(copyBuffer);
- inBuffer.clear();
+ inBuffer.reset();
}
buildReader.close();
}
@@ -530,8 +530,8 @@
RunFileReader probeReader = probeWriter.createReader();
probeReader.open();
while (probeReader.nextFrame(inBuffer)) {
- joiner.join(inBuffer, writer);
- inBuffer.clear();
+ joiner.join(inBuffer.getBuffer(), writer);
+ inBuffer.reset();
}
probeReader.close();
joiner.closeJoin(writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 860cdd4..d6a83d2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -21,6 +21,7 @@
import java.util.logging.Logger;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
@@ -30,6 +31,7 @@
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;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.std.structures.ISerializableTable;
import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
@@ -43,7 +45,6 @@
private final ITuplePartitionComputer tpcProbe;
private final FrameTupleAppender appender;
private final FrameTuplePairComparator tpComparator;
- private final ByteBuffer outBuffer;
private final boolean isLeftOuter;
private final ArrayTupleBuilder nullTupleBuild;
private final ISerializableTable table;
@@ -75,10 +76,8 @@
this.tpcBuild = tpc1;
this.accessorProbe = accessor0;
this.tpcProbe = tpc0;
- appender = new FrameTupleAppender(ctx.getFrameSize());
+ appender = new FrameTupleAppender(new VSizeFrame(ctx));
tpComparator = comparator;
- outBuffer = ctx.allocateFrame();
- appender.reset(outBuffer, true);
predEvaluator = predEval;
this.isLeftOuter = isLeftOuter;
if (isLeftOuter) {
@@ -136,24 +135,15 @@
} while (true);
}
if (!matchFound && isLeftOuter) {
- if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
- nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
- nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
- throw new HyracksDataException("Record size larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, i,
+ nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
+ nullTupleBuild.getSize());
}
}
}
public void closeJoin(IFrameWriter writer) throws HyracksDataException {
- if (appender.getTupleCount() > 0) {
- flushFrame(outBuffer, writer);
- }
+ appender.flush(writer, true);
int nFrames = buffers.size();
buffers.clear();
ctx.deallocateFrames(nFrames);
@@ -179,31 +169,11 @@
private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
if (!reverseOutputOrder) {
- if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessorProbe, probeSidetIx, accessorBuild, buildSidetIx)) {
- int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
- - accessorProbe.getTupleStartOffset(probeSidetIx)
- + accessorBuild.getTupleEndOffset(buildSidetIx)
- - accessorBuild.getTupleStartOffset(buildSidetIx);
- throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild,
+ buildSidetIx);
} else {
- if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessorBuild, buildSidetIx, accessorProbe, probeSidetIx)) {
- int tSize = accessorProbe.getTupleEndOffset(probeSidetIx)
- - accessorProbe.getTupleStartOffset(probeSidetIx)
- + accessorBuild.getTupleEndOffset(buildSidetIx)
- - accessorBuild.getTupleStartOffset(buildSidetIx);
- throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe,
+ probeSidetIx);
}
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index be369d2..35e22ad 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -77,7 +77,8 @@
public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
- IPredicateEvaluatorFactory predEvalFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
+ IPredicateEvaluatorFactory predEvalFactory, RecordDescriptor recordDescriptor, boolean isLeftOuter,
+ INullWriterFactory[] nullWriterFactories1,
int tableSize) {
super(spec, 2, 1);
this.keys0 = keys0;
@@ -90,21 +91,20 @@
this.nullWriterFactories1 = nullWriterFactories1;
this.tableSize = tableSize;
}
-
+
public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor, int tableSize) {
- this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, recordDescriptor, tableSize, null);
+ this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, recordDescriptor, tableSize, null);
}
-
+
public InMemoryHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys0, int[] keys1,
IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1,
int tableSize) {
- this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories,null,recordDescriptor,isLeftOuter,nullWriterFactories1,tableSize);
+ this(spec, keys0, keys1, hashFunctionFactories, comparatorFactories, null, recordDescriptor, isLeftOuter,
+ nullWriterFactories1, tableSize);
}
-
-
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
@@ -170,7 +170,9 @@
nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
}
}
- final IPredicateEvaluator predEvaluator = ( predEvaluatorFactory == null ? null : predEvaluatorFactory.createPredicateEvaluator());
+ final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ?
+ null :
+ predEvaluatorFactory.createPredicateEvaluator());
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
private HashBuildTaskState state;
@@ -185,15 +187,15 @@
partition));
ISerializableTable table = new SerializableHashTable(tableSize, ctx);
state.joiner = new InMemoryHashJoin(ctx, tableSize,
- new FrameTupleAccessor(ctx.getFrameSize(), rd0), hpc0, new FrameTupleAccessor(
- ctx.getFrameSize(), rd1), hpc1, new FrameTuplePairComparator(keys0, keys1,
+ new FrameTupleAccessor(rd0), hpc0, new FrameTupleAccessor(rd1), hpc1,
+ new FrameTuplePairComparator(keys0, keys1,
comparators), isLeftOuter, nullWriters1, table, predEvaluator);
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- ByteBuffer copyBuffer = ctx.allocateFrame();
- FrameUtils.copy(buffer, copyBuffer);
+ ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyBuffer);
state.joiner.build(copyBuffer);
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
index eab60bc..3bd0540 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -19,7 +19,9 @@
import java.util.ArrayList;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.IPredicateEvaluator;
@@ -38,8 +40,8 @@
private final FrameTupleAccessor accessorOuter;
private final FrameTupleAppender appender;
private final ITuplePairComparator tpComparator;
- private final ByteBuffer outBuffer;
- private final ByteBuffer innerBuffer;
+ private final IFrame outBuffer;
+ private final IFrame innerBuffer;
private final List<ByteBuffer> outBuffers;
private final int memSize;
private final IHyracksTaskContext ctx;
@@ -49,18 +51,18 @@
private final boolean isLeftOuter;
private final ArrayTupleBuilder nullTupleBuilder;
private final IPredicateEvaluator predEvaluator;
- private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
+ private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
-
public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessor0, FrameTupleAccessor accessor1,
- ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter, INullWriter[] nullWriters1)
+ ITuplePairComparator comparators, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
+ INullWriter[] nullWriters1)
throws HyracksDataException {
this.accessorInner = accessor1;
this.accessorOuter = accessor0;
- this.appender = new FrameTupleAppender(ctx.getFrameSize());
+ this.appender = new FrameTupleAppender();
this.tpComparator = comparators;
- this.outBuffer = ctx.allocateFrame();
- this.innerBuffer = ctx.allocateFrame();
+ this.outBuffer = new VSizeFrame(ctx);
+ this.innerBuffer = new VSizeFrame(ctx);
this.appender.reset(outBuffer, true);
this.outBuffers = new ArrayList<ByteBuffer>();
this.memSize = memSize;
@@ -107,7 +109,7 @@
runFileReader.open();
while (runFileReader.nextFrame(innerBuffer)) {
for (ByteBuffer outBuffer : outBuffers) {
- blockJoin(outBuffer, innerBuffer, writer);
+ blockJoin(outBuffer, innerBuffer.getBuffer(), writer);
}
}
runFileReader.close();
@@ -116,15 +118,18 @@
}
private void createAndCopyFrame(ByteBuffer outerBuffer) throws HyracksDataException {
- ByteBuffer outerBufferCopy = ctx.allocateFrame();
- FrameUtils.copy(outerBuffer, outerBufferCopy);
+ ByteBuffer outerBufferCopy = ctx.allocateFrame(outerBuffer.capacity());
+ FrameUtils.copyAndFlip(outerBuffer, outerBufferCopy);
outBuffers.add(outerBufferCopy);
currentMemSize++;
}
- private void reloadFrame(ByteBuffer outerBuffer) {
+ private void reloadFrame(ByteBuffer outerBuffer) throws HyracksDataException {
outBuffers.get(currentMemSize).clear();
- FrameUtils.copy(outerBuffer, outBuffers.get(currentMemSize));
+ if (outBuffers.get(currentMemSize).capacity() != outerBuffer.capacity()) {
+ outBuffers.set(currentMemSize, ctx.allocateFrame(outerBuffer.capacity()));
+ }
+ FrameUtils.copyAndFlip(outerBuffer, outBuffers.get(currentMemSize));
currentMemSize++;
}
@@ -141,8 +146,8 @@
int c = compare(accessorOuter, i, accessorInner, j);
boolean prdEval = evaluatePredicate(i, j);
if (c == 0 && prdEval) {
- matchFound = true;
- appendToResults(i, j, writer);
+ matchFound = true;
+ appendToResults(i, j, writer);
}
}
@@ -150,28 +155,20 @@
final int[] ntFieldEndOffsets = nullTupleBuilder.getFieldEndOffsets();
final byte[] ntByteArray = nullTupleBuilder.getByteArray();
final int ntSize = nullTupleBuilder.getSize();
- if (!appender.appendConcat(accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0, ntSize)) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0, ntSize)) {
- int tSize = accessorOuter.getTupleEndOffset(i) - accessorOuter.getTupleStartOffset(i) + ntSize;
- throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0,
+ ntSize);
}
}
}
-
- private boolean evaluatePredicate(int tIx1, int tIx2){
- if(isReversed){ //Role Reversal Optimization is triggered
- return ( (predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1) );
- }
- else {
- return ( (predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2) );
- }
+
+ private boolean evaluatePredicate(int tIx1, int tIx2) {
+ if (isReversed) { //Role Reversal Optimization is triggered
+ return ((predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1));
+ } else {
+ return ((predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2));
+ }
}
-
+
private void appendToResults(int outerTupleId, int innerTupleId, IFrameWriter writer) throws HyracksDataException {
if (!isReversed) {
appendResultToFrame(accessorOuter, outerTupleId, accessorInner, innerTupleId, writer);
@@ -183,18 +180,9 @@
private void appendResultToFrame(FrameTupleAccessor accessor1, int tupleId1, FrameTupleAccessor accessor2,
int tupleId2, IFrameWriter writer) throws HyracksDataException {
- if (!appender.appendConcat(accessor1, tupleId1, accessor2, tupleId2)) {
- flushFrame(outBuffer, writer);
- appender.reset(outBuffer, true);
- if (!appender.appendConcat(accessor1, tupleId1, accessor2, tupleId2)) {
- int tSize = accessor1.getTupleEndOffset(tupleId1) - accessor1.getTupleStartOffset(tupleId1)
- + accessor2.getTupleEndOffset(tupleId2) - accessor2.getTupleStartOffset(tupleId2);
- throw new HyracksDataException("Record size (" + tSize + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessor1, tupleId1, accessor2, tupleId2);
}
-
+
public void closeCache() throws HyracksDataException {
if (runFileWriter != null) {
runFileWriter.close();
@@ -206,24 +194,14 @@
runFileReader.open();
while (runFileReader.nextFrame(innerBuffer)) {
for (int i = 0; i < currentMemSize; i++) {
- blockJoin(outBuffers.get(i), innerBuffer, writer);
+ blockJoin(outBuffers.get(i), innerBuffer.getBuffer(), writer);
}
}
runFileReader.close();
outBuffers.clear();
currentMemSize = 0;
- if (appender.getTupleCount() > 0) {
- flushFrame(outBuffer, writer);
- }
- }
-
- private void flushFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
- buffer.position(0);
- buffer.limit(buffer.capacity());
- writer.nextFrame(buffer);
- buffer.position(0);
- buffer.limit(buffer.capacity());
+ appender.flush(writer, true);
}
private int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
@@ -234,8 +212,8 @@
}
return 0;
}
-
- public void setIsReversed(boolean b){
- this.isReversed = b;
+
+ public void setIsReversed(boolean b) {
+ this.isReversed = b;
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 41f0f4f..3e06bf3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -54,16 +54,17 @@
private final IPredicateEvaluatorFactory predEvaluatorFactory;
private final boolean isLeftOuter;
private final INullWriterFactory[] nullWriterFactories1;
-
+
public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
this(spec, comparatorFactory, recordDescriptor, memSize, null, isLeftOuter, nullWriterFactories1);
}
-
+
public NestedLoopJoinOperatorDescriptor(IOperatorDescriptorRegistry spec,
ITuplePairComparatorFactory comparatorFactory, RecordDescriptor recordDescriptor, int memSize,
- IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
+ IPredicateEvaluatorFactory predEvalFactory, boolean isLeftOuter,
+ INullWriterFactory[] nullWriterFactories1) {
super(spec, 2, 1);
this.comparatorFactory = comparatorFactory;
this.recordDescriptors[0] = recordDescriptor;
@@ -127,8 +128,10 @@
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
- final IPredicateEvaluator predEvaluator = ( (predEvaluatorFactory != null) ? predEvaluatorFactory.createPredicateEvaluator() : null);
-
+ final IPredicateEvaluator predEvaluator = ((predEvaluatorFactory != null) ?
+ predEvaluatorFactory.createPredicateEvaluator() :
+ null);
+
final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories1.length] : null;
if (isLeftOuter) {
for (int i = 0; i < nullWriterFactories1.length; i++) {
@@ -144,17 +147,16 @@
state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(),
partition));
- state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), rd0),
- new FrameTupleAccessor(ctx.getFrameSize(), rd1), comparator, memSize, predEvaluator, isLeftOuter,
+ state.joiner = new NestedLoopJoin(ctx, new FrameTupleAccessor(rd0),
+ new FrameTupleAccessor(rd1), comparator, memSize, predEvaluator, isLeftOuter,
nullWriters1);
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- ByteBuffer copyBuffer = ctx.allocateFrame();
- FrameUtils.copy(buffer, copyBuffer);
- FrameUtils.makeReadable(copyBuffer);
+ ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyBuffer);
state.joiner.cache(copyBuffer);
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 506da2e..ed25b4f 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -19,7 +19,9 @@
import java.util.BitSet;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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.INullWriter;
@@ -74,7 +76,7 @@
private final boolean isLeftOuter;
private final INullWriter[] nullWriters1;
- private ByteBuffer[] memBuffs; //Memory buffers for build
+ private IFrame[] memBuffs; //Memory buffers for build
private int[] curPBuff; //Current (last) Buffer for each partition
private int[] nextBuff; //Next buffer in the partition's buffer chain
private int[] buildPSizeInTups; //Size of build partitions (in tuples)
@@ -92,9 +94,9 @@
private FrameTupleAppender probeTupAppenderToSpilled;
private int numOfSpilledParts;
- private ByteBuffer[] sPartBuffs; //Buffers for probe spilled partitions (one buffer per spilled partition)
- private ByteBuffer probeResBuff; //Buffer for probe resident partition tuples
- private ByteBuffer reloadBuffer; //Buffer for reloading spilled partitions during partition tuning
+ private IFrame[] sPartBuffs; //Buffers for probe spilled partitions (one buffer per spilled partition)
+ private IFrame probeResBuff; //Buffer for probe resident partition tuples
+ private IFrame reloadBuffer; //Buffer for reloading spilled partitions during partition tuning
private int[] buildPSizeInFrames; //Used for partition tuning
private int freeFramesCounter; //Used for partition tuning
@@ -124,8 +126,8 @@
this.buildRFWriters = new RunFileWriter[numOfPartitions];
this.probeRFWriters = new RunFileWriter[numOfPartitions];
- this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
- this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+ this.accessorBuild = new FrameTupleAccessor(buildRd);
+ this.accessorProbe = new FrameTupleAccessor(probeRd);
this.predEvaluator = predEval;
this.isLeftOuter = false;
@@ -154,8 +156,8 @@
this.buildRFWriters = new RunFileWriter[numOfPartitions];
this.probeRFWriters = new RunFileWriter[numOfPartitions];
- this.accessorBuild = new FrameTupleAccessor(ctx.getFrameSize(), buildRd);
- this.accessorProbe = new FrameTupleAccessor(ctx.getFrameSize(), probeRd);
+ this.accessorBuild = new FrameTupleAccessor(buildRd);
+ this.accessorProbe = new FrameTupleAccessor(probeRd);
this.predEvaluator = predEval;
this.isLeftOuter = isLeftOuter;
@@ -170,7 +172,7 @@
}
public void initBuild() throws HyracksDataException {
- memBuffs = new ByteBuffer[memForJoin];
+ memBuffs = new IFrame[memForJoin];
curPBuff = new int[numOfPartitions];
nextBuff = new int[memForJoin];
pStatus = new BitSet(numOfPartitions);
@@ -179,19 +181,22 @@
buildPSizeInFrames = new int[numOfPartitions];
freeFramesCounter = memForJoin - numOfPartitions;
- for (int i = 0; i < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
- memBuffs[i] = ctx.allocateFrame();
+ for (int i = 0; i
+ < numOfPartitions; i++) { //Allocating one buffer per partition and setting as the head of the chain of buffers for that partition
+ memBuffs[i] = new VSizeFrame(ctx);
curPBuff[i] = i;
nextBuff[i] = -1;
buildPSizeInFrames[i] = 1; //The dedicated initial buffer
}
- nextFreeBuffIx = ((numOfPartitions < memForJoin) ? numOfPartitions : NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
+ nextFreeBuffIx = ((numOfPartitions < memForJoin) ?
+ numOfPartitions :
+ NO_MORE_FREE_BUFFER); //Setting the chain of unallocated frames
for (int i = numOfPartitions; i < memBuffs.length; i++) {
nextBuff[i] = UNALLOCATED_FRAME;
}
- buildTupAppender = new FrameTupleAppender(ctx.getFrameSize());
+ buildTupAppender = new FrameTupleAppender();
}
@@ -213,7 +218,7 @@
}
private void processTuple(int tid, int pid) throws HyracksDataException {
- ByteBuffer partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
+ IFrame partition = memBuffs[curPBuff[pid]]; //Getting current buffer for the target partition
if (!pStatus.get(pid)) { //resident partition
buildTupAppender.reset(partition, false);
@@ -226,7 +231,8 @@
if (newBuffIx == NO_MORE_FREE_BUFFER) { //Spill one partition
int pidToSpill = selectPartitionToSpill();
if (pidToSpill == -1) { //No more partition to spill
- throw new HyracksDataException("not enough memory for Hash Join (Allocation exceeds the limit)");
+ throw new HyracksDataException(
+ "not enough memory for Hash Join (Allocation exceeds the limit)");
}
spillPartition(pidToSpill);
buildTupAppender.reset(memBuffs[pidToSpill], true);
@@ -249,8 +255,8 @@
break;
}
//Dedicated in-memory buffer for the partition is full, needed to be flushed first
- buildWrite(pid, partition);
- partition.clear();
+ buildWrite(pid, partition.getBuffer());
+ partition.reset();
needClear = true;
buildPSizeInFrames[pid]++;
}
@@ -260,7 +266,7 @@
private int allocateFreeBuffer(int pid) throws HyracksDataException {
if (nextFreeBuffIx != NO_MORE_FREE_BUFFER) {
if (memBuffs[nextFreeBuffIx] == null) {
- memBuffs[nextFreeBuffIx] = ctx.allocateFrame();
+ memBuffs[nextFreeBuffIx] = new VSizeFrame(ctx);
}
int curPartBuffIx = curPBuff[pid];
curPBuff[pid] = nextFreeBuffIx;
@@ -274,7 +280,7 @@
} else {
nextFreeBuffIx = oldNext;
}
- (memBuffs[curPBuff[pid]]).clear();
+ memBuffs[curPBuff[pid]].reset();
freeFramesCounter--;
return (curPBuff[pid]);
@@ -300,11 +306,10 @@
+ " frames for Thread ID " + Thread.currentThread().getId() + " (free frames: " + freeFramesCounter
+ ").");
int curBuffIx = curPBuff[pid];
- ByteBuffer buff = null;
while (curBuffIx != END_OF_PARTITION) {
- buff = memBuffs[curBuffIx];
- buildWrite(pid, buff);
- buff.clear();
+ IFrame frame = memBuffs[curBuffIx];
+ buildWrite(pid, frame.getBuffer());
+ frame.reset();
int freedBuffIx = curBuffIx;
curBuffIx = nextBuff[curBuffIx];
@@ -346,8 +351,9 @@
}
ByteBuffer buff = null;
- for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus.nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
- buff = memBuffs[i];
+ for (int i = pStatus.nextSetBit(0); i >= 0; i = pStatus
+ .nextSetBit(i + 1)) { //flushing and DeAllocating the dedicated buffers for the spilled partitions
+ buff = memBuffs[i].getBuffer();
accessorBuild.reset(buff);
if (accessorBuild.getTupleCount() > 0) {
buildWrite(i, buff);
@@ -389,7 +395,7 @@
}
private void partitionTune() throws HyracksDataException {
- reloadBuffer = ctx.allocateFrame();
+ reloadBuffer = new VSizeFrame(ctx);
ArrayList<Integer> reloadSet = selectPartitionsToReload();
for (int i = 0; i < reloadSet.size(); i++) {
int pid = reloadSet.get(i);
@@ -414,7 +420,6 @@
loadPartitionInMem(pid, buildRFWriters[pid], buffsToLoad);
}
reloadSet.clear();
- reloadSet = null;
}
private void loadPartitionInMem(int pid, RunFileWriter wr, int[] buffs) throws HyracksDataException {
@@ -422,16 +427,16 @@
r.open();
int counter = 0;
ByteBuffer mBuff = null;
- reloadBuffer.clear();
+ reloadBuffer.reset();
while (r.nextFrame(reloadBuffer)) {
- mBuff = memBuffs[buffs[counter]];
- if (mBuff == null) {
- mBuff = ctx.allocateFrame();
- memBuffs[buffs[counter]] = mBuff;
+ if (memBuffs[buffs[counter]] == null) {
+ memBuffs[buffs[counter]] = new VSizeFrame(ctx);
}
- FrameUtils.copy(reloadBuffer, mBuff);
+ memBuffs[buffs[counter]].ensureFrameSize(reloadBuffer.getFrameSize());
+ mBuff = memBuffs[buffs[counter]].getBuffer();
+ FrameUtils.copyAndFlip(reloadBuffer.getBuffer(), mBuff);
counter++;
- reloadBuffer.clear();
+ reloadBuffer.reset();
}
int curNext = nextBuff[buffs[buffs.length - 1]];
@@ -459,10 +464,10 @@
private void createInMemoryJoiner(int inMemTupCount) throws HyracksDataException {
ISerializableTable table = new SerializableHashTable(inMemTupCount, ctx);
- this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount,
- new FrameTupleAccessor(ctx.getFrameSize(), probeRd), probeHpc, new FrameTupleAccessor(
- ctx.getFrameSize(), buildRd), buildHpc, new FrameTuplePairComparator(probeKeys, buildKeys,
- comparators), isLeftOuter, nullWriters1, table, predEvaluator, isReversed);
+ this.inMemJoiner = new InMemoryHashJoin(ctx, inMemTupCount, new FrameTupleAccessor(probeRd), probeHpc,
+ new FrameTupleAccessor(buildRd), buildHpc,
+ new FrameTuplePairComparator(probeKeys, buildKeys, comparators), isLeftOuter, nullWriters1, table,
+ predEvaluator, isReversed);
}
private void cacheInMemJoin() throws HyracksDataException {
@@ -471,7 +476,7 @@
if (!pStatus.get(pid)) {
int nextBuffIx = curPBuff[pid];
while (nextBuffIx > -1) { //It is not Invalid or End_Of_Partition
- inMemJoiner.build(memBuffs[nextBuffIx]);
+ inMemJoiner.build(memBuffs[nextBuffIx].getBuffer());
nextBuffIx = nextBuff[nextBuffIx];
}
}
@@ -480,9 +485,9 @@
public void initProbe() throws HyracksDataException {
- sPartBuffs = new ByteBuffer[numOfSpilledParts];
+ sPartBuffs = new IFrame[numOfSpilledParts];
for (int i = 0; i < numOfSpilledParts; i++) {
- sPartBuffs[i] = ctx.allocateFrame();
+ sPartBuffs[i] = new VSizeFrame(ctx);
}
curPBuff = new int[numOfPartitions];
int nextBuffIxToAlloc = 0;
@@ -495,12 +500,12 @@
probePSizeInTups = new int[numOfPartitions];
probeRFWriters = new RunFileWriter[numOfPartitions];
- probeResBuff = ctx.allocateFrame();
+ probeResBuff = new VSizeFrame(ctx);
- probeTupAppenderToResident = new FrameTupleAppender(ctx.getFrameSize());
+ probeTupAppenderToResident = new FrameTupleAppender();
probeTupAppenderToResident.reset(probeResBuff, true);
- probeTupAppenderToSpilled = new FrameTupleAppender(ctx.getFrameSize());
+ probeTupAppenderToSpilled = new FrameTupleAppender();
}
@@ -517,21 +522,20 @@
inMemJoiner.join(buffer, writer);
return;
}
- ByteBuffer buff = null;
for (int i = 0; i < tupleCount; ++i) {
int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
- if (buildPSizeInTups[pid] > 0) { //Tuple has potential match from previous phase
+ if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
if (pStatus.get(pid)) { //pid is Spilled
boolean needToClear = false;
- buff = sPartBuffs[curPBuff[pid]];
+ IFrame frame = sPartBuffs[curPBuff[pid]];
while (true) {
- probeTupAppenderToSpilled.reset(buff, needToClear);
+ probeTupAppenderToSpilled.reset(frame, needToClear);
if (probeTupAppenderToSpilled.append(accessorProbe, i)) {
break;
}
- probeWrite(pid, buff);
- buff.clear();
+ probeWrite(pid, frame.getBuffer());
+ frame.reset();
needToClear = true;
}
} else { //pid is Resident
@@ -539,7 +543,7 @@
if (probeTupAppenderToResident.append(accessorProbe, i)) {
break;
}
- inMemJoiner.join(probeResBuff, writer);
+ inMemJoiner.join(probeResBuff.getBuffer(), writer);
probeTupAppenderToResident.reset(probeResBuff, true);
}
@@ -551,13 +555,13 @@
}
- public void closeProbe(IFrameWriter writer) throws HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
- inMemJoiner.join(probeResBuff, writer);
+ public void closeProbe(IFrameWriter writer) throws
+ HyracksDataException { //We do NOT join the spilled partitions here, that decision is made at the descriptor level (which join technique to use)
+ inMemJoiner.join(probeResBuff.getBuffer(), writer);
inMemJoiner.closeJoin(writer);
- ByteBuffer buff = null;
for (int pid = pStatus.nextSetBit(0); pid >= 0; pid = pStatus.nextSetBit(pid + 1)) {
- buff = sPartBuffs[curPBuff[pid]];
+ ByteBuffer buff = sPartBuffs[curPBuff[pid]].getBuffer();
accessorProbe.reset(buff);
if (accessorProbe.getTupleCount() > 0) {
probeWrite(pid, buff);
@@ -651,10 +655,10 @@
double avgBuildSpSz = sumOfBuildSpilledSizes / numOfSpilledPartitions;
double avgProbeSpSz = sumOfProbeSpilledSizes / numOfSpilledPartitions;
- String s = "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t"
- + numOfSpilledPartitions + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t"
- + avgProbeSpSz + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t"
- + freeFramesCounter;
+ String s =
+ "Resident Partitions:\t" + numOfResidentPartitions + "\nSpilled Partitions:\t" + numOfSpilledPartitions
+ + "\nAvg Build Spilled Size:\t" + avgBuildSpSz + "\nAvg Probe Spilled Size:\t" + avgProbeSpSz
+ + "\nIn-Memory Tups:\t" + numOfInMemTups + "\nNum of Free Buffers:\t" + freeFramesCounter;
return s;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 0494288..840eb75 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -21,6 +21,8 @@
import java.util.BitSet;
import java.util.logging.Logger;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -368,7 +370,7 @@
IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private BuildAndPartitionTaskState state;
- private ByteBuffer rPartbuff = ctx.allocateFrame();
+ private IFrame rPartbuff = new VSizeFrame(ctx);
@Override
public void open() throws HyracksDataException {
@@ -397,13 +399,14 @@
BitSet partitionStatus = state.hybridHJ.getPartitionStatus();
- rPartbuff.clear();
+ rPartbuff.reset();
for (int pid = partitionStatus.nextSetBit(0); pid >= 0; pid = partitionStatus.nextSetBit(pid + 1)) {
RunFileReader bReader = state.hybridHJ.getBuildRFReader(pid);
RunFileReader pReader = state.hybridHJ.getProbeRFReader(pid);
- if (bReader == null || pReader == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
+ if (bReader == null || pReader
+ == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
continue;
}
int bSize = state.hybridHJ.getBuildPartitionSizeInTup(pid);
@@ -423,10 +426,14 @@
ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
hashFunctionGeneratorFactories).createPartitioner(level);
- long buildPartSize = wasReversed ? (ohhj.getProbePartitionSize(pid) / ctx.getFrameSize()) : (ohhj
- .getBuildPartitionSize(pid) / ctx.getFrameSize());
- long probePartSize = wasReversed ? (ohhj.getBuildPartitionSize(pid) / ctx.getFrameSize()) : (ohhj
- .getProbePartitionSize(pid) / ctx.getFrameSize());
+ long buildPartSize = wasReversed ?
+ (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize()) :
+ (ohhj
+ .getBuildPartitionSize(pid) / ctx.getInitialFrameSize());
+ long probePartSize = wasReversed ?
+ (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize()) :
+ (ohhj
+ .getProbePartitionSize(pid) / ctx.getInitialFrameSize());
LOGGER.fine("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId() + ") (pid "
+ pid + ") - (level " + level + ") - wasReversed " + wasReversed + " - BuildSize:\t"
@@ -437,7 +444,8 @@
if (!skipInMemoryHJ && (buildPartSize < state.memForJoin)
|| (probePartSize < state.memForJoin && !isLeftOuter)) {
int tabSize = -1;
- if (!forceRR && (isLeftOuter || (buildPartSize < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
+ if (!forceRR && (isLeftOuter || (buildPartSize
+ < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
LOGGER.fine("\t>>>Case 1.1 (IsLeftOuter || buildSize<probe) AND ApplyInMemHJ - [Level "
+ level + "]");
tabSize = wasReversed ? ohhj.getProbePartitionSizeInTup(pid) : ohhj
@@ -450,8 +458,9 @@
applyInMemHashJoin(buildKeys, probeKeys, tabSize, probeRd, buildRd, probeHpc, buildHpc,
buildSideReader, probeSideReader, false, pid); //checked-confirmed
} else { //Case 1.2 - InMemHJ with Role Reversal
- LOGGER.fine("\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
- + level + "]");
+ LOGGER.fine(
+ "\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
+ + level + "]");
tabSize = wasReversed ? ohhj.getBuildPartitionSizeInTup(pid) : ohhj
.getProbePartitionSizeInTup(pid);
if (tabSize == 0) {
@@ -467,7 +476,8 @@
else {
LOGGER.fine("\t>>>Case 2. ApplyRecursiveHHJ - [Level " + level + "]");
OptimizedHybridHashJoin rHHj;
- if (!forceRR && (isLeftOuter || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
+ if (!forceRR && (isLeftOuter
+ || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
LOGGER.fine("\t\t>>>Case 2.1 - RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ level + "]");
int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
@@ -478,18 +488,18 @@
buildSideReader.open();
rHHj.initBuild();
- rPartbuff.clear();
+ rPartbuff.reset();
while (buildSideReader.nextFrame(rPartbuff)) {
- rHHj.build(rPartbuff);
+ rHHj.build(rPartbuff.getBuffer());
}
rHHj.closeBuild();
probeSideReader.open();
rHHj.initProbe();
- rPartbuff.clear();
+ rPartbuff.reset();
while (probeSideReader.nextFrame(rPartbuff)) {
- rHHj.probe(rPartbuff, writer);
+ rHHj.probe(rPartbuff.getBuffer(), writer);
}
rHHj.closeProbe(writer);
@@ -499,10 +509,13 @@
: maxAfterProbeSize;
BitSet rPStatus = rHHj.getPartitionStatus();
- if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
- LOGGER.fine("\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
- + level + "]");
- for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
+ * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
+ LOGGER.fine(
+ "\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ + level + "]");
+ for (int rPid = rPStatus.nextSetBit(0);
+ rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -510,13 +523,16 @@
continue;
}
- joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1), false); //checked-confirmed
+ joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1),
+ false); //checked-confirmed
}
} else { //Case 2.1.2 - Switch to NLJ
- LOGGER.fine("\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
- + level + "]");
- for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ LOGGER.fine(
+ "\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ + level + "]");
+ for (int rPid = rPStatus.nextSetBit(0);
+ rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -547,16 +563,16 @@
probeSideReader.open();
rHHj.initBuild();
- rPartbuff.clear();
+ rPartbuff.reset();
while (probeSideReader.nextFrame(rPartbuff)) {
- rHHj.build(rPartbuff);
+ rHHj.build(rPartbuff.getBuffer());
}
rHHj.closeBuild();
rHHj.initProbe();
buildSideReader.open();
- rPartbuff.clear();
+ rPartbuff.reset();
while (buildSideReader.nextFrame(rPartbuff)) {
- rHHj.probe(rPartbuff, writer);
+ rHHj.probe(rPartbuff.getBuffer(), writer);
}
rHHj.closeProbe(writer);
int maxAfterBuildSize = rHHj.getMaxBuildPartitionSize();
@@ -565,10 +581,12 @@
: maxAfterProbeSize;
BitSet rPStatus = rHHj.getPartitionStatus();
- if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
+ if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
+ * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
LOGGER.fine("\t\t>>>Case 2.2.1 - KEEP APPLYING RecursiveHHJ WITH RoleReversal - [Level "
+ level + "]");
- for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ for (int rPid = rPStatus.nextSetBit(0);
+ rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -576,12 +594,15 @@
continue;
}
- joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1), true); //checked-confirmed
+ joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1),
+ true); //checked-confirmed
}
} else { //Case 2.2.2 - Switch to NLJ
- LOGGER.fine("\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
- + level + "]");
- for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ LOGGER.fine(
+ "\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
+ + level + "]");
+ for (int rPid = rPStatus.nextSetBit(0);
+ rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -611,27 +632,27 @@
ITuplePartitionComputer hpcRepSmaller, RunFileReader bReader, RunFileReader pReader,
boolean reverse, int pid) throws HyracksDataException {
ISerializableTable table = new SerializableHashTable(tabSize, ctx);
- InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(
- ctx.getFrameSize(), probeRDesc), hpcRepLarger, new FrameTupleAccessor(ctx.getFrameSize(),
- buildRDesc), hpcRepSmaller, new FrameTuplePairComparator(pKeys, bKeys, comparators),
- isLeftOuter, nullWriters1, table, predEvaluator, reverse);
+ InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tabSize, new FrameTupleAccessor(probeRDesc),
+ hpcRepLarger, new FrameTupleAccessor(buildRDesc), hpcRepSmaller,
+ new FrameTuplePairComparator(pKeys, bKeys, comparators), isLeftOuter, nullWriters1, table,
+ predEvaluator, reverse);
bReader.open();
- rPartbuff.clear();
+ rPartbuff.reset();
while (bReader.nextFrame(rPartbuff)) {
- ByteBuffer copyBuffer = ctx.allocateFrame(); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
- FrameUtils.copy(rPartbuff, copyBuffer);
- FrameUtils.makeReadable(copyBuffer);
+ ByteBuffer copyBuffer = ctx
+ .allocateFrame(rPartbuff.getFrameSize()); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
+ FrameUtils.copyAndFlip(rPartbuff.getBuffer(), copyBuffer);
joiner.build(copyBuffer);
- rPartbuff.clear();
+ rPartbuff.reset();
}
bReader.close();
- rPartbuff.clear();
+ rPartbuff.reset();
// probe
pReader.open();
while (pReader.nextFrame(rPartbuff)) {
- joiner.join(rPartbuff, writer);
- rPartbuff.clear();
+ joiner.join(rPartbuff.getBuffer(), writer);
+ rPartbuff.reset();
}
pReader.close();
joiner.closeJoin(writer);
@@ -640,27 +661,26 @@
private void applyNestedLoopJoin(RecordDescriptor outerRd, RecordDescriptor innerRd, int memorySize,
RunFileReader outerReader, RunFileReader innerReader, ITuplePairComparator nljComparator,
boolean reverse) throws HyracksDataException {
- NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(ctx.getFrameSize(), outerRd),
- new FrameTupleAccessor(ctx.getFrameSize(), innerRd), nljComparator, memorySize,
+ NestedLoopJoin nlj = new NestedLoopJoin(ctx,
+ new FrameTupleAccessor(outerRd),
+ new FrameTupleAccessor(innerRd), nljComparator, memorySize,
predEvaluator, isLeftOuter, nullWriters1);
nlj.setIsReversed(reverse);
- ByteBuffer cacheBuff = ctx.allocateFrame();
+ IFrame cacheBuff = new VSizeFrame(ctx);
innerReader.open();
while (innerReader.nextFrame(cacheBuff)) {
- FrameUtils.makeReadable(cacheBuff);
- nlj.cache(cacheBuff);
- cacheBuff.clear();
+ nlj.cache(cacheBuff.getBuffer());
+ cacheBuff.reset();
}
nlj.closeCache();
- ByteBuffer joinBuff = ctx.allocateFrame();
+ IFrame joinBuff = new VSizeFrame(ctx);
outerReader.open();
while (outerReader.nextFrame(joinBuff)) {
- FrameUtils.makeReadable(joinBuff);
- nlj.join(joinBuff, writer);
- joinBuff.clear();
+ nlj.join(joinBuff.getBuffer(), writer);
+ joinBuff.reset();
}
nlj.closeJoin(writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
index 12f319f..ffe3abd 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/ConstantTupleSourceOperatorNodePushable.java
@@ -15,12 +15,10 @@
package edu.uci.ics.hyracks.dataflow.std.misc;
-import java.nio.ByteBuffer;
-
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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 {
@@ -41,14 +39,12 @@
@Override
public void initialize() throws HyracksDataException {
- ByteBuffer writeBuffer = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(writeBuffer, true);
+ FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
if (fieldSlots != null && tupleData != null && tupleSize > 0)
appender.append(fieldSlots, tupleData, 0, tupleSize);
writer.open();
try {
- FrameUtils.flushFrame(writeBuffer, writer);
+ appender.flush(writer, false);
} catch (Exception e) {
writer.fail();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
index 1fff4fe..4356181 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/LimitOperatorDescriptor.java
@@ -16,6 +16,7 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -50,7 +51,7 @@
@Override
public void open() throws HyracksDataException {
- fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptors[0]);
+ fta = new FrameTupleAccessor(recordDescriptors[0]);
currentSize = 0;
finished = false;
writer.open();
@@ -62,16 +63,13 @@
fta.reset(buffer);
int count = fta.getTupleCount();
if ((currentSize + count) > outputLimit) {
- ByteBuffer b = ctx.allocateFrame();
- FrameTupleAppender partialAppender = new FrameTupleAppender(ctx.getFrameSize());
- partialAppender.reset(b, true);
+ FrameTupleAppender partialAppender = new FrameTupleAppender(new VSizeFrame(ctx));
int copyCount = outputLimit - currentSize;
for (int i = 0; i < copyCount; i++) {
- partialAppender.append(fta, i);
+ FrameUtils.appendToWriter(writer, partialAppender, fta, i);
currentSize++;
}
- FrameUtils.makeReadable(b);
- FrameUtils.flushFrame(b, writer);
+ partialAppender.flush(writer,false);
finished = true;
} else {
FrameUtils.flushFrame(buffer, writer);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
index 48de837..9339b34 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializerTaskState.java
@@ -19,6 +19,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.TaskId;
@@ -27,6 +28,7 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
public class MaterializerTaskState extends AbstractStateObject {
@@ -61,15 +63,13 @@
out.nextFrame(buffer);
}
- public void writeOut(IFrameWriter writer, ByteBuffer frame) throws HyracksDataException {
+ public void writeOut(IFrameWriter writer, IFrame frame) throws HyracksDataException {
RunFileReader in = out.createReader();
writer.open();
try {
in.open();
while (in.nextFrame(frame)) {
- frame.flip();
- writer.nextFrame(frame);
- frame.clear();
+ writer.nextFrame(frame.getBuffer());
}
in.close();
} catch (Exception e) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index 3a405d0..36fdd50 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -16,6 +16,7 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -25,6 +26,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
@@ -107,8 +110,7 @@
@Override
public void close() throws HyracksDataException {
state.close();
- ByteBuffer frame = ctx.allocateFrame();
- state.writeOut(writer, frame);
+ state.writeOut(writer, new VSizeFrame(ctx));
}
};
@@ -166,10 +168,9 @@
return new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
public void initialize() throws HyracksDataException {
- ByteBuffer frame = ctx.allocateFrame();
MaterializerTaskState state = (MaterializerTaskState) ctx.getStateObject(new TaskId(new ActivityId(
getOperatorId(), MATERIALIZER_ACTIVITY_ID), partition));
- state.writeOut(writer, frame);
+ state.writeOut(writer, new VSizeFrame(ctx));
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
index b8e1ac8..1f6d965 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitOperatorDescriptor.java
@@ -17,6 +17,7 @@
import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
@@ -172,10 +173,9 @@
@Override
public void initialize() throws HyracksDataException {
- ByteBuffer frame = ctx.allocateFrame();
MaterializerTaskState state = (MaterializerTaskState) ctx.getStateObject(new TaskId(new ActivityId(
getOperatorId(), SPLITTER_MATERIALIZER_ACTIVITY_ID), partition));
- state.writeOut(writer, frame);
+ state.writeOut(writer, new VSizeFrame(ctx));
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
index 42ed59e..3a6b5d2 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -18,7 +18,9 @@
import java.io.PrintStream;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -61,17 +63,17 @@
throws HyracksDataException {
final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
- final ByteBuffer outputBuffer = ctx.allocateFrame();
+ final IFrame frame = new VSizeFrame(ctx);
- final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
- frameOutputStream.reset(outputBuffer, true);
+ final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getInitialFrameSize());
+ frameOutputStream.reset(frame, true);
PrintStream printStream = new PrintStream(frameOutputStream);
final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
printStream);
- final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+ final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(outRecordDesc);
return new AbstractUnaryInputSinkOperatorNodePushable() {
IFrameWriter datasetPartitionWriter;
@@ -94,12 +96,8 @@
for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
resultSerializer.appendTuple(frameTupleAccessor, tIndex);
if (!frameOutputStream.appendTuple()) {
- datasetPartitionWriter.nextFrame(outputBuffer);
- frameOutputStream.reset(outputBuffer, true);
+ frameOutputStream.flush(datasetPartitionWriter);
- /* TODO(madhusudancs): This works under the assumption that no single serialized record is
- * longer than the buffer size.
- */
resultSerializer.appendTuple(frameTupleAccessor, tIndex);
frameOutputStream.appendTuple();
}
@@ -114,8 +112,7 @@
@Override
public void close() throws HyracksDataException {
if (frameOutputStream.getTupleCount() > 0) {
- datasetPartitionWriter.nextFrame(outputBuffer);
- frameOutputStream.reset(outputBuffer, true);
+ frameOutputStream.flush(datasetPartitionWriter);
}
datasetPartitionWriter.close();
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java
new file mode 100644
index 0000000..2a1a403
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractFrameSorter.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+
+public abstract class AbstractFrameSorter implements IFrameSorter {
+
+ protected Logger LOGGER = Logger.getLogger(AbstractFrameSorter.class.getName());
+ static final int PTR_SIZE = 4;
+ static final int ID_FRAMEID = 0;
+ static final int ID_TUPLE_START = 1;
+ static final int ID_TUPLE_END = 2;
+ static final int ID_NORMAL_KEY = 3;
+
+ protected final int[] sortFields;
+ protected final IBinaryComparator[] comparators;
+ protected final INormalizedKeyComputer nkc;
+ protected final IFrameBufferManager bufferManager;
+ protected final FrameTupleAccessor inputTupleAccessor;
+ protected final IFrameTupleAppender outputAppender;
+ protected final IFrame outputFrame;
+ protected final int outputLimit;
+
+ protected int[] tPointers;
+ protected int tupleCount;
+
+ public AbstractFrameSorter(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) throws HyracksDataException {
+ this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ Integer.MAX_VALUE);
+ }
+
+ public AbstractFrameSorter(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, int outputLimit)
+ throws HyracksDataException {
+ this.bufferManager = bufferManager;
+ this.sortFields = sortFields;
+ this.nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+ this.comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ this.inputTupleAccessor = new FrameTupleAccessor(recordDescriptor);
+ this.outputAppender = new FrameTupleAppender();
+ this.outputFrame = new VSizeFrame(ctx);
+ this.outputLimit = outputLimit;
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ this.tupleCount = 0;
+ this.bufferManager.reset();
+ }
+
+ @Override
+ public boolean insertFrame(ByteBuffer inputBuffer) throws HyracksDataException {
+ if (bufferManager.insertFrame(inputBuffer) >= 0) {
+ return true;
+ }
+ if (getFrameCount() == 0) {
+ throw new HyracksDataException(
+ "The input frame is too big for the sorting buffer, please allocate bigger buffer size");
+ }
+ return false;
+ }
+
+ @Override
+ public void sort() throws HyracksDataException {
+ tupleCount = 0;
+ for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
+ inputTupleAccessor
+ .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
+ bufferManager.getFrameSize(i));
+ tupleCount += inputTupleAccessor.getTupleCount();
+ }
+ if (tPointers == null || tPointers.length < tupleCount * PTR_SIZE) {
+ tPointers = new int[tupleCount * PTR_SIZE];
+ }
+ int ptr = 0;
+ int sfIdx = sortFields[0];
+ for (int i = 0; i < bufferManager.getNumFrames(); ++i) {
+ inputTupleAccessor
+ .reset(bufferManager.getFrame(i), bufferManager.getFrameStartOffset(i),
+ bufferManager.getFrameSize(i));
+ int tCount = inputTupleAccessor.getTupleCount();
+ byte[] array = inputTupleAccessor.getBuffer().array();
+ for (int j = 0; j < tCount; ++j) {
+ int tStart = inputTupleAccessor.getTupleStartOffset(j);
+ int tEnd = inputTupleAccessor.getTupleEndOffset(j);
+ tPointers[ptr * PTR_SIZE + ID_FRAMEID] = i;
+ tPointers[ptr * PTR_SIZE + ID_TUPLE_START] = tStart;
+ tPointers[ptr * PTR_SIZE + ID_TUPLE_END] = tEnd;
+ int f0StartRel = inputTupleAccessor.getFieldStartOffset(j, sfIdx);
+ int f0EndRel = inputTupleAccessor.getFieldEndOffset(j, sfIdx);
+ int f0Start = f0StartRel + tStart + inputTupleAccessor.getFieldSlotsLength();
+ tPointers[ptr * PTR_SIZE + ID_NORMAL_KEY] =
+ nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
+ ++ptr;
+ }
+ }
+ if (tupleCount > 0) {
+ sortTupleReferences();
+ }
+ }
+
+ abstract void sortTupleReferences() throws HyracksDataException;
+
+ @Override
+ public int getFrameCount() {
+ return bufferManager.getNumFrames();
+ }
+
+ @Override
+ public boolean hasRemaining() {
+ return getFrameCount() > 0;
+ }
+
+ @Override
+ public int flush(IFrameWriter writer) throws HyracksDataException {
+ outputAppender.reset(outputFrame, true);
+ int maxFrameSize = outputFrame.getFrameSize();
+ int limit = Math.min(tupleCount, outputLimit);
+ int io = 0;
+ for (int ptr = 0; ptr < limit; ++ptr) {
+ int i = tPointers[ptr * PTR_SIZE + ID_FRAMEID];
+ int tStart = tPointers[ptr * PTR_SIZE + ID_TUPLE_START];
+ int tEnd = tPointers[ptr * PTR_SIZE + ID_TUPLE_END];
+ ByteBuffer buffer = bufferManager.getFrame(i);
+ inputTupleAccessor.reset(buffer, bufferManager.getFrameStartOffset(i), bufferManager.getFrameSize(i));
+
+ int flushed = FrameUtils.appendToWriter(writer, outputAppender, inputTupleAccessor, tStart, tEnd);
+ if (flushed > 0) {
+ maxFrameSize = Math.max(maxFrameSize, flushed);
+ io++;
+ }
+ }
+ maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
+ outputAppender.flush(writer, true);
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine(
+ "Flushed records:" + limit + " out of " + tupleCount + "; Flushed through " + (io + 1) + " frames");
+ }
+ return maxFrameSize;
+ }
+
+ @Override
+ public void close() {
+ tupleCount = 0;
+ bufferManager.close();
+ tPointers = null;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
new file mode 100644
index 0000000..1dd35a8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSortRunGenerator.java
@@ -0,0 +1,77 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+
+public abstract class AbstractSortRunGenerator implements IRunGenerator {
+ protected final List<RunAndMaxFrameSizePair> runAndMaxSizes;
+
+ public AbstractSortRunGenerator() {
+ runAndMaxSizes = new LinkedList<>();
+ }
+
+ abstract public ISorter getSorter() throws HyracksDataException;
+
+ @Override
+ public void open() throws HyracksDataException {
+ runAndMaxSizes.clear();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (getSorter().hasRemaining()) {
+ if (runAndMaxSizes.size() <= 0) {
+ getSorter().sort();
+ } else {
+ flushFramesToRun();
+ }
+ }
+ }
+
+ abstract protected RunFileWriter getRunFileWriter() throws HyracksDataException;
+
+ abstract protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException;
+
+ protected void flushFramesToRun() throws HyracksDataException {
+ getSorter().sort();
+ RunFileWriter runWriter = getRunFileWriter();
+ IFrameWriter flushWriter = getFlushableFrameWriter(runWriter);
+ flushWriter.open();
+ int maxFlushedFrameSize;
+ try {
+ maxFlushedFrameSize = getSorter().flush(flushWriter);
+ } finally {
+ flushWriter.close();
+ }
+ runAndMaxSizes.add(new RunAndMaxFrameSizePair(runWriter.createReader(), maxFlushedFrameSize));
+ getSorter().reset();
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+
+ @Override
+ public List<RunAndMaxFrameSizePair> getRuns() {
+ return runAndMaxSizes;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
new file mode 100644
index 0000000..0c1c622
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/AbstractSorterOperatorDescriptor.java
@@ -0,0 +1,197 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.TaskId;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+public abstract class AbstractSorterOperatorDescriptor extends AbstractOperatorDescriptor {
+
+ private static final Logger LOGGER = Logger.getLogger(AbstractSorterOperatorDescriptor.class.getName());
+
+ private static final long serialVersionUID = 1L;
+
+ protected static final int SORT_ACTIVITY_ID = 0;
+ protected static final int MERGE_ACTIVITY_ID = 1;
+
+ protected final int[] sortFields;
+ protected final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
+ protected final IBinaryComparatorFactory[] comparatorFactories;
+ protected final int framesLimit;
+
+ public AbstractSorterOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) {
+ super(spec, 1, 1);
+ this.framesLimit = framesLimit;
+ this.sortFields = sortFields;
+ this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
+ this.comparatorFactories = comparatorFactories;
+ recordDescriptors[0] = recordDescriptor;
+ }
+
+ public abstract SortActivity getSortActivity(ActivityId id);
+
+ public abstract MergeActivity getMergeActivity(ActivityId id);
+
+ @Override
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ SortActivity sa = getSortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
+ MergeActivity ma = getMergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
+
+ builder.addActivity(this, sa);
+ builder.addSourceEdge(0, sa, 0);
+
+ builder.addActivity(this, ma);
+ builder.addTargetEdge(0, ma, 0);
+
+ builder.addBlockingEdge(sa, ma);
+ }
+
+ public static class SortTaskState extends AbstractStateObject {
+ public List<RunAndMaxFrameSizePair> runAndMaxFrameSizePairs;
+ public ISorter sorter;
+
+ public SortTaskState(JobId jobId, TaskId taskId) {
+ super(jobId, taskId);
+ }
+
+ @Override
+ public void toBytes(DataOutput out) throws IOException {
+
+ }
+
+ @Override
+ public void fromBytes(DataInput in) throws IOException {
+
+ }
+ }
+
+ protected abstract class SortActivity extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ public SortActivity(ActivityId id) {
+ super(id);
+ }
+
+ protected abstract AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider) throws HyracksDataException;
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
+ private AbstractSortRunGenerator runGen;
+
+ @Override
+ public void open() throws HyracksDataException {
+ runGen = getRunGenerator(ctx, recordDescProvider);
+ runGen.open();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ runGen.nextFrame(buffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(),
+ new TaskId(getActivityId(), partition));
+ runGen.close();
+ state.runAndMaxFrameSizePairs = runGen.getRuns();
+ state.sorter = runGen.getSorter();
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("InitialNumberOfRuns:" + runGen.getRuns().size());
+ }
+ ctx.setStateObject(state);
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ runGen.fail();
+ }
+ };
+ return op;
+ }
+ }
+
+ protected abstract class MergeActivity extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ public MergeActivity(ActivityId id) {
+ super(id);
+ }
+
+ protected abstract ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter,
+ List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+ int necessaryFrames);
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
+
+ @Override
+ public void initialize() throws HyracksDataException {
+ SortTaskState state = (SortTaskState) ctx
+ .getStateObject(new TaskId(new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
+ List<RunAndMaxFrameSizePair> runs = state.runAndMaxFrameSizePairs;
+ ISorter sorter = state.sorter;
+ IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ?
+ null :
+ firstKeyNormalizerFactory.createNormalizedKeyComputer();
+ ExternalSortRunMerger merger = getSortRunMerger(ctx, recordDescProvider, writer, sorter, runs,
+ comparators, nmkComputer, framesLimit);
+ merger.process();
+ }
+ };
+ return op;
+ }
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
deleted file mode 100644
index 110bddb..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTMemMgr.java
+++ /dev/null
@@ -1,717 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-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.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-
-/**
- * @author pouria Implements Memory Manager based on creating Binary Search Tree
- * (BST) while Free slot size is the key for the BST nodes. Each node in
- * BST shows a class of free slots, while all the free slots within a
- * class have same lengths. Slots in a class are stored as a LinkedList,
- * whose head is the BST node, corresponding to that class. BST is not
- * stored as a separate data structure, but the free slots in the memory
- * are used to hold BST nodes. Each BST node has the logical structure,
- * defined in the BSTNodeUtil class.
- */
-public class BSTMemMgr implements IMemoryManager {
-
- private final IHyracksTaskContext ctx;
- public static int frameSize;
-
- private ByteBuffer[] frames;
- private ByteBuffer convertBuffer;
- private Slot root;
- private Slot result; // A reusable object to hold one node returned as
- // method result
- private Slot insertSlot; // A reusable object to hold one node within insert
- // process
- private Slot lastLeftParent; // A reusable object for the search process
- private Slot lastLeft; // A reusable object for the search process
- private Slot parent; // A reusable object for the search process
-
- private Slot[] parentRes;
- private int lastFrame;
-
- public BSTMemMgr(IHyracksTaskContext ctx, int memSize) {
- this.ctx = ctx;
- frameSize = ctx.getFrameSize();
- convertBuffer = ByteBuffer.allocate(4);
- frames = new ByteBuffer[memSize];
- lastFrame = -1;
- root = new Slot();
- insertSlot = new Slot();
- result = new Slot();
- lastLeftParent = new Slot();
- lastLeft = new Slot();
- parent = new Slot();
- parentRes = new Slot[] { new Slot(), new Slot() };
- }
-
- /**
- * result is the container sent by the caller to hold the results
- */
- @Override
- public void allocate(int length, Slot result) throws HyracksDataException {
- search(length, parentRes);
- if (parentRes[1].isNull()) {
- addFrame(parentRes);
- if (parentRes[1].isNull()) {
- return;
- }
- }
-
- int sl = BSTNodeUtil.getLength(parentRes[1], frames, convertBuffer);
- int acLen = BSTNodeUtil.getActualLength(length);
- if (shouldSplit(sl, acLen)) {
- int[] s = split(parentRes[1], parentRes[0], acLen);
- int insertLen = BSTNodeUtil.getLength(s[2], s[3], frames, convertBuffer);
- insert(s[2], s[3], insertLen); // inserting second half of the split
- // slot
- BSTNodeUtil.setHeaderFooter(s[0], s[1], length, false, frames);
- result.set(s[0], s[1]);
- return;
- }
- allocate(parentRes[1], parentRes[0], length, result);
- }
-
- @Override
- public int unallocate(Slot s) throws HyracksDataException {
- int usedLen = BSTNodeUtil.getLength(s, frames, convertBuffer);
- int actualLen = BSTNodeUtil.getActualLength(usedLen);
- int fix = s.getFrameIx();
- int off = s.getOffset();
-
- int prevMemSlotFooterOffset = ((off - BSTNodeUtil.HEADER_SIZE) >= 0 ? (off - BSTNodeUtil.HEADER_SIZE)
- : BSTNodeUtil.INVALID_INDEX);
- int t = off + 2 * BSTNodeUtil.HEADER_SIZE + actualLen;
- int nextMemSlotHeaderOffset = (t < frameSize ? t : BSTNodeUtil.INVALID_INDEX);
- // Remember: next and prev memory slots have the same frame index as the
- // unallocated slot
- if (!isNodeNull(fix, prevMemSlotFooterOffset) && BSTNodeUtil.isFree(fix, prevMemSlotFooterOffset, frames)) {
- int leftLength = BSTNodeUtil.getLength(fix, prevMemSlotFooterOffset, frames, convertBuffer);
- removeFromList(fix, prevMemSlotFooterOffset - leftLength - BSTNodeUtil.HEADER_SIZE);
- int concatLength = actualLen + leftLength + 2 * BSTNodeUtil.HEADER_SIZE;
- if (!isNodeNull(fix, nextMemSlotHeaderOffset) && BSTNodeUtil.isFree(fix, nextMemSlotHeaderOffset, frames)) {
- removeFromList(fix, nextMemSlotHeaderOffset);
- concatLength += BSTNodeUtil.getLength(fix, nextMemSlotHeaderOffset, frames, convertBuffer) + 2
- * BSTNodeUtil.HEADER_SIZE;
- }
- insert(fix, prevMemSlotFooterOffset - leftLength - BSTNodeUtil.HEADER_SIZE, concatLength); // newly
- // (merged)
- // slot
- // starts
- // at
- // the
- // prev
- // slot
- // offset
- return concatLength;
-
- } else if (!isNodeNull(fix, nextMemSlotHeaderOffset)
- && BSTNodeUtil.isFree(fix, nextMemSlotHeaderOffset, frames)) {
- removeFromList(fix, nextMemSlotHeaderOffset);
- int concatLength = actualLen + BSTNodeUtil.getLength(fix, nextMemSlotHeaderOffset, frames, convertBuffer)
- + 2 * BSTNodeUtil.HEADER_SIZE;
- insert(fix, off, concatLength); // newly (merged) slot starts at the
- // unallocated slot offset
- return concatLength;
- }
- // unallocated slot is not merging with any neighbor
- insert(fix, off, actualLen);
- return actualLen;
- }
-
- @Override
- public boolean readTuple(int frameIx, int offset, FrameTupleAppender dest) {
- int offToRead = offset + BSTNodeUtil.HEADER_SIZE;
- int length = BSTNodeUtil.getLength(frameIx, offset, frames, convertBuffer);
- return dest.append(frames[frameIx].array(), offToRead, length);
- }
-
- @Override
- public boolean writeTuple(int frameIx, int offset, FrameTupleAccessor src, int tIndex) {
- int offToCopy = offset + BSTNodeUtil.HEADER_SIZE;
- int tStartOffset = src.getTupleStartOffset(tIndex);
- int tEndOffset = src.getTupleEndOffset(tIndex);
- int tupleLength = tEndOffset - tStartOffset;
- ByteBuffer srcBuffer = src.getBuffer();
- System.arraycopy(srcBuffer.array(), tStartOffset, frames[frameIx].array(), offToCopy, tupleLength);
- return true;
- }
-
- @Override
- public ByteBuffer getFrame(int frameIndex) {
- return frames[frameIndex];
- }
-
- @Override
- public void close() {
- //clean up all frames
- for (int i = 0; i < frames.length; i++)
- frames[i] = null;
- }
-
- /**
- * @param parentResult
- * is the container passed by the caller to contain the results
- * @throws HyracksDataException
- */
- private void addFrame(Slot[] parentResult) throws HyracksDataException {
- clear(parentResult);
- if ((lastFrame + 1) >= frames.length) {
- return;
- }
- frames[++lastFrame] = allocateFrame();
- int l = frameSize - 2 * BSTNodeUtil.HEADER_SIZE;
- BSTNodeUtil.setHeaderFooter(lastFrame, 0, l, true, frames);
- initNewNode(lastFrame, 0);
-
- parentResult[1].copy(root);
- if (parentResult[1].isNull()) { // root is null
- root.set(lastFrame, 0);
- initNewNode(root.getFrameIx(), root.getOffset());
- parentResult[1].copy(root);
- return;
- }
-
- while (!parentResult[1].isNull()) {
- if (BSTNodeUtil.getLength(parentResult[1], frames, convertBuffer) == l) {
- append(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0);
- parentResult[1].set(lastFrame, 0);
- return;
- }
- if (l < BSTNodeUtil.getLength(parentResult[1], frames, convertBuffer)) {
- if (isNodeNull(BSTNodeUtil.getLeftChildFrameIx(parentResult[1], frames, convertBuffer),
- BSTNodeUtil.getLeftChildOffset(parentResult[1], frames, convertBuffer))) {
- BSTNodeUtil.setLeftChild(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0,
- frames);
- parentResult[0].copy(parentResult[1]);
- parentResult[1].set(lastFrame, 0);
- return;
- } else {
- parentResult[0].copy(parentResult[1]);
- parentResult[1].set(BSTNodeUtil.getLeftChildFrameIx(parentResult[1], frames, convertBuffer),
- BSTNodeUtil.getLeftChildOffset(parentResult[1], frames, convertBuffer));
- }
- } else {
- if (isNodeNull(BSTNodeUtil.getRightChildFrameIx(parentResult[1], frames, convertBuffer),
- BSTNodeUtil.getRightChildOffset(parentResult[1], frames, convertBuffer))) {
- BSTNodeUtil.setRightChild(parentResult[1].getFrameIx(), parentResult[1].getOffset(), lastFrame, 0,
- frames);
- parentResult[0].copy(parentResult[1]);
- parentResult[1].set(lastFrame, 0);
- return;
- } else {
- parentResult[0].copy(parentResult[1]);
- parentResult[1].set(BSTNodeUtil.getRightChildFrameIx(parentResult[1], frames, convertBuffer),
- BSTNodeUtil.getRightChildOffset(parentResult[1], frames, convertBuffer));
- }
- }
- }
- throw new HyracksDataException("New Frame could not be added to BSTMemMgr");
- }
-
- private void insert(int fix, int off, int length) throws HyracksDataException {
- BSTNodeUtil.setHeaderFooter(fix, off, length, true, frames);
- initNewNode(fix, off);
-
- if (root.isNull()) {
- root.set(fix, off);
- return;
- }
-
- insertSlot.clear();
- insertSlot.copy(root);
- while (!insertSlot.isNull()) {
- int curSlotLen = BSTNodeUtil.getLength(insertSlot, frames, convertBuffer);
- if (curSlotLen == length) {
- append(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off);
- return;
- }
- if (length < curSlotLen) {
- int leftChildFIx = BSTNodeUtil.getLeftChildFrameIx(insertSlot, frames, convertBuffer);
- int leftChildOffset = BSTNodeUtil.getLeftChildOffset(insertSlot, frames, convertBuffer);
- if (isNodeNull(leftChildFIx, leftChildOffset)) {
- initNewNode(fix, off);
- BSTNodeUtil.setLeftChild(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off, frames);
- return;
- } else {
- insertSlot.set(leftChildFIx, leftChildOffset);
- }
- } else {
- int rightChildFIx = BSTNodeUtil.getRightChildFrameIx(insertSlot, frames, convertBuffer);
- int rightChildOffset = BSTNodeUtil.getRightChildOffset(insertSlot, frames, convertBuffer);
- if (isNodeNull(rightChildFIx, rightChildOffset)) {
- initNewNode(fix, off);
- BSTNodeUtil.setRightChild(insertSlot.getFrameIx(), insertSlot.getOffset(), fix, off, frames);
- return;
- } else {
- insertSlot.set(rightChildFIx, rightChildOffset);
- }
- }
- }
- throw new HyracksDataException("Failure in node insertion into BST in BSTMemMgr");
- }
-
- /**
- * @param length
- * @param target
- * is the container sent by the caller to hold the results
- */
- private void search(int length, Slot[] target) {
- clear(target);
- result.clear();
-
- if (root.isNull()) {
- return;
- }
-
- lastLeftParent.clear();
- lastLeft.clear();
- parent.clear();
- result.copy(root);
-
- while (!result.isNull()) {
- if (BSTNodeUtil.getLength(result, frames, convertBuffer) == length) {
- target[0].copy(parent);
- target[1].copy(result);
- return;
- }
- if (length < BSTNodeUtil.getLength(result, frames, convertBuffer)) {
- lastLeftParent.copy(parent);
- lastLeft.copy(result);
- parent.copy(result);
- int fix = BSTNodeUtil.getLeftChildFrameIx(result, frames, convertBuffer);
- int off = BSTNodeUtil.getLeftChildOffset(result, frames, convertBuffer);
- result.set(fix, off);
- } else {
- parent.copy(result);
- int fix = BSTNodeUtil.getRightChildFrameIx(result, frames, convertBuffer);
- int off = BSTNodeUtil.getRightChildOffset(result, frames, convertBuffer);
- result.set(fix, off);
- }
- }
-
- target[0].copy(lastLeftParent);
- target[1].copy(lastLeft);
-
- }
-
- private void append(int headFix, int headOff, int nodeFix, int nodeOff) {
- initNewNode(nodeFix, nodeOff);
-
- int fix = BSTNodeUtil.getNextFrameIx(headFix, headOff, frames, convertBuffer); // frameIx
- // for
- // the
- // current
- // next
- // of
- // head
- int off = BSTNodeUtil.getNextOffset(headFix, headOff, frames, convertBuffer); // offset
- // for
- // the
- // current
- // next
- // of
- // head
- BSTNodeUtil.setNext(nodeFix, nodeOff, fix, off, frames);
-
- if (!isNodeNull(fix, off)) {
- BSTNodeUtil.setPrev(fix, off, nodeFix, nodeOff, frames);
- }
- BSTNodeUtil.setPrev(nodeFix, nodeOff, headFix, headOff, frames);
- BSTNodeUtil.setNext(headFix, headOff, nodeFix, nodeOff, frames);
- }
-
- private int[] split(Slot listHead, Slot parent, int length) {
- int l2 = BSTNodeUtil.getLength(listHead, frames, convertBuffer) - length - 2 * BSTNodeUtil.HEADER_SIZE;
- // We split the node after slots-list head
- if (!isNodeNull(BSTNodeUtil.getNextFrameIx(listHead, frames, convertBuffer),
- BSTNodeUtil.getNextOffset(listHead, frames, convertBuffer))) {
- int afterHeadFix = BSTNodeUtil.getNextFrameIx(listHead, frames, convertBuffer);
- int afterHeadOff = BSTNodeUtil.getNextOffset(listHead, frames, convertBuffer);
- int afHNextFix = BSTNodeUtil.getNextFrameIx(afterHeadFix, afterHeadOff, frames, convertBuffer);
- int afHNextOff = BSTNodeUtil.getNextOffset(afterHeadFix, afterHeadOff, frames, convertBuffer);
- BSTNodeUtil.setNext(listHead.getFrameIx(), listHead.getOffset(), afHNextFix, afHNextOff, frames);
- if (!isNodeNull(afHNextFix, afHNextOff)) {
- BSTNodeUtil.setPrev(afHNextFix, afHNextOff, listHead.getFrameIx(), listHead.getOffset(), frames);
- }
- int secondOffset = afterHeadOff + length + 2 * BSTNodeUtil.HEADER_SIZE;
- BSTNodeUtil.setHeaderFooter(afterHeadFix, afterHeadOff, length, true, frames);
- BSTNodeUtil.setHeaderFooter(afterHeadFix, secondOffset, l2, true, frames);
-
- return new int[] { afterHeadFix, afterHeadOff, afterHeadFix, secondOffset };
- }
- // We split the head
- int secondOffset = listHead.getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE;
- BSTNodeUtil.setHeaderFooter(listHead.getFrameIx(), listHead.getOffset(), length, true, frames);
- BSTNodeUtil.setHeaderFooter(listHead.getFrameIx(), secondOffset, l2, true, frames);
-
- fixTreePtrs(listHead.getFrameIx(), listHead.getOffset(), parent.getFrameIx(), parent.getOffset());
- return new int[] { listHead.getFrameIx(), listHead.getOffset(), listHead.getFrameIx(), secondOffset };
- }
-
- private void fixTreePtrs(int nodeFrameIx, int nodeOffset, int parentFrameIx, int parentOffset) {
- int nodeLeftChildFrameIx = BSTNodeUtil.getLeftChildFrameIx(nodeFrameIx, nodeOffset, frames, convertBuffer);
- int nodeLeftChildOffset = BSTNodeUtil.getLeftChildOffset(nodeFrameIx, nodeOffset, frames, convertBuffer);
- int nodeRightChildFrameIx = BSTNodeUtil.getRightChildFrameIx(nodeFrameIx, nodeOffset, frames, convertBuffer);
- int nodeRightChildOffset = BSTNodeUtil.getRightChildOffset(nodeFrameIx, nodeOffset, frames, convertBuffer);
-
- int status = -1; // (status==0 if node is left child of parent)
- // (status==1 if node is right child of parent)
- if (!isNodeNull(parentFrameIx, parentOffset)) {
- int nlen = BSTNodeUtil.getActualLength(BSTNodeUtil
- .getLength(nodeFrameIx, nodeOffset, frames, convertBuffer));
- int plen = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(parentFrameIx, parentOffset, frames,
- convertBuffer));
- status = ((nlen < plen) ? 0 : 1);
- }
-
- if (!isNodeNull(nodeLeftChildFrameIx, nodeLeftChildOffset)
- && !isNodeNull(nodeRightChildFrameIx, nodeRightChildOffset)) { // Node
- // has
- // two
- // children
- int pMinFIx = nodeFrameIx;
- int pMinOff = nodeOffset;
- int minFIx = nodeRightChildFrameIx;
- int minOff = nodeRightChildOffset;
- int nextLeftFIx = BSTNodeUtil.getLeftChildFrameIx(minFIx, minOff, frames, convertBuffer);
- int nextLeftOff = BSTNodeUtil.getLeftChildOffset(minFIx, minOff, frames, convertBuffer);
-
- while (!isNodeNull(nextLeftFIx, nextLeftOff)) {
- pMinFIx = minFIx;
- pMinOff = minOff;
- minFIx = nextLeftFIx;
- minOff = nextLeftOff;
- nextLeftFIx = BSTNodeUtil.getLeftChildFrameIx(minFIx, minOff, frames, convertBuffer); // min
- // is
- // now
- // pointing
- // to
- // current
- // (old)
- // next
- // left
- nextLeftOff = BSTNodeUtil.getLeftChildOffset(minFIx, minOff, frames, convertBuffer); // min
- // is
- // now
- // pointing
- // to
- // current
- // (old)
- // next
- // left
- }
-
- if ((nodeRightChildFrameIx == minFIx) && (nodeRightChildOffset == minOff)) { // nrc
- // is
- // the
- // same as min
- BSTNodeUtil.setLeftChild(nodeRightChildFrameIx, nodeRightChildOffset, nodeLeftChildFrameIx,
- nodeLeftChildOffset, frames);
- } else { // min is different from nrc
- int minRightFIx = BSTNodeUtil.getRightChildFrameIx(minFIx, minOff, frames, convertBuffer);
- int minRightOffset = BSTNodeUtil.getRightChildOffset(minFIx, minOff, frames, convertBuffer);
- BSTNodeUtil.setRightChild(minFIx, minOff, nodeRightChildFrameIx, nodeRightChildOffset, frames);
- BSTNodeUtil.setLeftChild(minFIx, minOff, nodeLeftChildFrameIx, nodeLeftChildOffset, frames);
- BSTNodeUtil.setLeftChild(pMinFIx, pMinOff, minRightFIx, minRightOffset, frames);
- }
-
- // Now dealing with the parent
- if (!isNodeNull(parentFrameIx, parentOffset)) {
- if (status == 0) {
- BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, minFIx, minOff, frames);
- } else if (status == 1) {
- BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, minFIx, minOff, frames);
- }
- } else { // No parent (node was the root)
- root.set(minFIx, minOff);
- }
- return;
- }
-
- else if (!isNodeNull(nodeLeftChildFrameIx, nodeLeftChildOffset)) { // Node
- // has
- // only
- // left
- // child
- if (status == 0) {
- BSTNodeUtil
- .setLeftChild(parentFrameIx, parentOffset, nodeLeftChildFrameIx, nodeLeftChildOffset, frames);
- } else if (status == 1) {
- BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, nodeLeftChildFrameIx, nodeLeftChildOffset,
- frames);
- } else if (status == -1) { // No parent, so node is root
- root.set(nodeLeftChildFrameIx, nodeLeftChildOffset);
- }
- return;
- }
-
- else if (!isNodeNull(nodeRightChildFrameIx, nodeRightChildOffset)) { // Node
- // has
- // only
- // right
- // child
- if (status == 0) {
- BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, nodeRightChildFrameIx, nodeRightChildOffset,
- frames);
- } else if (status == 1) {
- BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, nodeRightChildFrameIx, nodeRightChildOffset,
- frames);
- } else if (status == -1) { // No parent, so node is root
- root.set(nodeRightChildFrameIx, nodeRightChildOffset);
- }
- return;
- }
-
- else { // Node is leaf (no children)
- if (status == 0) {
- BSTNodeUtil.setLeftChild(parentFrameIx, parentOffset, BSTNodeUtil.INVALID_INDEX,
- BSTNodeUtil.INVALID_INDEX, frames);
- } else if (status == 1) {
- BSTNodeUtil.setRightChild(parentFrameIx, parentOffset, BSTNodeUtil.INVALID_INDEX,
- BSTNodeUtil.INVALID_INDEX, frames);
- } else { // node was the only node in the tree
- root.clear();
- }
- return;
- }
- }
-
- /**
- * Allocation with no splitting but padding
- *
- * @param node
- * @param parent
- * @param result
- * is the container sent by the caller to hold the results
- */
- private void allocate(Slot node, Slot parent, int length, Slot result) {
- int nextFix = BSTNodeUtil.getNextFrameIx(node, frames, convertBuffer);
- int nextOff = BSTNodeUtil.getNextOffset(node, frames, convertBuffer);
- if (!isNodeNull(nextFix, nextOff)) {
- int nextOfNextFIx = BSTNodeUtil.getNextFrameIx(nextFix, nextOff, frames, convertBuffer);
- int nextOfNextOffset = BSTNodeUtil.getNextOffset(nextFix, nextOff, frames, convertBuffer);
- BSTNodeUtil.setNext(node.getFrameIx(), node.getOffset(), nextOfNextFIx, nextOfNextOffset, frames);
- if (!isNodeNull(nextOfNextFIx, nextOfNextOffset)) {
- BSTNodeUtil.setPrev(nextOfNextFIx, nextOfNextOffset, node.getFrameIx(), node.getOffset(), frames);
- }
- BSTNodeUtil.setHeaderFooter(nextFix, nextOff, length, false, frames);
- result.set(nextFix, nextOff);
- return;
- }
-
- fixTreePtrs(node.getFrameIx(), node.getOffset(), parent.getFrameIx(), parent.getOffset());
- BSTNodeUtil.setHeaderFooter(node.getFrameIx(), node.getOffset(), length, false, frames);
- result.copy(node);
- }
-
- private void removeFromList(int fix, int off) {
- int nextFIx = BSTNodeUtil.getNextFrameIx(fix, off, frames, convertBuffer);
- int nextOffset = BSTNodeUtil.getNextOffset(fix, off, frames, convertBuffer);
- int prevFIx = BSTNodeUtil.getPrevFrameIx(fix, off, frames, convertBuffer);
- int prevOffset = BSTNodeUtil.getPrevOffset(fix, off, frames, convertBuffer);
- if (!isNodeNull(prevFIx, prevOffset) && !isNodeNull(nextFIx, nextOffset)) {
- BSTNodeUtil.setNext(prevFIx, prevOffset, nextFIx, nextOffset, frames);
- BSTNodeUtil.setPrev(nextFIx, nextOffset, prevFIx, prevOffset, frames);
- BSTNodeUtil.setNext(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- BSTNodeUtil.setPrev(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- return;
- }
- if (!isNodeNull(prevFIx, prevOffset)) {
- BSTNodeUtil.setNext(prevFIx, prevOffset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- BSTNodeUtil.setPrev(fix, off, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- return;
- }
-
- // We need to find the parent, so we can fix the tree
- int parentFIx = BSTNodeUtil.INVALID_INDEX;
- int parentOffset = BSTNodeUtil.INVALID_INDEX;
- int length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(fix, off, frames, convertBuffer));
- fix = root.getFrameIx();
- off = root.getOffset();
- int curLen = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
- while (length != curLen) {
- parentFIx = fix;
- parentOffset = off;
- if (length < curLen) {
- fix = BSTNodeUtil.getLeftChildFrameIx(parentFIx, parentOffset, frames, convertBuffer); // parentFIx
- // is
- // now
- // the
- // old(current)
- // fix
- off = BSTNodeUtil.getLeftChildOffset(parentFIx, parentOffset, frames, convertBuffer); // parentOffset
- // is
- // now
- // the
- // old(current)
- // off
- } else {
- fix = BSTNodeUtil.getRightChildFrameIx(parentFIx, parentOffset, frames, convertBuffer); // parentFIx
- // is
- // now
- // the
- // old(current)
- // fix
- off = BSTNodeUtil.getRightChildOffset(parentFIx, parentOffset, frames, convertBuffer); // parentOffset
- // is
- // now
- // the
- // old(current)
- // off
- }
- curLen = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
- }
-
- if (!isNodeNull(nextFIx, nextOffset)) { // it is head of the list (in
- // the
- // tree)
- BSTNodeUtil.setPrev(nextFIx, nextOffset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- int nodeLeftChildFIx = BSTNodeUtil.getLeftChildFrameIx(fix, off, frames, convertBuffer);
- int nodeLeftChildOffset = BSTNodeUtil.getLeftChildOffset(fix, off, frames, convertBuffer);
- int nodeRightChildFix = BSTNodeUtil.getRightChildFrameIx(fix, off, frames, convertBuffer);
- int nodeRightChildOffset = BSTNodeUtil.getRightChildOffset(fix, off, frames, convertBuffer);
- BSTNodeUtil.setLeftChild(nextFIx, nextOffset, nodeLeftChildFIx, nodeLeftChildOffset, frames);
- BSTNodeUtil.setRightChild(nextFIx, nextOffset, nodeRightChildFix, nodeRightChildOffset, frames);
- if (!isNodeNull(parentFIx, parentOffset)) {
- int parentLength = BSTNodeUtil.getLength(parentFIx, parentOffset, frames, convertBuffer);
- if (length < parentLength) {
- BSTNodeUtil.setLeftChild(parentFIx, parentOffset, nextFIx, nextOffset, frames);
- } else {
- BSTNodeUtil.setRightChild(parentFIx, parentOffset, nextFIx, nextOffset, frames);
- }
- }
-
- if ((root.getFrameIx() == fix) && (root.getOffset() == off)) {
- root.set(nextFIx, nextOffset);
- }
-
- return;
- }
-
- fixTreePtrs(fix, off, parentFIx, parentOffset);
- }
-
- private void clear(Slot[] s) {
- s[0].clear();
- s[1].clear();
- }
-
- private boolean isNodeNull(int frameIx, int offset) {
- return ((frameIx == BSTNodeUtil.INVALID_INDEX) || (offset == BSTNodeUtil.INVALID_INDEX) || (frames[frameIx] == null));
- }
-
- private boolean shouldSplit(int slotLength, int reqLength) {
- return ((slotLength - reqLength) >= BSTNodeUtil.MINIMUM_FREE_SLOT_SIZE);
- }
-
- private void initNewNode(int frameIx, int offset) {
- BSTNodeUtil.setLeftChild(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- BSTNodeUtil.setRightChild(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- BSTNodeUtil.setNext(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- BSTNodeUtil.setPrev(frameIx, offset, BSTNodeUtil.INVALID_INDEX, BSTNodeUtil.INVALID_INDEX, frames);
- }
-
- private ByteBuffer allocateFrame() throws HyracksDataException {
- return ctx.allocateFrame();
- }
-
- public String debugPrintMemory() {
- Slot s = new Slot(0, 0);
- if (s.isNull()) {
- return "memory:\tNull";
- }
-
- String m = "memory:\n" + debugPrintSlot(0, 0) + "\n";
- int length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(0, 0, frames, convertBuffer));
- int noff = (length + 2 * BSTNodeUtil.HEADER_SIZE >= frameSize ? BSTNodeUtil.INVALID_INDEX : length + 2
- * BSTNodeUtil.HEADER_SIZE);
- int nfix = (noff == BSTNodeUtil.INVALID_INDEX ? ((frames.length == 1) ? BSTNodeUtil.INVALID_INDEX : 1) : 0);
- if (noff == BSTNodeUtil.INVALID_INDEX && nfix != BSTNodeUtil.INVALID_INDEX) {
- noff = 0;
- }
- s.set(nfix, noff);
- while (!isNodeNull(s.getFrameIx(), s.getOffset())) {
- m += debugPrintSlot(s.getFrameIx(), s.getOffset()) + "\n";
- length = BSTNodeUtil.getActualLength(BSTNodeUtil.getLength(s.getFrameIx(), s.getOffset(), frames,
- convertBuffer));
- noff = (s.getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE >= frameSize ? BSTNodeUtil.INVALID_INDEX : s
- .getOffset() + length + 2 * BSTNodeUtil.HEADER_SIZE);
- nfix = (noff == BSTNodeUtil.INVALID_INDEX ? ((frames.length - 1 == s.getFrameIx()) ? BSTNodeUtil.INVALID_INDEX
- : s.getFrameIx() + 1)
- : s.getFrameIx());
- if (noff == BSTNodeUtil.INVALID_INDEX && nfix != BSTNodeUtil.INVALID_INDEX) {
- noff = 0;
- }
- s.set(nfix, noff);
- }
- return m;
- }
-
- public String debugPrintTree() {
- Slot node = new Slot();
- node.copy(root);
- if (!node.isNull()) {
- return debugPrintSubTree(node);
- }
- return "Null";
- }
-
- private String debugPrintSubTree(Slot r) {
- Slot node = new Slot();
- node.copy(r);
- int fix = node.getFrameIx();
- int off = node.getOffset();
- int lfix = BSTNodeUtil.getLeftChildFrameIx(node, frames, convertBuffer);
- int loff = BSTNodeUtil.getLeftChildOffset(node, frames, convertBuffer);
- int rfix = BSTNodeUtil.getRightChildFrameIx(node, frames, convertBuffer);
- int roff = BSTNodeUtil.getRightChildOffset(node, frames, convertBuffer);
- int nfix = BSTNodeUtil.getNextFrameIx(node, frames, convertBuffer);
- int noff = BSTNodeUtil.getNextOffset(node, frames, convertBuffer);
- int pfix = BSTNodeUtil.getPrevFrameIx(node, frames, convertBuffer);
- int poff = BSTNodeUtil.getPrevOffset(node, frames, convertBuffer);
-
- String s = "{" + r.getFrameIx() + ", " + r.getOffset() + " (Len: "
- + BSTNodeUtil.getLength(fix, off, frames, convertBuffer) + ") - " + "(LC: "
- + debugPrintSlot(lfix, loff) + ") - " + "(RC: " + debugPrintSlot(rfix, roff) + ") - " + "(NX: "
- + debugPrintSlot(nfix, noff) + ") - " + "(PR: " + debugPrintSlot(pfix, poff) + ") }\n";
- if (!isNodeNull(lfix, loff)) {
- s += debugPrintSubTree(new Slot(lfix, loff)) + "\n";
- }
- if (!isNodeNull(rfix, roff)) {
- s += debugPrintSubTree(new Slot(rfix, roff)) + "\n";
- }
-
- return s;
- }
-
- private String debugPrintSlot(int fix, int off) {
- if (isNodeNull(fix, off)) {
- return BSTNodeUtil.INVALID_INDEX + ", " + BSTNodeUtil.INVALID_INDEX;
- }
- int l = BSTNodeUtil.getLength(fix, off, frames, convertBuffer);
- int al = BSTNodeUtil.getActualLength(l);
- boolean f = BSTNodeUtil.isFree(fix, off, frames);
- return fix + ", " + off + " (free: " + f + ") (Len: " + l + ") (actual len: " + al + ") ";
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
deleted file mode 100644
index 8a1bcd3..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/BSTNodeUtil.java
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-/**
- * @author pouria
- * Implements utility methods, used extensively and repeatedly within
- * the BSTMemMgr.
- * Mainly includes methods to set/get different types of pointers,
- * required and accessed within BST traversal, along with the methods
- * for setting/getting length/header/footer of free slots, which have
- * been used as the containers for BST nodes.
- */
-public class BSTNodeUtil {
-
- static final int MINIMUM_FREE_SLOT_SIZE = 32;
-
- private static final int FRAME_PTR_SIZE = 4;
- private static final int OFFSET_SIZE = 2;
-
- static final int HEADER_SIZE = 2;
- private static final int HEADER_INDEX = 0;
-
- private static final int LEFT_CHILD_FRAME_INDEX = HEADER_INDEX + HEADER_SIZE;
- private static final int LEFT_CHILD_OFFSET_INDEX = LEFT_CHILD_FRAME_INDEX + FRAME_PTR_SIZE;
-
- private static final int RIGHT_CHILD_FRAME_INDEX = LEFT_CHILD_OFFSET_INDEX + OFFSET_SIZE;
- private static final int RIGHT_CHILD_OFFSET_INDEX = RIGHT_CHILD_FRAME_INDEX + FRAME_PTR_SIZE;
-
- private static final int NEXT_FRAME_INDEX = RIGHT_CHILD_OFFSET_INDEX + OFFSET_SIZE;
- private static final int NEXT_OFFSET_INDEX = NEXT_FRAME_INDEX + FRAME_PTR_SIZE;
-
- private static final int PREV_FRAME_INDEX = NEXT_OFFSET_INDEX + OFFSET_SIZE;
- private static final int PREV_OFFSET_INDEX = PREV_FRAME_INDEX + FRAME_PTR_SIZE;
-
- private static final byte INVALID = -128;
- private static final byte MASK = 127;
- static final int INVALID_INDEX = -1;
-
- /*
- * Structure of a free slot:
- * [HEADER][LEFT_CHILD][RIGHT_CHILD][NEXT][PREV]...[FOOTER] MSB in the
- * HEADER is set to 1 in a free slot
- *
- * Structure of a used slot: [HEADER]...[FOOTER] MSB in the HEADER is set to
- * 0 in a used slot
- */
-
- static int getLeftChildFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getLeftChildFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getLeftChildOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getLeftChildOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getLeftChildFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + LEFT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
-
- }
-
- static int getLeftChildOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + LEFT_CHILD_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
- }
-
- static void setLeftChild(Slot node, Slot lc, ByteBuffer[] frames) {
- setLeftChild(node.getFrameIx(), node.getOffset(), lc.getFrameIx(), lc.getOffset(), frames);
- }
-
- static void setLeftChild(int nodeFix, int nodeOff, int lcFix, int lcOff, ByteBuffer[] frames) {
- storeInt(frames[nodeFix], nodeOff + LEFT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, lcFix);
- storeInt(frames[nodeFix], nodeOff + LEFT_CHILD_OFFSET_INDEX, OFFSET_SIZE, lcOff);
- }
-
- static int getRightChildFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getRightChildFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getRightChildOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getRightChildOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getRightChildFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + RIGHT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
- }
-
- static int getRightChildOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + RIGHT_CHILD_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
- }
-
- static void setRightChild(Slot node, Slot rc, ByteBuffer[] frames) {
- setRightChild(node.getFrameIx(), node.getOffset(), rc.getFrameIx(), rc.getOffset(), frames);
- }
-
- static void setRightChild(int nodeFix, int nodeOff, int rcFix, int rcOff, ByteBuffer[] frames) {
- storeInt(frames[nodeFix], nodeOff + RIGHT_CHILD_FRAME_INDEX, FRAME_PTR_SIZE, rcFix);
- storeInt(frames[nodeFix], nodeOff + RIGHT_CHILD_OFFSET_INDEX, OFFSET_SIZE, rcOff);
- }
-
- static int getNextFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getNextFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getNextOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getNextOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getNextFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + NEXT_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
- }
-
- static int getNextOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + NEXT_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
- }
-
- static void setNext(Slot node, Slot next, ByteBuffer[] frames) {
- setNext(node.getFrameIx(), node.getOffset(), next.getFrameIx(), node.getOffset(), frames);
- }
-
- static void setNext(int nodeFix, int nodeOff, int nFix, int nOff, ByteBuffer[] frames) {
- storeInt(frames[nodeFix], nodeOff + NEXT_FRAME_INDEX, FRAME_PTR_SIZE, nFix);
- storeInt(frames[nodeFix], nodeOff + NEXT_OFFSET_INDEX, OFFSET_SIZE, nOff);
- }
-
- static int getPrevFrameIx(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getPrevFrameIx(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getPrevOffset(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getPrevOffset(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getPrevFrameIx(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + PREV_FRAME_INDEX, FRAME_PTR_SIZE, convertBuffer));
- }
-
- static int getPrevOffset(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return (retrieveAsInt(frames[frameIx], offset + PREV_OFFSET_INDEX, OFFSET_SIZE, convertBuffer));
- }
-
- static void setPrev(Slot node, Slot prev, ByteBuffer[] frames) {
- setPrev(node.getFrameIx(), node.getOffset(), prev.getFrameIx(), prev.getOffset(), frames);
- }
-
- static void setPrev(int nodeFix, int nodeOff, int pFix, int pOff, ByteBuffer[] frames) {
- storeInt(frames[nodeFix], nodeOff + PREV_FRAME_INDEX, FRAME_PTR_SIZE, pFix);
- storeInt(frames[nodeFix], nodeOff + PREV_OFFSET_INDEX, OFFSET_SIZE, pOff);
- }
-
- static boolean slotsTheSame(Slot s, Slot t) {
- return ((s.getFrameIx() == t.getFrameIx()) && (s.getOffset() == t.getOffset()));
- }
-
- static void setHeaderFooter(int frameIx, int offset, int usedLength, boolean isFree, ByteBuffer[] frames) {
- int slotLength = getActualLength(usedLength);
- int footerOffset = offset + HEADER_SIZE + slotLength;
- storeInt(frames[frameIx], offset, HEADER_SIZE, usedLength);
- storeInt(frames[frameIx], footerOffset, HEADER_SIZE, usedLength);
- setFree(frameIx, offset, isFree, frames);
- setFree(frameIx, footerOffset, isFree, frames);
- }
-
- static int getLength(Slot s, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- return getLength(s.getFrameIx(), s.getOffset(), frames, convertBuffer);
- }
-
- static int getLength(int frameIx, int offset, ByteBuffer[] frames, ByteBuffer convertBuffer) {
- convertBuffer.clear();
- for (int i = 0; i < 4 - HEADER_SIZE; i++) { // padding
- convertBuffer.put(i, (byte) 0x00);
- }
-
- convertBuffer.put(4 - HEADER_SIZE, (byte) ((frames[frameIx].get(offset)) & (MASK)));
- System.arraycopy(frames[frameIx].array(), offset + 1, convertBuffer.array(), 5 - HEADER_SIZE, HEADER_SIZE - 1);
- return convertBuffer.getInt(0);
- }
-
- // MSB equal to 1 means FREE
- static boolean isFree(int frameIx, int offset, ByteBuffer[] frames) {
- return ((((frames[frameIx]).array()[offset]) & 0x80) == 0x80);
- }
-
- static void setFree(int frameIx, int offset, boolean free, ByteBuffer[] frames) {
- if (free) { // set MSB to 1 (for free)
- frames[frameIx].put(offset, (byte) (((frames[frameIx]).array()[offset]) | 0x80));
- } else { // set MSB to 0 (for used)
- frames[frameIx].put(offset, (byte) (((frames[frameIx]).array()[offset]) & 0x7F));
- }
- }
-
- static int getActualLength(int l) {
- int r = (l + 2 * HEADER_SIZE) % MINIMUM_FREE_SLOT_SIZE;
- return (r == 0 ? l : (l + (BSTNodeUtil.MINIMUM_FREE_SLOT_SIZE - r)));
- }
-
- private static int retrieveAsInt(ByteBuffer b, int fromIndex, int size, ByteBuffer convertBuffer) {
- if ((b.get(fromIndex) & INVALID) == INVALID) {
- return INVALID_INDEX;
- }
-
- convertBuffer.clear();
- for (int i = 0; i < 4 - size; i++) { // padding
- convertBuffer.put(i, (byte) 0x00);
- }
-
- System.arraycopy(b.array(), fromIndex, convertBuffer.array(), 4 - size, size);
- return convertBuffer.getInt(0);
- }
-
- private static void storeInt(ByteBuffer b, int fromIndex, int size, int value) {
- if (value == INVALID_INDEX) {
- b.put(fromIndex, INVALID);
- return;
- }
- for (int i = 0; i < size; i++) {
- b.put(fromIndex + i, (byte) ((value >>> (8 * ((size - 1 - i)))) & 0xff));
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index e1315e7..3dc8b41 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -1,64 +1,42 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
+ * 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.
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.*;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
-public class ExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
+import java.util.List;
+
+public class ExternalSortOperatorDescriptor extends AbstractSorterOperatorDescriptor {
+
private static final long serialVersionUID = 1L;
- private static final int SORT_ACTIVITY_ID = 0;
- private static final int MERGE_ACTIVITY_ID = 1;
-
- private final int[] sortFields;
- private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
- private final IBinaryComparatorFactory[] comparatorFactories;
- private final int framesLimit;
-
private Algorithm alg = Algorithm.MERGE_SORT;
+ private EnumFreeSlotPolicy policy = EnumFreeSlotPolicy.LAST_FIT;
+ private final int outputLimit;
public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor, Algorithm alg) {
- this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
- this.alg = alg;
+ this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor, alg,
+ EnumFreeSlotPolicy.LAST_FIT);
}
public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
@@ -69,127 +47,52 @@
public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor) {
- super(spec, 1, 1);
- this.framesLimit = framesLimit;
- this.sortFields = sortFields;
- this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
- this.comparatorFactories = comparatorFactories;
- if (framesLimit <= 1) {
- throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
- }
- recordDescriptors[0] = recordDescriptor;
+ this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ Algorithm.MERGE_SORT, EnumFreeSlotPolicy.LAST_FIT);
}
@Override
- public void contributeActivities(IActivityGraphBuilder builder) {
- SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
- MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
- builder.addActivity(this, sa);
- builder.addSourceEdge(0, sa, 0);
-
- builder.addActivity(this, ma);
- builder.addTargetEdge(0, ma, 0);
-
- builder.addBlockingEdge(sa, ma);
+ public AbstractSorterOperatorDescriptor.SortActivity getSortActivity(ActivityId id) {
+ return new AbstractSorterOperatorDescriptor.SortActivity(id) {
+ @Override
+ protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider) throws HyracksDataException {
+ return new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
+ comparatorFactories, recordDescriptors[0], alg, policy, framesLimit, outputLimit);
+ }
+ };
}
- public static class SortTaskState extends AbstractStateObject {
- private List<IFrameReader> runs;
- private IFrameSorter frameSorter;
-
- public SortTaskState() {
- }
-
- private SortTaskState(JobId jobId, TaskId taskId) {
- super(jobId, taskId);
- }
-
- @Override
- public void toBytes(DataOutput out) throws IOException {
-
- }
-
- @Override
- public void fromBytes(DataInput in) throws IOException {
-
- }
+ @Override
+ public AbstractSorterOperatorDescriptor.MergeActivity getMergeActivity(ActivityId id) {
+ return new AbstractSorterOperatorDescriptor.MergeActivity(id) {
+ @Override
+ protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+ INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+ return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
+ nmkComputer, recordDescriptors[0], necessaryFrames, outputLimit, writer);
+ }
+ };
}
- private class SortActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public SortActivity(ActivityId id) {
- super(id);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
- private ExternalSortRunGenerator runGen;
-
- @Override
- public void open() throws HyracksDataException {
- runGen = new ExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
- comparatorFactories, recordDescriptors[0], alg, framesLimit);
- runGen.open();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- runGen.nextFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- SortTaskState state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(
- getActivityId(), partition));
- runGen.close();
- state.runs = runGen.getRuns();
- state.frameSorter = runGen.getFrameSorter();
- ctx.setStateObject(state);
- }
-
- @Override
- public void fail() throws HyracksDataException {
- runGen.fail();
- }
- };
- return op;
- }
+ public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, Algorithm alg, EnumFreeSlotPolicy policy) {
+ this(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor, alg,
+ policy, Integer.MAX_VALUE);
}
- private class MergeActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public MergeActivity(ActivityId id) {
- super(id);
+ public ExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, Algorithm alg, EnumFreeSlotPolicy policy, int outputLimit) {
+ super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+ if (framesLimit <= 1) {
+ throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
}
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
- @Override
- public void initialize() throws HyracksDataException {
- SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
- SORT_ACTIVITY_ID), partition));
- List<IFrameReader> runs = state.runs;
- IFrameSorter frameSorter = state.frameSorter;
- IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
- : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- int necessaryFrames = Math.min(runs.size() + 2, framesLimit);
- ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, frameSorter, runs, sortFields,
- comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
- merger.process();
- }
- };
- return op;
- }
+ this.alg = alg;
+ this.policy = policy;
+ this.outputLimit = outputLimit;
}
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
index 3736fca..4048726 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunGenerator.java
@@ -1,24 +1,21 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
+ * 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.
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -27,77 +24,89 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotSmallestFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameFreeSlotPolicy;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
-public class ExternalSortRunGenerator implements IFrameWriter {
- private final IHyracksTaskContext ctx;
- private final IFrameSorter frameSorter;
- private final List<IFrameReader> runs;
- private final int maxSortFrames;
+public class ExternalSortRunGenerator extends AbstractSortRunGenerator {
+
+ protected final IHyracksTaskContext ctx;
+ protected final IFrameSorter frameSorter;
+ protected final int maxSortFrames;
public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDesc, Algorithm alg, int framesLimit) throws HyracksDataException {
- this.ctx = ctx;
- if (alg == Algorithm.MERGE_SORT) {
- frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
- recordDesc);
- } else {
- frameSorter = new FrameSorterQuickSort(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories,
- recordDesc);
- }
- runs = new LinkedList<IFrameReader>();
- maxSortFrames = framesLimit - 1;
+ this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg,
+ EnumFreeSlotPolicy.LAST_FIT, framesLimit);
}
- @Override
- public void open() throws HyracksDataException {
- runs.clear();
- frameSorter.reset();
+ public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit)
+ throws HyracksDataException {
+ this(ctx, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDesc, alg, policy, framesLimit,
+ Integer.MAX_VALUE);
+ }
+
+ public ExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDesc, Algorithm alg, EnumFreeSlotPolicy policy, int framesLimit, int outputLimit)
+ throws HyracksDataException {
+ this.ctx = ctx;
+ maxSortFrames = framesLimit - 1;
+
+ IFrameFreeSlotPolicy freeSlotPolicy = null;
+ switch (policy) {
+ case SMALLEST_FIT:
+ freeSlotPolicy = new FrameFreeSlotSmallestFit();
+ break;
+ case LAST_FIT:
+ freeSlotPolicy = new FrameFreeSlotLastFit(maxSortFrames);
+ break;
+ case BIGGEST_FIT:
+ freeSlotPolicy = new FrameFreeSlotBiggestFirst(maxSortFrames);
+ break;
+ }
+ IFrameBufferManager bufferManager = new VariableFrameMemoryManager(
+ new VariableFramePool(ctx, maxSortFrames * ctx.getInitialFrameSize()), freeSlotPolicy);
+ if (alg == Algorithm.MERGE_SORT) {
+ frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+ comparatorFactories, recordDesc, outputLimit);
+ } else {
+ frameSorter = new FrameSorterQuickSort(ctx, bufferManager, sortFields, firstKeyNormalizerFactory,
+ comparatorFactories, recordDesc, outputLimit);
+ }
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- if (frameSorter.getFrameCount() >= maxSortFrames) {
+ if (!frameSorter.insertFrame(buffer)) {
flushFramesToRun();
- }
- frameSorter.insertFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- if (frameSorter.getFrameCount() > 0) {
- if (runs.size() <= 0) {
- frameSorter.sortFrames();
- } else {
- flushFramesToRun();
+ if (!frameSorter.insertFrame(buffer)) {
+ throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
}
}
}
- private void flushFramesToRun() throws HyracksDataException {
- frameSorter.sortFrames();
+ protected RunFileWriter getRunFileWriter() throws HyracksDataException {
FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
ExternalSortRunGenerator.class.getSimpleName());
- RunFileWriter writer = new RunFileWriter(file, ctx.getIOManager());
- writer.open();
- try {
- frameSorter.flushFrames(writer);
- } finally {
- writer.close();
- }
- frameSorter.reset();
- runs.add(writer.createReader());
+ return new RunFileWriter(file, ctx.getIOManager());
+ }
+
+ protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
+ return writer;
}
@Override
- public void fail() throws HyracksDataException {
- }
-
- public IFrameSorter getFrameSorter() {
+ public ISorter getSorter() {
return frameSorter;
}
- public List<IFrameReader> getRuns() {
- return runs;
- }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index 9178094..6b36480 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -1,318 +1,271 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
+ * 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.
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
+import java.util.BitSet;
import java.util.LinkedList;
import java.util.List;
+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.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileReader;
import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria This class defines the logic for merging the run, generated
- * during the first phase of external sort (for both sorting without
- * replacement selection and with it). For the case with replacement
- * selection, this code also takes the limit on the output into account
- * (if specified). If number of input runs is less than the available
- * memory frames, then merging can be done in one pass, by allocating
- * one buffer per run, and one buffer as the output buffer. A
- * priorityQueue is used to find the top tuple at each iteration, among
- * all the runs' heads in memory (check RunMergingFrameReader for more
- * details). Otherwise, assuming that we have R runs and M memory
- * buffers, where (R > M), we first merge first (M-1) runs and create a
- * new sorted run, out of them. Discarding the first (M-1) runs, now
- * merging procedure gets applied recursively on the (R-M+2) remaining
- * runs using the M memory buffers. For the case of replacement
- * selection, if outputLimit is specified, once the final pass is done
- * on the runs (which is the pass that generates the final sorted
- * output), as soon as the output size hits the output limit, the
- * process stops, closes, and returns.
- */
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
public class ExternalSortRunMerger {
- private final IHyracksTaskContext ctx;
- private final List<IFrameReader> runs;
+ protected final IHyracksTaskContext ctx;
+ protected final IFrameWriter writer;
+
+ private final List<RunAndMaxFrameSizePair> runs;
+ private final BitSet currentGenerationRunAvailable;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
private final INormalizedKeyComputer nmkComputer;
private final RecordDescriptor recordDesc;
private final int framesLimit;
- private final IFrameWriter writer;
- private List<ByteBuffer> inFrames;
- private ByteBuffer outFrame;
- private FrameTupleAppender outFrameAppender;
+ private final int MAX_FRAME_SIZE;
+ private final ArrayList<IFrameReader> tempRuns;
+ private final int topK;
+ private List<GroupVSizeFrame> inFrames;
+ private VSizeFrame outputFrame;
+ private ISorter sorter;
- private IFrameSorter frameSorter; // Used in External sort, no replacement
- // selection
- private FrameTupleAccessor outFrameAccessor; // Used in External sort, with
- // replacement selection
- private final int outputLimit; // Used in External sort, with replacement
- // selection and limit on output size
- private int currentSize; // Used in External sort, with replacement
- // selection and limit on output size
+ private static final Logger LOGGER = Logger.getLogger(ExternalSortRunMerger.class.getName());
- // Constructor for external sort, no replacement selection
- public ExternalSortRunMerger(IHyracksTaskContext ctx, IFrameSorter frameSorter, List<IFrameReader> runs,
+ public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
RecordDescriptor recordDesc, int framesLimit, IFrameWriter writer) {
- this.ctx = ctx;
- this.frameSorter = frameSorter;
- this.runs = new LinkedList<IFrameReader>(runs);
- this.sortFields = sortFields;
- this.comparators = comparators;
- this.nmkComputer = nmkComputer;
- this.recordDesc = recordDesc;
- this.framesLimit = framesLimit;
- this.writer = writer;
- this.outputLimit = -1;
+ this(ctx, sorter, runs, sortFields, comparators, nmkComputer, recordDesc, framesLimit,
+ Integer.MAX_VALUE, writer);
}
- // Constructor for external sort with replacement selection
- public ExternalSortRunMerger(IHyracksTaskContext ctx, int outputLimit, List<IFrameReader> runs, int[] sortFields,
- IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc,
- int framesLimit, IFrameWriter writer) {
+ public ExternalSortRunMerger(IHyracksTaskContext ctx, ISorter sorter, List<RunAndMaxFrameSizePair> runs,
+ int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
+ RecordDescriptor recordDesc, int framesLimit, int topK, IFrameWriter writer) {
this.ctx = ctx;
- this.runs = new LinkedList<IFrameReader>(runs);
+ this.sorter = sorter;
+ this.runs = new LinkedList<>(runs);
+ this.currentGenerationRunAvailable = new BitSet(runs.size());
this.sortFields = sortFields;
this.comparators = comparators;
this.nmkComputer = nmkComputer;
this.recordDesc = recordDesc;
this.framesLimit = framesLimit;
this.writer = writer;
- this.outputLimit = outputLimit;
- this.currentSize = 0;
- this.frameSorter = null;
+ this.MAX_FRAME_SIZE = FrameConstants.MAX_NUM_MINFRAME * ctx.getInitialFrameSize();
+ this.topK = topK;
+ this.tempRuns = new ArrayList<>(runs.size());
}
public void process() throws HyracksDataException {
- writer.open();
+ IFrameWriter finalWriter = null;
try {
if (runs.size() <= 0) {
- if (frameSorter != null && frameSorter.getFrameCount() > 0) {
- frameSorter.flushFrames(writer);
+ finalWriter = prepareSkipMergingFinalResultWriter(writer);
+ finalWriter.open();
+ if (sorter != null) {
+ if (sorter.hasRemaining()) {
+ sorter.flush(finalWriter);
+ }
+ sorter.close();
}
- /** recycle sort buffer */
- frameSorter.close();
} else {
/** recycle sort buffer */
- frameSorter.close();
+ if (sorter != null) {
+ sorter.close();
+ }
- inFrames = new ArrayList<ByteBuffer>();
- outFrame = ctx.allocateFrame();
- outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
- outFrameAppender.reset(outFrame, true);
- for (int i = 0; i < framesLimit - 1; ++i) {
- inFrames.add(ctx.allocateFrame());
- }
+ finalWriter = prepareFinalMergeResultWriter(writer);
+ finalWriter.open();
+
int maxMergeWidth = framesLimit - 1;
- while (runs.size() > maxMergeWidth) {
- int generationSeparator = 0;
- while (generationSeparator < runs.size() && runs.size() > maxMergeWidth) {
- int mergeWidth = Math.min(Math.min(runs.size() - generationSeparator, maxMergeWidth),
- runs.size() - maxMergeWidth + 1);
- FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class
- .getSimpleName());
- IFrameWriter mergeResultWriter = new RunFileWriter(newRun, ctx.getIOManager());
- mergeResultWriter.open();
- IFrameReader[] runCursors = new RunFileReader[mergeWidth];
- for (int i = 0; i < mergeWidth; i++) {
- runCursors[i] = runs.get(generationSeparator + i);
+
+ inFrames = new ArrayList<>(maxMergeWidth);
+ outputFrame = new VSizeFrame(ctx);
+ List<RunAndMaxFrameSizePair> partialRuns = new ArrayList<>(maxMergeWidth);
+
+ int stop = runs.size();
+ currentGenerationRunAvailable.set(0, stop);
+
+ while (true) {
+
+ int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
+ currentGenerationRunAvailable,
+ stop);
+ prepareFrames(unUsed, inFrames, partialRuns);
+
+ if (!currentGenerationRunAvailable.isEmpty() || stop < runs.size()) {
+ IFrameReader reader;
+ int mergedMaxFrameSize;
+ if (partialRuns.size() == 1) {
+ if (!currentGenerationRunAvailable.isEmpty()) {
+ throw new HyracksDataException(
+ "The record is too big to put into the merging frame, please"
+ + " allocate more sorting memory");
+ } else {
+ reader = partialRuns.get(0).run;
+ mergedMaxFrameSize = partialRuns.get(0).maxFrameSize;
+ }
+
+ } else {
+ RunFileWriter mergeFileWriter = prepareIntermediateMergeRunFile();
+ IFrameWriter mergeResultWriter = prepareIntermediateMergeResultWriter(mergeFileWriter);
+
+ mergeResultWriter.open();
+ mergedMaxFrameSize = merge(mergeResultWriter, partialRuns);
+ mergeResultWriter.close();
+
+ reader = mergeFileWriter.createReader();
}
- merge(mergeResultWriter, runCursors);
- mergeResultWriter.close();
- runs.subList(generationSeparator, mergeWidth + generationSeparator).clear();
- runs.add(generationSeparator++, ((RunFileWriter) mergeResultWriter).createReader());
+
+ appendNewRuns(reader, mergedMaxFrameSize);
+ if (currentGenerationRunAvailable.isEmpty()) {
+
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("generated runs:" + stop);
+ }
+ runs.subList(0, stop).clear();
+ currentGenerationRunAvailable.clear();
+ currentGenerationRunAvailable.set(0, runs.size());
+ stop = runs.size();
+ }
+ } else {
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("final runs:" + stop);
+ }
+ merge(finalWriter, partialRuns);
+ break;
}
}
- if (!runs.isEmpty()) {
- IFrameReader[] runCursors = new RunFileReader[runs.size()];
- for (int i = 0; i < runCursors.length; i++) {
- runCursors[i] = runs.get(i);
- }
- merge(writer, runCursors);
- }
}
} catch (Exception e) {
- writer.fail();
+ finalWriter.fail();
throw new HyracksDataException(e);
} finally {
- writer.close();
+ finalWriter.close();
}
}
- private void merge(IFrameWriter mergeResultWriter, IFrameReader[] runCursors) throws HyracksDataException {
- RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields, comparators,
- nmkComputer, recordDesc);
+ private void appendNewRuns(IFrameReader reader, int mergedPartialMaxSize) {
+ runs.add(new RunAndMaxFrameSizePair(reader, mergedPartialMaxSize));
+ }
+
+ private static int selectPartialRuns(int budget, List<RunAndMaxFrameSizePair> runs,
+ List<RunAndMaxFrameSizePair> partialRuns, BitSet runAvailable, int stop) {
+ partialRuns.clear();
+ int maxFrameSizeOfGenRun = 0;
+ int nextRunId = runAvailable.nextSetBit(0);
+ while (budget > 0 && nextRunId >= 0 && nextRunId < stop) {
+ int runFrameSize = runs.get(nextRunId).maxFrameSize;
+ if (budget - runFrameSize >= 0) {
+ partialRuns.add(runs.get(nextRunId));
+ budget -= runFrameSize;
+ runAvailable.clear(nextRunId);
+ maxFrameSizeOfGenRun = runFrameSize > maxFrameSizeOfGenRun ? runFrameSize : maxFrameSizeOfGenRun;
+ }
+ nextRunId = runAvailable.nextSetBit(nextRunId + 1);
+ }
+ return budget;
+ }
+
+ private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames,
+ List<RunAndMaxFrameSizePair> patialRuns)
+ throws HyracksDataException {
+ if (extraFreeMem > 0 && patialRuns.size() > 1) {
+ int extraFrames = extraFreeMem / ctx.getInitialFrameSize();
+ int avg = (extraFrames / patialRuns.size()) * ctx.getInitialFrameSize();
+ int residue = (extraFrames % patialRuns.size());
+ for (int i = 0; i < residue; i++) {
+ patialRuns.get(i).updateSize(
+ Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg + ctx.getInitialFrameSize()));
+ }
+ for (int i = residue; i < patialRuns.size() && avg > 0; i++) {
+ patialRuns.get(i).updateSize(Math.min(MAX_FRAME_SIZE, patialRuns.get(i).maxFrameSize + avg));
+ }
+ }
+
+ if (inFrames.size() > patialRuns.size()) {
+ inFrames.subList(patialRuns.size(), inFrames.size()).clear();
+ }
+ int i;
+ for (i = 0; i < inFrames.size(); i++) {
+ inFrames.get(i).resize(patialRuns.get(i).maxFrameSize);
+ }
+ for (; i < patialRuns.size(); i++) {
+ inFrames.add(new GroupVSizeFrame(ctx, patialRuns.get(i).maxFrameSize));
+ }
+ }
+
+ protected IFrameWriter prepareSkipMergingFinalResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+ return nextWriter;
+ }
+
+ protected RunFileWriter prepareIntermediateMergeRunFile() throws HyracksDataException {
+ FileReference newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
+ return new RunFileWriter(newRun, ctx.getIOManager());
+ }
+
+ protected IFrameWriter prepareIntermediateMergeResultWriter(RunFileWriter mergeFileWriter)
+ throws HyracksDataException {
+ return mergeFileWriter;
+ }
+
+ protected IFrameWriter prepareFinalMergeResultWriter(IFrameWriter nextWriter) throws HyracksDataException {
+ return nextWriter;
+ }
+
+ protected int[] getSortFields() {
+ return sortFields;
+ }
+
+ private int merge(IFrameWriter writer, List<RunAndMaxFrameSizePair> partialRuns)
+ throws HyracksDataException {
+ tempRuns.clear();
+ for (int i = 0; i < partialRuns.size(); i++) {
+ tempRuns.add(partialRuns.get(i).run);
+ }
+ RunMergingFrameReader merger = new RunMergingFrameReader(ctx, tempRuns, inFrames, getSortFields(),
+ comparators, nmkComputer, recordDesc, topK);
+ int maxFrameSize = 0;
+ int io = 0;
merger.open();
try {
- while (merger.nextFrame(outFrame)) {
- FrameUtils.flushFrame(outFrame, mergeResultWriter);
+ while (merger.nextFrame(outputFrame)) {
+ FrameUtils.flushFrame(outputFrame.getBuffer(), writer);
+ maxFrameSize = maxFrameSize < outputFrame.getFrameSize() ? outputFrame.getFrameSize() : maxFrameSize;
+ io++;
}
} finally {
merger.close();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Output " + io + " frames");
+ }
}
+ return maxFrameSize;
}
- public void processWithReplacementSelection() throws HyracksDataException {
- writer.open();
- try {
- outFrameAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
- outFrame = ctx.allocateFrame();
- outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
- outFrameAppender.reset(outFrame, true);
- if (runs.size() == 1) {
- if (outputLimit < 1) {
- runs.get(0).open();
- ByteBuffer nextFrame = ctx.allocateFrame();
- while (runs.get(0).nextFrame(nextFrame)) {
- FrameUtils.flushFrame(nextFrame, writer);
- outFrameAppender.reset(nextFrame, true);
- }
- return;
- }
- // Limit on the output size
- int totalCount = 0;
- runs.get(0).open();
- FrameTupleAccessor fta = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
- ByteBuffer nextFrame = ctx.allocateFrame();
- while (totalCount <= outputLimit && runs.get(0).nextFrame(nextFrame)) {
- fta.reset(nextFrame);
- int tupCount = fta.getTupleCount();
- if ((totalCount + tupCount) < outputLimit) {
- FrameUtils.flushFrame(nextFrame, writer);
- totalCount += tupCount;
- continue;
- }
- // The very last buffer, which exceeds the limit
- int copyCount = outputLimit - totalCount;
- outFrameAppender.reset(outFrame, true);
- for (int i = 0; i < copyCount; i++) {
- if (!outFrameAppender.append(fta, i)) {
- throw new HyracksDataException("Record size ("
- + (fta.getTupleEndOffset(i) - fta.getTupleStartOffset(i))
- + ") larger than frame size (" + outFrameAppender.getBuffer().capacity() + ")");
- }
- totalCount++;
- }
- }
- if (outFrameAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- outFrameAppender.reset(outFrame, true);
- }
- return;
- }
- // More than one run, actual merging is needed
- inFrames = new ArrayList<ByteBuffer>();
- for (int i = 0; i < framesLimit - 1; ++i) {
- inFrames.add(ctx.allocateFrame());
- }
- while (runs.size() > 0) {
- try {
- doPassWithReplacementSelection(runs);
- } catch (Exception e) {
- throw new HyracksDataException(e);
- }
- }
-
- } catch (Exception e) {
- writer.fail();
- throw new HyracksDataException(e);
- } finally {
- writer.close();
- }
- }
-
- // creates a new run from runs that can fit in memory.
- private void doPassWithReplacementSelection(List<IFrameReader> runs) throws HyracksDataException {
- FileReference newRun = null;
- IFrameWriter writer = this.writer;
- boolean finalPass = false;
- if (runs.size() + 1 <= framesLimit) { // + 1 outFrame
- finalPass = true;
- for (int i = inFrames.size() - 1; i >= runs.size(); i--) {
- inFrames.remove(i);
- }
- } else {
- newRun = ctx.createManagedWorkspaceFile(ExternalSortRunMerger.class.getSimpleName());
- writer = new RunFileWriter(newRun, ctx.getIOManager());
- writer.open();
- }
- try {
- IFrameReader[] runCursors = new RunFileReader[inFrames.size()];
- for (int i = 0; i < inFrames.size(); i++) {
- runCursors[i] = runs.get(i);
- }
- RunMergingFrameReader merger = new RunMergingFrameReader(ctx, runCursors, inFrames, sortFields,
- comparators, nmkComputer, recordDesc);
- merger.open();
- try {
- while (merger.nextFrame(outFrame)) {
- if (outputLimit > 0 && finalPass) {
- outFrameAccessor.reset(outFrame);
- int count = outFrameAccessor.getTupleCount();
- if ((currentSize + count) > outputLimit) {
- ByteBuffer b = ctx.allocateFrame();
- FrameTupleAppender partialAppender = new FrameTupleAppender(ctx.getFrameSize());
- partialAppender.reset(b, true);
- int copyCount = outputLimit - currentSize;
- for (int i = 0; i < copyCount; i++) {
- partialAppender.append(outFrameAccessor, i);
- currentSize++;
- }
- FrameUtils.makeReadable(b);
- FrameUtils.flushFrame(b, writer);
- break;
- } else {
- FrameUtils.flushFrame(outFrame, writer);
- currentSize += count;
- }
- } else {
- FrameUtils.flushFrame(outFrame, writer);
- }
- }
- } finally {
- merger.close();
- }
-
- if (outputLimit > 0 && finalPass && (currentSize >= outputLimit)) {
- runs.clear();
- return;
- }
-
- runs.subList(0, inFrames.size()).clear();
- if (!finalPass) {
- runs.add(0, ((RunFileWriter) writer).createReader());
- }
- } finally {
- if (!finalPass) {
- writer.close();
- }
- }
- }
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
index 8dbdbd4..82a8453 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
@@ -1,161 +1,69 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ * 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
-public class FrameSorterMergeSort implements IFrameSorter {
- private final IHyracksTaskContext ctx;
- private final int[] sortFields;
- private final INormalizedKeyComputer nkc;
- private final IBinaryComparator[] comparators;
- private final List<ByteBuffer> buffers;
+public class FrameSorterMergeSort extends AbstractFrameSorter {
- private final FrameTupleAccessor fta1;
- private final FrameTupleAccessor fta2;
-
- private final FrameTupleAppender appender;
-
- private final ByteBuffer outFrame;
-
- private int dataFrameCount;
- private int[] tPointers;
private int[] tPointersTemp;
- private int tupleCount;
+ private FrameTupleAccessor fta2;
- public FrameSorterMergeSort(IHyracksTaskContext ctx, int[] sortFields,
+ public FrameSorterMergeSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor) throws HyracksDataException {
- this.ctx = ctx;
- this.sortFields = sortFields;
- nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- buffers = new ArrayList<ByteBuffer>();
- fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- appender = new FrameTupleAppender(ctx.getFrameSize());
- outFrame = ctx.allocateFrame();
+ this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ Integer.MAX_VALUE);
+ }
- dataFrameCount = 0;
+ public FrameSorterMergeSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
+ super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ outputLimit);
+ fta2 = new FrameTupleAccessor(recordDescriptor);
}
@Override
- public void reset() {
- dataFrameCount = 0;
- tupleCount = 0;
- }
-
- @Override
- public int getFrameCount() {
- return dataFrameCount;
- }
-
- @Override
- public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
- ByteBuffer copyFrame;
- if (dataFrameCount == buffers.size()) {
- copyFrame = ctx.allocateFrame();
- buffers.add(copyFrame);
- } else {
- copyFrame = buffers.get(dataFrameCount);
- }
- FrameUtils.copy(buffer, copyFrame);
- ++dataFrameCount;
- }
-
- @Override
- public void sortFrames() throws HyracksDataException {
- int nBuffers = dataFrameCount;
- tupleCount = 0;
- for (int i = 0; i < nBuffers; ++i) {
- fta1.reset(buffers.get(i));
- tupleCount += fta1.getTupleCount();
- }
- int sfIdx = sortFields[0];
- tPointers = tPointers == null || tPointers.length < tupleCount * 4 ? new int[tupleCount * 4] : tPointers;
- int ptr = 0;
- for (int i = 0; i < nBuffers; ++i) {
- fta1.reset(buffers.get(i));
- int tCount = fta1.getTupleCount();
- byte[] array = fta1.getBuffer().array();
- for (int j = 0; j < tCount; ++j) {
- int tStart = fta1.getTupleStartOffset(j);
- int tEnd = fta1.getTupleEndOffset(j);
- tPointers[ptr * 4] = i;
- tPointers[ptr * 4 + 1] = tStart;
- tPointers[ptr * 4 + 2] = tEnd;
- int f0StartRel = fta1.getFieldStartOffset(j, sfIdx);
- int f0EndRel = fta1.getFieldEndOffset(j, sfIdx);
- int f0Start = f0StartRel + tStart + fta1.getFieldSlotsLength();
- tPointers[ptr * 4 + 3] = nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
- ++ptr;
- }
- }
- if (tupleCount > 0) {
+ void sortTupleReferences() throws HyracksDataException {
+ if (tPointersTemp == null || tPointersTemp.length < tPointers.length) {
tPointersTemp = new int[tPointers.length];
- sort(0, tupleCount);
}
+ sort(0, tupleCount);
}
@Override
- public void flushFrames(IFrameWriter writer) throws HyracksDataException {
- appender.reset(outFrame, true);
- for (int ptr = 0; ptr < tupleCount; ++ptr) {
- int i = tPointers[ptr * 4];
- int tStart = tPointers[ptr * 4 + 1];
- int tEnd = tPointers[ptr * 4 + 2];
- ByteBuffer buffer = buffers.get(i);
- fta1.reset(buffer);
- if (!appender.append(fta1, tStart, tEnd)) {
- FrameUtils.flushFrame(outFrame, writer);
- appender.reset(outFrame, true);
- if (!appender.append(fta1, tStart, tEnd)) {
- throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
- }
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- }
+ public void close() {
+ super.close();
+ tPointersTemp = null;
}
- private void sort(int offset, int length) throws HyracksDataException {
+ void sort(int offset, int length) throws HyracksDataException {
int step = 1;
- int len = length;
- int end = offset + len;
+ int end = offset + length;
/** bottom-up merge */
- while (step < len) {
+ while (step < length) {
/** merge */
for (int i = offset; i < end; i += 2 * step) {
int next = i + step;
@@ -175,8 +83,6 @@
/**
* Merge two subarrays into one
- *
- * @throws HyracksDataException
*/
private void merge(int start1, int start2, int len1, int len2) throws HyracksDataException {
int targetPos = start1;
@@ -226,20 +132,20 @@
}
int i2 = tp2i;
int j2 = tp2j;
- ByteBuffer buf1 = buffers.get(i1);
- ByteBuffer buf2 = buffers.get(i2);
+ ByteBuffer buf1 = super.bufferManager.getFrame(i1);
+ ByteBuffer buf2 = super.bufferManager.getFrame(i2);
byte[] b1 = buf1.array();
byte[] b2 = buf2.array();
- fta1.reset(buf1);
+ inputTupleAccessor.reset(buf1);
fta2.reset(buf2);
for (int f = 0; f < comparators.length; ++f) {
int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf1.array(), j1 + (fIdx - 1) * 4);
- int f1End = IntSerDeUtils.getInt(buf1.array(), j1 + fIdx * 4);
- int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
+ int f1Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b1, j1 + (fIdx - 1) * 4);
+ int f1End = IntSerDeUtils.getInt(b1, j1 + fIdx * 4);
+ int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(buf2.array(), j2 + (fIdx - 1) * 4);
- int f2End = IntSerDeUtils.getInt(buf2.array(), j2 + fIdx * 4);
+ int f2Start = fIdx == 0 ? 0 : IntSerDeUtils.getInt(b2, j2 + (fIdx - 1) * 4);
+ int f2End = IntSerDeUtils.getInt(b2, j2 + fIdx * 4);
int s2 = j2 + fta2.getFieldSlotsLength() + f2Start;
int l2 = f2End - f2Start;
int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
@@ -250,8 +156,4 @@
return 0;
}
- @Override
- public void close() {
- this.buffers.clear();
- }
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
index d607a51..328bb5e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/FrameSorterQuickSort.java
@@ -1,153 +1,54 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ * 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
-public class FrameSorterQuickSort implements IFrameSorter {
- private final IHyracksTaskContext ctx;
- private final int[] sortFields;
- private final INormalizedKeyComputer nkc;
- private final IBinaryComparator[] comparators;
- private final List<ByteBuffer> buffers;
+public class FrameSorterQuickSort extends AbstractFrameSorter {
- private final FrameTupleAccessor fta1;
- private final FrameTupleAccessor fta2;
+ private FrameTupleAccessor fta2;
- private final FrameTupleAppender appender;
-
- private final ByteBuffer outFrame;
-
- private int dataFrameCount;
- private int[] tPointers;
- private int tupleCount;
-
- public FrameSorterQuickSort(IHyracksTaskContext ctx, int[] sortFields,
+ public FrameSorterQuickSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
RecordDescriptor recordDescriptor) throws HyracksDataException {
- this.ctx = ctx;
- this.sortFields = sortFields;
- nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- buffers = new ArrayList<ByteBuffer>();
- fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- appender = new FrameTupleAppender(ctx.getFrameSize());
- outFrame = ctx.allocateFrame();
+ this(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ Integer.MAX_VALUE);
+ }
- dataFrameCount = 0;
+ public FrameSorterQuickSort(IHyracksTaskContext ctx, IFrameBufferManager bufferManager, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, int outputLimit) throws HyracksDataException {
+ super(ctx, bufferManager, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor,
+ outputLimit);
+ fta2 = new FrameTupleAccessor(recordDescriptor);
}
@Override
- public void reset() {
- dataFrameCount = 0;
- tupleCount = 0;
+ void sortTupleReferences() throws HyracksDataException {
+ sort(tPointers, 0, tupleCount);
}
- @Override
- public int getFrameCount() {
- return dataFrameCount;
- }
-
- @Override
- public void insertFrame(ByteBuffer buffer) throws HyracksDataException {
- ByteBuffer copyFrame;
- if (dataFrameCount == buffers.size()) {
- copyFrame = ctx.allocateFrame();
- buffers.add(copyFrame);
- } else {
- copyFrame = buffers.get(dataFrameCount);
- }
- FrameUtils.copy(buffer, copyFrame);
- ++dataFrameCount;
- }
-
- @Override
- public void sortFrames() throws HyracksDataException {
- int nBuffers = dataFrameCount;
- tupleCount = 0;
- for (int i = 0; i < nBuffers; ++i) {
- fta1.reset(buffers.get(i));
- tupleCount += fta1.getTupleCount();
- }
- int sfIdx = sortFields[0];
- tPointers = tPointers == null || tPointers.length < tupleCount * 4 ? new int[tupleCount * 4] : tPointers;
- int ptr = 0;
- for (int i = 0; i < nBuffers; ++i) {
- fta1.reset(buffers.get(i));
- int tCount = fta1.getTupleCount();
- byte[] array = fta1.getBuffer().array();
- for (int j = 0; j < tCount; ++j) {
- int tStart = fta1.getTupleStartOffset(j);
- int tEnd = fta1.getTupleEndOffset(j);
- tPointers[ptr * 4] = i;
- tPointers[ptr * 4 + 1] = tStart;
- tPointers[ptr * 4 + 2] = tEnd;
- int f0StartRel = fta1.getFieldStartOffset(j, sfIdx);
- int f0EndRel = fta1.getFieldEndOffset(j, sfIdx);
- int f0Start = f0StartRel + tStart + fta1.getFieldSlotsLength();
- tPointers[ptr * 4 + 3] = nkc == null ? 0 : nkc.normalize(array, f0Start, f0EndRel - f0StartRel);
- ++ptr;
- }
- }
- if (tupleCount > 0) {
- sort(tPointers, 0, tupleCount);
- }
- }
-
- @Override
- public void flushFrames(IFrameWriter writer) throws HyracksDataException {
- appender.reset(outFrame, true);
- for (int ptr = 0; ptr < tupleCount; ++ptr) {
- int i = tPointers[ptr * 4];
- int tStart = tPointers[ptr * 4 + 1];
- int tEnd = tPointers[ptr * 4 + 2];
- ByteBuffer buffer = buffers.get(i);
- fta1.reset(buffer);
- if (!appender.append(fta1, tStart, tEnd)) {
- FrameUtils.flushFrame(outFrame, writer);
- appender.reset(outFrame, true);
- if (!appender.append(fta1, tStart, tEnd)) {
- throw new HyracksDataException("Record size (" + (tEnd - tStart) + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
- }
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outFrame, writer);
- }
- }
-
- private void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
+ void sort(int[] tPointers, int offset, int length) throws HyracksDataException {
int m = offset + (length >> 1);
int mi = tPointers[m * 4];
int mj = tPointers[m * 4 + 1];
@@ -221,17 +122,17 @@
}
int i2 = tp2i;
int j2 = tp2j;
- ByteBuffer buf1 = buffers.get(i1);
- ByteBuffer buf2 = buffers.get(i2);
+ ByteBuffer buf1 = super.bufferManager.getFrame(i1);
+ ByteBuffer buf2 = super.bufferManager.getFrame(i2);
byte[] b1 = buf1.array();
byte[] b2 = buf2.array();
- fta1.reset(buf1);
+ inputTupleAccessor.reset(buf1);
fta2.reset(buf2);
for (int f = 0; f < comparators.length; ++f) {
int fIdx = sortFields[f];
int f1Start = fIdx == 0 ? 0 : buf1.getInt(j1 + (fIdx - 1) * 4);
int f1End = buf1.getInt(j1 + fIdx * 4);
- int s1 = j1 + fta1.getFieldSlotsLength() + f1Start;
+ int s1 = j1 + inputTupleAccessor.getFieldSlotsLength() + f1Start;
int l1 = f1End - f1Start;
int f2Start = fIdx == 0 ? 0 : buf2.getInt(j2 + (fIdx - 1) * 4);
int f2End = buf2.getInt(j2 + fIdx * 4);
@@ -245,8 +146,4 @@
return 0;
}
- @Override
- public void close() {
- this.buffers.clear();
- }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
new file mode 100644
index 0000000..564a462
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HeapSortRunGenerator.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFramePool;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableTupleMemoryManager;
+
+public class HeapSortRunGenerator extends AbstractSortRunGenerator {
+ protected final IHyracksTaskContext ctx;
+ protected final int frameLimit;
+ protected final int topK;
+ protected final int[] sortFields;
+ protected final INormalizedKeyComputerFactory nmkFactory;
+ protected final IBinaryComparatorFactory[] comparatorFactories;
+ protected final RecordDescriptor recordDescriptor;
+ protected ITupleSorter tupleSorter;
+ protected IFrameTupleAccessor inAccessor;
+
+ public HeapSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int topK, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) {
+ super();
+ this.ctx = ctx;
+ this.frameLimit = frameLimit;
+ this.topK = topK;
+ this.sortFields = sortFields;
+ this.nmkFactory = firstKeyNormalizerFactory;
+ this.comparatorFactories = comparatorFactories;
+ this.inAccessor = new FrameTupleAccessor(recordDescriptor);
+ this.recordDescriptor = recordDescriptor;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ IFramePool framePool = new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize());
+ ITupleBufferManager bufferManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
+ tupleSorter = new TupleSorterHeapSort(ctx, bufferManager, topK, sortFields, nmkFactory,
+ comparatorFactories);
+ super.open();
+ }
+
+ @Override
+ public ISorter getSorter() throws HyracksDataException {
+ return tupleSorter;
+ }
+
+ @Override
+ protected RunFileWriter getRunFileWriter() throws HyracksDataException {
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+ HeapSortRunGenerator.class.getSimpleName());
+ return new RunFileWriter(file, ctx.getIOManager());
+ }
+
+ @Override
+ protected IFrameWriter getFlushableFrameWriter(RunFileWriter writer) throws HyracksDataException {
+ return writer;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ inAccessor.reset(buffer);
+ for (int i = 0; i < inAccessor.getTupleCount(); i++) {
+ if (!tupleSorter.insertTuple(inAccessor, i)) {
+ flushFramesToRun();
+ if (!tupleSorter.insertTuple(inAccessor, i)) {
+ throw new HyracksDataException("The given tuple is too big to insert into the sorting memory.");
+ }
+ }
+ }
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
new file mode 100644
index 0000000..9976aad
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotBiggestFirst;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+
+public class HybridTopKSortRunGenerator extends HeapSortRunGenerator {
+ private static final Logger LOG = Logger.getLogger(HybridTopKSortRunGenerator.class.getName());
+
+ private static final int SWITCH_TO_FRAME_SORTER_THRESHOLD = 2;
+ private IFrameSorter frameSorter = null;
+ private int tupleSorterFlushedTimes = 0;
+
+ public HybridTopKSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int topK, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) {
+ super(ctx, frameLimit, topK, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+ }
+
+ @Override
+ public ISorter getSorter() throws HyracksDataException {
+ if (tupleSorter != null) {
+ return tupleSorter;
+ } else if (frameSorter != null) {
+ return frameSorter;
+ }
+ return null;
+ }
+
+ @Override
+ protected RunFileWriter getRunFileWriter() throws HyracksDataException {
+ FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
+ HybridTopKSortRunGenerator.class.getSimpleName());
+ return new RunFileWriter(file, ctx.getIOManager());
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ inAccessor.reset(buffer);
+ if (tupleSorter != null) {
+ boolean isBadK = false;
+ for (int i = 0; i < inAccessor.getTupleCount(); i++) {
+ if (!tupleSorter.insertTuple(inAccessor, i)) {
+ flushFramesToRun();
+ isBadK = true;
+ if (!tupleSorter.insertTuple(inAccessor, i)) {
+ throw new HyracksDataException("The given tuple is too big to insert into the sorting memory.");
+ }
+ }
+ }
+ if (isBadK) {
+ tupleSorterFlushedTimes++;
+ if (tupleSorterFlushedTimes > SWITCH_TO_FRAME_SORTER_THRESHOLD) {
+ if (tupleSorter.hasRemaining()) {
+ flushFramesToRun();
+ }
+ tupleSorter.close();
+ tupleSorter = null;
+ if (LOG.isLoggable(Level.FINE)) {
+ LOG.fine("clear tupleSorter");
+ }
+ }
+ }
+ } else {
+ if (frameSorter == null) {
+ VariableFrameMemoryManager bufferManager = new VariableFrameMemoryManager(
+ new VariableFramePool(ctx, (frameLimit - 1) * ctx.getInitialFrameSize()),
+ new FrameFreeSlotBiggestFirst(frameLimit - 1));
+ frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sortFields, nmkFactory, comparatorFactories,
+ recordDescriptor, topK);
+ if (LOG.isLoggable(Level.FINE)) {
+ LOG.fine("create frameSorter");
+ }
+ }
+ if (!frameSorter.insertFrame(buffer)) {
+ flushFramesToRun();
+ if (!frameSorter.insertFrame(buffer)) {
+ throw new HyracksDataException("The given frame is too big to insert into the sorting memory.");
+ }
+ }
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
index d9b8d37..6d0b100 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IFrameSorter.java
@@ -1,37 +1,28 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ * 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-public interface IFrameSorter {
+public interface IFrameSorter extends ISorter {
- public void reset();
+ int getFrameCount();
- public int getFrameCount();
-
- public void insertFrame(ByteBuffer buffer) throws HyracksDataException;
-
- public void sortFrames() throws HyracksDataException;
-
- public void flushFrames(IFrameWriter writer) throws HyracksDataException;
-
- public void close();
+ boolean insertFrame(ByteBuffer inputBuffer) throws HyracksDataException;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
deleted file mode 100644
index e669335..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IMemoryManager.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-
-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;
-
-/**
- * @author pouria Defines the required operations, needed for any memory
- * manager, used in sorting with replacement selection, to manage the
- * free spaces
- */
-
-public interface IMemoryManager {
-
- /**
- * Allocates a free slot equal or greater than requested length. Pointer to
- * the allocated slot is put in result, and gets returned to the caller. If
- * no proper free slot is available, result would contain a null/invalid
- * pointer (may vary between different implementations)
- *
- * @param length
- * @param result
- * @throws HyracksDataException
- */
- void allocate(int length, Slot result) throws HyracksDataException;
-
- /**
- * Unallocates the specified slot (and returns it back to the free slots
- * set)
- *
- * @param s
- * @return the total length of unallocted slot
- * @throws HyracksDataException
- */
- int unallocate(Slot s) throws HyracksDataException;
-
- /**
- * @param frameIndex
- * @return the specified frame, from the set of memory buffers, being
- * managed by this memory manager
- */
- ByteBuffer getFrame(int frameIndex);
-
- /**
- * Writes the specified tuple into the specified memory slot (denoted by
- * frameIx and offset)
- *
- * @param frameIx
- * @param offset
- * @param src
- * @param tIndex
- * @return
- */
- boolean writeTuple(int frameIx, int offset, FrameTupleAccessor src, int tIndex);
-
- /**
- * Reads the specified tuple (denoted by frameIx and offset) and appends it
- * to the passed FrameTupleAppender
- *
- * @param frameIx
- * @param offset
- * @param dest
- * @return
- */
- boolean readTuple(int frameIx, int offset, FrameTupleAppender dest);
-
- /**
- * close and cleanup the memory manager
- */
- void close();
-
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
index 2840d01..d21255e 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/IRunGenerator.java
@@ -16,7 +16,6 @@
import java.util.List;
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
/**
@@ -28,5 +27,5 @@
/**
* @return the list of generated (sorted) runs
*/
- public List<IFrameReader> getRuns();
+ List<RunAndMaxFrameSizePair> getRuns();
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
deleted file mode 100644
index 8cff0df..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISelectionTree.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * @author pouria
- * Defines the selection tree, used in sorting with replacement
- * selection to manage the order of output tuples into the runs, during
- * the run generation phase. This tree contains tuples, belonging to two
- * different runs: - Current run (being written to the output) - Next
- * run
- */
-
-public interface ISelectionTree {
-
- /**
- * Inserts a new element into the selectionTree
- *
- * @param element
- * contains the pointer to the memory slot, containing the tuple,
- * along with its run number
- * @throws HyracksDataException
- */
- void insert(int[] element) throws HyracksDataException;
-
- /**
- * Removes and returns the smallest element in the tree
- *
- * @param result
- * is the array that will eventually contain minimum entry
- * pointer
- * @throws HyracksDataException
- */
- void getMin(int[] result) throws HyracksDataException;
-
- /**
- * Removes and returns the largest element in the tree
- *
- * @param result
- * is the array that will eventually contain maximum entry
- * pointer
- * @throws HyracksDataException
- */
- void getMax(int[] result) throws HyracksDataException;
-
- /**
- * @return True of the selection tree does not have any element, false
- * otherwise
- */
- boolean isEmpty();
-
- /**
- * Removes all the elements in the tree
- */
- void reset();
-
- /**
- * Returns (and does NOT remove) the smallest element in the tree
- *
- * @param result
- * is the array that will eventually contain minimum entry
- * pointer
- */
- void peekMin(int[] result);
-
- /**
- * Returns (and does NOT remove) the largest element in the tree
- *
- * @param result
- * is the array that will eventually contain maximum entry
- * pointer
- * @throws HyracksDataException
- */
- void peekMax(int[] result) throws HyracksDataException;
-
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java
new file mode 100644
index 0000000..09a8169
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ISorter.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISorter {
+
+ boolean hasRemaining();
+
+ void reset() throws HyracksDataException;
+
+ void sort() throws HyracksDataException;
+
+ void close();
+
+ int flush(IFrameWriter writer) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java
new file mode 100644
index 0000000..bea8b35
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ITupleSorter.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface ITupleSorter extends ISorter {
+ int getTupleCount();
+
+ boolean insertTuple(IFrameTupleAccessor frameTupleAccessor, int index) throws HyracksDataException;
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index 6fa21b5..379a783 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -36,6 +36,10 @@
import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.FrameFreeSlotLastFit;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.IFrameBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFrameMemoryManager;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
public class InMemorySortOperatorDescriptor extends AbstractOperatorDescriptor {
private static final long serialVersionUID = 1L;
@@ -76,7 +80,7 @@
builder.addBlockingEdge(sa, ma);
}
- public static class SortTaskState extends AbstractStateObject {
+ private static class SortTaskState extends AbstractStateObject {
private FrameSorterMergeSort frameSorter;
public SortTaskState() {
@@ -110,20 +114,29 @@
@Override
public void open() throws HyracksDataException {
- state = new SortTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
- state.frameSorter = new FrameSorterMergeSort(ctx, sortFields, firstKeyNormalizerFactory,
- comparatorFactories, recordDescriptors[0]);
+ state = new SortTaskState(ctx.getJobletContext().getJobId(),
+ new TaskId(getActivityId(), partition));
+
+ IFrameBufferManager frameBufferManager = new VariableFrameMemoryManager(
+ new VariableFramePool(ctx, VariableFramePool.UNLIMITED_MEMORY), new FrameFreeSlotLastFit());
+
+ state.frameSorter = new FrameSorterMergeSort(ctx, frameBufferManager, sortFields,
+ firstKeyNormalizerFactory, comparatorFactories, recordDescriptors[0]);
state.frameSorter.reset();
}
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- state.frameSorter.insertFrame(buffer);
+ if (!state.frameSorter.insertFrame(buffer)) {
+ throw new HyracksDataException("Failed to insert the given frame into sorting buffer. "
+ + "Please increase the sorting memory budget to enable the in-memory sorting, "
+ + "or you could use ExternalSort instead.");
+ }
}
@Override
public void close() throws HyracksDataException {
- state.frameSorter.sortFrames();
+ state.frameSorter.sort();
ctx.setStateObject(state);
}
@@ -152,7 +165,7 @@
try {
SortTaskState state = (SortTaskState) ctx.getStateObject(new TaskId(new ActivityId(
getOperatorId(), SORT_ACTIVITY_ID), partition));
- state.frameSorter.flushFrames(writer);
+ state.frameSorter.flush(writer);
} catch (Exception e) {
writer.fail();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
deleted file mode 100644
index ef1ae88..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.TaskId;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractActivityNode;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractStateObject;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/**
- * @author pouria
- * Operator descriptor for sorting with replacement, consisting of two
- * phases:
- * - Run Generation: Denoted by OptimizedSortActivity below, in which
- * sort runs get generated from the input data. This phases uses the
- * Selection Tree and Memory Manager to benefit from the replacement
- * selection optimization, to create runs which are longer than the
- * available memory size.
- * - Merging: Denoted by MergeActivity below, in which runs (generated
- * in the previous phase) get merged via a merger. Each run has a single
- * buffer in memory, and a priority queue is used to select the top
- * tuple each time. Top tuple is send to a new run or output
- */
-
-public class OptimizedExternalSortOperatorDescriptor extends AbstractOperatorDescriptor {
-
- private static final int NO_LIMIT = -1;
- private static final long serialVersionUID = 1L;
- private static final int SORT_ACTIVITY_ID = 0;
- private static final int MERGE_ACTIVITY_ID = 1;
-
- private final int[] sortFields;
- private final INormalizedKeyComputerFactory firstKeyNormalizerFactory;
- private final IBinaryComparatorFactory[] comparatorFactories;
- private final int memSize;
- private final int outputLimit;
-
- public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int[] sortFields,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
- this(spec, framesLimit, NO_LIMIT, sortFields, null, comparatorFactories, recordDescriptor);
- }
-
- public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int outputLimit,
- int[] sortFields, IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
- this(spec, framesLimit, outputLimit, sortFields, null, comparatorFactories, recordDescriptor);
- }
-
- public OptimizedExternalSortOperatorDescriptor(IOperatorDescriptorRegistry spec, int memSize, int outputLimit,
- int[] sortFields, INormalizedKeyComputerFactory firstKeyNormalizerFactory,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
- super(spec, 1, 1);
- this.memSize = memSize;
- this.outputLimit = outputLimit;
- this.sortFields = sortFields;
- this.firstKeyNormalizerFactory = firstKeyNormalizerFactory;
- this.comparatorFactories = comparatorFactories;
- if (memSize <= 1) {
- throw new IllegalStateException();// minimum of 2 fames (1 in,1 out)
- }
- recordDescriptors[0] = recordDescriptor;
- }
-
- @Override
- public void contributeActivities(IActivityGraphBuilder builder) {
- OptimizedSortActivity osa = new OptimizedSortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
- OptimizedMergeActivity oma = new OptimizedMergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
-
- builder.addActivity(this, osa);
- builder.addSourceEdge(0, osa, 0);
-
- builder.addActivity(this, oma);
- builder.addTargetEdge(0, oma, 0);
-
- builder.addBlockingEdge(osa, oma);
- }
-
- public static class OptimizedSortTaskState extends AbstractStateObject {
- private List<IFrameReader> runs;
-
- public OptimizedSortTaskState() {
- }
-
- private OptimizedSortTaskState(JobId jobId, TaskId taskId) {
- super(jobId, taskId);
- }
-
- @Override
- public void toBytes(DataOutput out) throws IOException {
-
- }
-
- @Override
- public void fromBytes(DataInput in) throws IOException {
-
- }
- }
-
- private class OptimizedSortActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public OptimizedSortActivity(ActivityId id) {
- super(id);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- final IRunGenerator runGen;
- if (outputLimit == NO_LIMIT) {
- runGen = new OptimizedExternalSortRunGenerator(ctx, sortFields, firstKeyNormalizerFactory,
- comparatorFactories, recordDescriptors[0], memSize);
- } else {
- runGen = new OptimizedExternalSortRunGeneratorWithLimit(ctx, sortFields, firstKeyNormalizerFactory,
- comparatorFactories, recordDescriptors[0], memSize, outputLimit);
- }
-
- IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
- @Override
- public void open() throws HyracksDataException {
-
- runGen.open();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- runGen.nextFrame(buffer);
- }
-
- @Override
- public void close() throws HyracksDataException {
- OptimizedSortTaskState state = new OptimizedSortTaskState(ctx.getJobletContext().getJobId(),
- new TaskId(getActivityId(), partition));
- runGen.close();
- state.runs = runGen.getRuns();
- ctx.setStateObject(state);
-
- }
-
- @Override
- public void fail() throws HyracksDataException {
- runGen.fail();
- }
- };
- return op;
- }
- }
-
- private class OptimizedMergeActivity extends AbstractActivityNode {
- private static final long serialVersionUID = 1L;
-
- public OptimizedMergeActivity(ActivityId id) {
- super(id);
- }
-
- @Override
- public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
- IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
- @Override
- public void initialize() throws HyracksDataException {
- OptimizedSortTaskState state = (OptimizedSortTaskState) ctx.getStateObject(new TaskId(
- new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
-
- List<IFrameReader> runs = state.runs;
-
- IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
-
- INormalizedKeyComputer nmkComputer = firstKeyNormalizerFactory == null ? null
- : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- int necessaryFrames = Math.min(runs.size() + 2, memSize);
- ExternalSortRunMerger merger = new ExternalSortRunMerger(ctx, outputLimit, runs, sortFields,
- comparators, nmkComputer, recordDescriptors[0], necessaryFrames, writer);
-
- merger.processWithReplacementSelection();
-
- }
- };
- return op;
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
deleted file mode 100644
index d78af12..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGenerator.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria This class implements the run generator for sorting with
- * replacement selection, where there is no limit on the output, i.e.
- * the whole data should be sorted. A SortMinHeap is used as the
- * selectionTree to decide the order of writing tuples into the runs,
- * while memory manager is based on a binary search tree to allocate
- * tuples in the memory. The overall process is as follows: - Read the
- * input data frame by frame. For each tuple T in the current frame: -
- * Try to allocate a memory slot for writing T along with the attached
- * header/footer (for memory management purpose) - If T can not be
- * allocated, try to output as many tuples, currently resident in
- * memory, as needed so that a free slot, large enough to hold T, gets
- * created. MinHeap decides about which tuple should be sent to the
- * output at each step. - Write T into the memory - Calculate the runID
- * of T (based on the last output tuple for the current run). It is
- * either the current run or the next run. Also calculate Poorman's
- * Normalized Key (PNK) for T, to make comparisons faster later. -
- * Create a heap element for T, containing: its runID, the slot pointer
- * to its memory location, and its PNK. - Insert the created heap
- * element into the heap - Upon closing, write all the tuples, currently
- * resident in memory, into their corresponding run(s). Again min heap
- * decides about which tuple is the next for output.
- * OptimizedSortOperatorDescriptor will merge the generated runs, to
- * generate the final sorted output of the data.
- */
-public class OptimizedExternalSortRunGenerator implements IRunGenerator {
- private final IHyracksTaskContext ctx;
- private final int[] sortFields;
- private final INormalizedKeyComputer nkc;
- private final IBinaryComparatorFactory[] comparatorFactories;
- private final IBinaryComparator[] comparators;
- private final RecordDescriptor recordDescriptor;
- private final List<IFrameReader> runs;
-
- private ISelectionTree sTree;
- private IMemoryManager memMgr;
-
- private final int memSize;
- private FrameTupleAccessor inputAccessor; // Used to read tuples in
- // nextFrame()
- private FrameTupleAppender outputAppender; // Used to write tuple to the
- // dedicated output buffer
- private ByteBuffer outputBuffer; // Dedicated output buffer to write tuples
- // into run(s)
- private FrameTupleAccessor lastRecordAccessor; // Used to read last output
- // record from the output
- // buffer
- private int lastTupleIx; // Holds index of last output tuple in the
- // dedicated output buffer
- private Slot allocationPtr; // Contains the ptr to the allocated memory slot
- // by the memory manager for the new tuple
- private Slot outputedTuple; // Contains the ptr to the next tuple chosen by
- // the selectionTree to output
- private int[] sTreeTop;
-
- private RunFileWriter writer;
-
- private boolean newRun;
- private int curRunId;
-
- public OptimizedExternalSortRunGenerator(IHyracksTaskContext ctx, int[] sortFields,
- INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, int memSize) {
- this.ctx = ctx;
- this.sortFields = sortFields;
- nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- this.comparatorFactories = comparatorFactories;
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- this.recordDescriptor = recordDesc;
- this.runs = new LinkedList<IFrameReader>();
- this.memSize = memSize;
- }
-
- @Override
- public void open() throws HyracksDataException {
- runs.clear();
- inputAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- outputAppender = new FrameTupleAppender(ctx.getFrameSize());
- outputBuffer = ctx.allocateFrame();
- outputAppender.reset(outputBuffer, true);
- lastRecordAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
-
- this.memMgr = new BSTMemMgr(ctx, memSize);
- this.sTree = new SortMinHeap(ctx, sortFields, comparatorFactories, recordDescriptor, memMgr);
- this.allocationPtr = new Slot();
- this.outputedTuple = new Slot();
- this.sTreeTop = new int[] { -1, -1, -1, -1 };
- curRunId = -1;
- openNewRun();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- inputAccessor.reset(buffer);
- byte[] bufferArray = buffer.array();
- int tupleCount = inputAccessor.getTupleCount();
- for (int i = 0; i < tupleCount; ++i) {
- allocationPtr.clear();
- int tLength = inputAccessor.getTupleEndOffset(i) - inputAccessor.getTupleStartOffset(i);
- memMgr.allocate(tLength, allocationPtr);
- while (allocationPtr.isNull()) {
- int unAllocSize = -1;
- while (unAllocSize < tLength) {
- unAllocSize = outputRecord();
- if (unAllocSize < 1) {
- throw new HyracksDataException(
- "Unable to allocate space for the new tuple, while there is no more tuple to output");
- }
- }
- memMgr.allocate(tLength, allocationPtr);
- }
- memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
- int runId = getRunId(inputAccessor, i);
- int pnk = getPNK(inputAccessor, i, bufferArray);
- int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
- sTree.insert(entry);
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- }
-
- @Override
- public void close() throws HyracksDataException {
- while (!sTree.isEmpty()) { // Outputting remaining elements in the
- // selectionTree
- outputRecord();
- }
- if (outputAppender.getTupleCount() > 0) { // Writing out very last
- // resident records to file
- FrameUtils.flushFrame(outputBuffer, writer);
- }
- outputAppender.reset(outputBuffer, true);
- writer.close();
- runs.add(writer.createReader());
- memMgr.close();
- }
-
- public List<IFrameReader> getRuns() {
- return runs;
- }
-
- private int outputRecord() throws HyracksDataException {
- outputedTuple.clear();
- sTree.getMin(sTreeTop);
- if (!isEntryValid(sTreeTop)) {
- throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
- }
-
- if (sTreeTop[SortMinHeap.RUN_ID_IX] != curRunId) { // We need to switch
- // runs
- openNewRun();
- }
-
- int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
- int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
- // append to
- // the
- // tupleAppender
- FrameUtils.flushFrame(outputBuffer, writer);
- outputAppender.reset(outputBuffer, true);
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
- throw new HyracksDataException("Can not append to the ouput buffer in sort");
- }
- lastTupleIx = 0;
- } else {
- lastTupleIx++;
- }
- outputedTuple.set(tFrameIx, tOffset);
- newRun = false;
- return memMgr.unallocate(outputedTuple);
-
- }
-
- private int getPNK(FrameTupleAccessor fta, int tIx, byte[] buffInArray) {
- // Moved buffInArray out for better performance (not converting for each and every tuple)
- int sfIdx = sortFields[0];
- int tStart = fta.getTupleStartOffset(tIx);
- int f0StartRel = fta.getFieldStartOffset(tIx, sfIdx);
- int f0EndRel = fta.getFieldEndOffset(tIx, sfIdx);
- int f0Start = f0StartRel + tStart + fta.getFieldSlotsLength();
- return (nkc == null ? 0 : nkc.normalize(buffInArray, f0Start, f0EndRel - f0StartRel));
- }
-
- private int getRunId(FrameTupleAccessor fta, int tupIx) throws HyracksDataException {
- // Comparing current record to last output record, it decides about current record's runId
- if (newRun) { // Very first record for a new run
- return curRunId;
- }
-
- byte[] lastRecBuff = outputBuffer.array();
- lastRecordAccessor.reset(outputBuffer);
- int lastStartOffset = lastRecordAccessor.getTupleStartOffset(lastTupleIx);
-
- ByteBuffer fr2 = fta.getBuffer();
- byte[] curRecBuff = fr2.array();
- int r2StartOffset = fta.getTupleStartOffset(tupIx);
-
- for (int f = 0; f < comparators.length; ++f) {
- int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : outputBuffer.getInt(lastStartOffset + (fIdx - 1) * 4);
- int f1End = outputBuffer.getInt(lastStartOffset + fIdx * 4);
- int s1 = lastStartOffset + lastRecordAccessor.getFieldSlotsLength() + f1Start;
- int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
- int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
- int s2 = r2StartOffset + fta.getFieldSlotsLength() + f2Start;
- int l2 = f2End - f2Start;
- int c = comparators[f].compare(lastRecBuff, s1, l1, curRecBuff, s2, l2);
- if (c != 0) {
- if (c <= 0) {
- return curRunId;
- } else {
- return (curRunId + 1);
- }
- }
- }
- return curRunId;
- }
-
- private void openNewRun() throws HyracksDataException {
- if (writer != null) { // There is a prev run, so flush its tuples and
- // close it first
- if (outputAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outputBuffer, writer);
- }
- outputAppender.reset(outputBuffer, true);
- writer.close();
- runs.add(writer.createReader());
- }
-
- FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
- ExternalSortRunGenerator.class.getSimpleName());
- writer = new RunFileWriter(file, ctx.getIOManager());
- writer.open();
- curRunId++;
- newRun = true;
- lastTupleIx = -1;
- }
-
- private boolean isEntryValid(int[] entry) {
- return ((entry[SortMinHeap.RUN_ID_IX] > -1) && (entry[SortMinHeap.FRAME_IX] > -1) && (entry[SortMinHeap.OFFSET_IX] > -1));
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
deleted file mode 100644
index 5b01fb8..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortRunGeneratorWithLimit.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.LinkedList;
-import java.util.List;
-
-import edu.uci.ics.hyracks.api.comm.IFrameReader;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.io.RunFileWriter;
-
-/**
- * @author pouria
- * This class implements the run generator for sorting with replacement
- * selection, where there is a limit on the output, i.e. we are looking
- * for top-k tuples (first k smallest tuples w.r.t sorting keys).
- * A SortMinMaxHeap is used as the selectionTree to decide the order of
- * writing tuples into the runs, and also to prune tuples (if possible).
- * Memory manager is based on a binary search tree and is used to
- * allocate memory slots for tuples.
- * The overall process is as follows (Assuming that the limit is K):
- * - Read the input data frame by frame. For each tuple T in the current
- * frame:
- * - If currentRun R has reached the limit of K on the size, and (T >
- * maximum tuple of R), then ignore T.
- * - Otherwise, try to allocate a memory slot for writing T along with
- * the attached header/footer (for memory management purpose)
- * - If T can not be allocated, try to output as many tuples, currently
- * resident in memory, as needed so that a free slot, large enough to
- * hold T, gets created. MinMaxHeap decides about which tuple should be
- * sent to the output at each step.
- * - Write T into memory.
- * - Calculate the runID of T (based on the last output tuple for the
- * current run). It is either the current run or the next run. Also
- * calculate Poorman's Normalized Key (PNK) for T, to make comparisons
- * faster later.
- * - Create an heap element for T, containing its runID, the slot ptr to
- * its memory location, and its PNK.
- * - If runID is the nextRun, insert the heap element into the heap, and
- * increment the size of nextRun.
- * - If runID is the currentRun, then:
- * - If currentRun has not hit the limit of k, insert the element into
- * the heap, and increase currentRun size. - Otherwise, currentRun has
- * hit the limit of K, while T is less than the max. So discard the
- * current max for the current run (by poping it from the heap and
- * unallocating its memory location) and insert the heap element into
- * the heap. No need to change the currentRun size as we are replacing
- * an old element (the old max) with T.
- * - Upon closing, write all the tuples, currently resident in memory,
- * into their corresponding run(s).
- * - Note that upon opening a new Run R, if size of R (based on stats)
- * is S and (S > K), then (S-K) current maximum tuples of R (which are
- * resident in memory) get discarded at the beginning. MinMax heap can
- * be used to find these tuples.
- */
-public class OptimizedExternalSortRunGeneratorWithLimit implements IRunGenerator {
-
- private final IHyracksTaskContext ctx;
- private final int[] sortFields;
- private final INormalizedKeyComputer nkc;
- private final IBinaryComparatorFactory[] comparatorFactories;
- private final IBinaryComparator[] comparators;
- private final RecordDescriptor recordDescriptor;
- private final List<IFrameReader> runs;
-
- private ISelectionTree sTree;
- private IMemoryManager memMgr;
-
- private final int memSize;
- private FrameTupleAccessor inputAccessor; // Used to read tuples in
- // nextFrame()
- private FrameTupleAppender outputAppender; // Used to write tuple to the
- // dedicated output buffer
- private ByteBuffer outputBuffer; // Dedicated output buffer to write tuples
- // into run(s)
- private FrameTupleAccessor lastRecordAccessor; // Used to read last output
- // record from the output
- // buffer
- private FrameTupleAccessor fta2; // Used to read max record
- private final int outputLimit;
- private int curRunSize;
- private int nextRunSize;
- private int lastTupleIx; // Holds index of last output tuple in the
- // dedicated output buffer
- private Slot allocationPtr; // Contains the ptr to the allocated memory slot
- // by the memory manager for the new tuple
- private Slot outputedTuple; // Contains the ptr to the next tuple chosen by
- // the selectionTree to output
- private Slot discard;
- private int[] sTreeTop;
- private int[] peek;
- private RunFileWriter writer;
- private boolean newRun;
- private int curRunId;
-
- public OptimizedExternalSortRunGeneratorWithLimit(IHyracksTaskContext ctx, int[] sortFields,
- INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, int memSize, int limit) {
-
- this.ctx = ctx;
- this.sortFields = sortFields;
- nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
- this.comparatorFactories = comparatorFactories;
- comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- this.recordDescriptor = recordDesc;
- this.runs = new LinkedList<IFrameReader>();
- this.memSize = memSize;
-
- this.outputLimit = limit;
- }
-
- @Override
- public void open() throws HyracksDataException {
- runs.clear();
- inputAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- outputAppender = new FrameTupleAppender(ctx.getFrameSize());
- outputBuffer = ctx.allocateFrame();
- outputAppender.reset(outputBuffer, true);
- lastRecordAccessor = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- this.memMgr = new BSTMemMgr(ctx, memSize);
- this.sTree = new SortMinMaxHeap(ctx, sortFields, comparatorFactories, recordDescriptor, memMgr);
- this.allocationPtr = new Slot();
- this.outputedTuple = new Slot();
- this.sTreeTop = new int[] { -1, -1, -1, -1 };
- this.peek = new int[] { -1, -1, -1, -1 };
- this.discard = new Slot();
-
- curRunId = -1;
- curRunSize = 0;
- nextRunSize = 0;
- openNewRun();
- }
-
- @Override
- public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- inputAccessor.reset(buffer);
- byte[] bufferArray = buffer.array();
- int tupleCount = inputAccessor.getTupleCount();
- for (int i = 0; i < tupleCount; i++) {
- if (curRunSize >= outputLimit) {
- sTree.peekMax(peek);
- if (isEntryValid(peek)
- && compareRecords(inputAccessor, i, peek[SortMinMaxHeap.FRAME_IX],
- peek[SortMinMaxHeap.OFFSET_IX]) >= 0) {
- continue;
- }
- }
-
- allocationPtr.clear();
- int tLength = inputAccessor.getTupleEndOffset(i) - inputAccessor.getTupleStartOffset(i);
- memMgr.allocate(tLength, allocationPtr);
- while (allocationPtr.isNull()) {
- int unAllocSize = -1;
- while (unAllocSize < tLength) {
- unAllocSize = outputRecord();
- if (unAllocSize < 1) {
- throw new HyracksDataException(
- "Unable to allocate space for the new tuple, while there is no more tuple to output");
- }
- }
- memMgr.allocate(tLength, allocationPtr);
- }
-
- int pnk = getPNK(inputAccessor, i, bufferArray);
- int runId = getRunId(inputAccessor, i);
- if (runId != curRunId) { // tuple belongs to the next run
- memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
- int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
- sTree.insert(entry);
- nextRunSize++;
- continue;
- }
- // belongs to the current run
- if (curRunSize < outputLimit) {
- memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
- int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
- sTree.insert(entry);
- curRunSize++;
- continue;
- }
-
- sTree.peekMax(peek);
- if (compareRecords(inputAccessor, i, peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]) > 0) {
- continue;
- }
- // replacing the max
- sTree.getMax(peek);
- discard.set(peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]);
- memMgr.unallocate(discard);
- memMgr.writeTuple(allocationPtr.getFrameIx(), allocationPtr.getOffset(), inputAccessor, i);
- int[] entry = new int[] { runId, allocationPtr.getFrameIx(), allocationPtr.getOffset(), pnk };
- sTree.insert(entry);
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- }
-
- @Override
- public void close() throws HyracksDataException {
- while (!sTree.isEmpty()) { // Outputting remaining elements in the
- // selectionTree
- outputRecordForClose();
- }
-
- if (outputAppender.getTupleCount() > 0) { // Writing out very last
- // resident records to file
- FrameUtils.flushFrame(outputBuffer, writer);
- }
-
- writer.close();
- runs.add(writer.createReader());
- memMgr.close();
- }
-
- public List<IFrameReader> getRuns() {
- return runs;
- }
-
- private int outputRecord() throws HyracksDataException {
- outputedTuple.clear();
- sTree.getMin(sTreeTop);
- if (!isEntryValid(sTreeTop)) {
- throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
- }
- int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
- int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
- if (sTreeTop[SortMinMaxHeap.RUN_ID_IX] == curRunId) {
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can
- // not
- // append
- // to
- // the
- // tupleAppender
- FrameUtils.flushFrame(outputBuffer, writer);
- outputAppender.reset(outputBuffer, true);
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
- throw new HyracksDataException("Can not append to the ouput buffer in sort");
- }
- lastTupleIx = 0;
- } else {
- lastTupleIx++;
- }
- outputedTuple.set(tFrameIx, tOffset);
- newRun = false;
- return memMgr.unallocate(outputedTuple);
- }
- // Minimum belongs to the next Run
- openNewRun();
- int popCount = curRunSize - outputLimit;
- int l = 0;
- int maxFreedSpace = 0;
- for (int p = 0; p < popCount; p++) {
- sTree.getMax(peek);
- if (!isEntryValid(peek)) {
- throw new HyracksDataException("Invalid Maximum extracted from MinMaxHeap");
- }
- discard.set(peek[SortMinMaxHeap.FRAME_IX], peek[SortMinMaxHeap.OFFSET_IX]);
- l = memMgr.unallocate(discard);
- if (l > maxFreedSpace) {
- maxFreedSpace = l;
- }
- curRunSize--;
- }
-
- if (maxFreedSpace != 0) {
- return maxFreedSpace;
- }
- // No max discarded (We just flushed out the prev run, so the output
- // buffer should be clear)
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
- // append to
- // the
- // tupleAppender
- throw new HyracksDataException("Can not append to the ouput buffer in sort");
- }
- lastTupleIx = 0;
- outputedTuple.set(tFrameIx, tOffset);
- newRun = false;
- return memMgr.unallocate(outputedTuple);
- }
-
- private void outputRecordForClose() throws HyracksDataException {
- sTree.getMin(sTreeTop);
- if (!isEntryValid(sTreeTop)) {
- throw new HyracksDataException("Invalid outputed tuple (Top of the selection tree is invalid)");
- }
- int tFrameIx = sTreeTop[SortMinHeap.FRAME_IX];
- int tOffset = sTreeTop[SortMinHeap.OFFSET_IX];
- if (sTreeTop[SortMinMaxHeap.RUN_ID_IX] != curRunId) {
- openNewRun();
- }
-
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) { // Can not
- // append to
- // the
- // tupleAppender
- FrameUtils.flushFrame(outputBuffer, writer);
- outputAppender.reset(outputBuffer, true);
- if (!memMgr.readTuple(tFrameIx, tOffset, outputAppender)) {
- throw new HyracksDataException("Can not append to the ouput buffer in sort");
- }
- }
- }
-
- private int getPNK(FrameTupleAccessor fta, int tIx, byte[] buffInArray) {
- // Moved buffInArray out for better performance (not converting for each and every tuple)
- int sfIdx = sortFields[0];
- int tStart = fta.getTupleStartOffset(tIx);
- int f0StartRel = fta.getFieldStartOffset(tIx, sfIdx);
- int f0EndRel = fta.getFieldEndOffset(tIx, sfIdx);
- int f0Start = f0StartRel + tStart + fta.getFieldSlotsLength();
- return (nkc == null ? 0 : nkc.normalize(buffInArray, f0Start, f0EndRel - f0StartRel));
- }
-
- private int getRunId(FrameTupleAccessor fta, int tupIx) throws HyracksDataException {
- // Comparing current record to last output record, it decides about current record's runId
- if (newRun) { // Very first record for a new run
- return curRunId;
- }
-
- byte[] lastRecBuff = outputBuffer.array();
- lastRecordAccessor.reset(outputBuffer);
- int lastStartOffset = lastRecordAccessor.getTupleStartOffset(lastTupleIx);
-
- ByteBuffer fr2 = fta.getBuffer();
- byte[] curRecBuff = fr2.array();
- int r2StartOffset = fta.getTupleStartOffset(tupIx);
-
- for (int f = 0; f < comparators.length; ++f) {
- int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : outputBuffer.getInt(lastStartOffset + (fIdx - 1) * 4);
- int f1End = outputBuffer.getInt(lastStartOffset + fIdx * 4);
- int s1 = lastStartOffset + lastRecordAccessor.getFieldSlotsLength() + f1Start;
- int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
- int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
- int s2 = r2StartOffset + fta.getFieldSlotsLength() + f2Start;
- int l2 = f2End - f2Start;
- int c = comparators[f].compare(lastRecBuff, s1, l1, curRecBuff, s2, l2);
- if (c != 0) {
- if (c <= 0) {
- return curRunId;
- } else {
- return (curRunId + 1);
- }
- }
- }
- return curRunId;
- }
-
- // first<sec : -1
- private int compareRecords(FrameTupleAccessor fta1, int ix1, int fix2, int offset2) throws HyracksDataException {
- ByteBuffer buff1 = fta1.getBuffer();
- byte[] recBuff1 = buff1.array();
- int offset1 = fta1.getTupleStartOffset(ix1);
-
- offset2 += BSTNodeUtil.HEADER_SIZE;
- ByteBuffer buff2 = memMgr.getFrame(fix2);
- fta2.reset(buff2);
- byte[] recBuff2 = buff2.array();
-
- for (int f = 0; f < comparators.length; ++f) {
- int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : buff1.getInt(offset1 + (fIdx - 1) * 4);
- int f1End = buff1.getInt(offset1 + fIdx * 4);
- int s1 = offset1 + fta1.getFieldSlotsLength() + f1Start;
- int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : buff2.getInt(offset2 + (fIdx - 1) * 4);
- int f2End = buff2.getInt(offset2 + fIdx * 4);
- int s2 = offset2 + fta2.getFieldSlotsLength() + f2Start;
- int l2 = f2End - f2Start;
- int c = comparators[f].compare(recBuff1, s1, l1, recBuff2, s2, l2);
-
- if (c != 0) {
- return c;
- }
- }
- return 0;
-
- }
-
- private void openNewRun() throws HyracksDataException {
- if (writer != null) { // There is a prev run, so flush its tuples and
- // close it first
- if (outputAppender.getTupleCount() > 0) {
- FrameUtils.flushFrame(outputBuffer, writer);
- }
- outputAppender.reset(outputBuffer, true);
- writer.close();
- runs.add(writer.createReader());
- }
-
- FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
- ExternalSortRunGenerator.class.getSimpleName());
- writer = new RunFileWriter(file, ctx.getIOManager());
- writer.open();
- curRunId++;
- newRun = true;
- curRunSize = nextRunSize;
- nextRunSize = 0;
- lastTupleIx = -1;
- }
-
- private boolean isEntryValid(int[] entry) {
- return ((entry[SortMinHeap.RUN_ID_IX] > -1) && (entry[SortMinHeap.FRAME_IX] > -1) && (entry[SortMinHeap.OFFSET_IX] > -1));
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
new file mode 100644
index 0000000..c68f1e7
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunAndMaxFrameSizePair.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+
+public class RunAndMaxFrameSizePair {
+ public IFrameReader run;
+ public int maxFrameSize;
+
+ public RunAndMaxFrameSizePair(IFrameReader run, int maxFrameSize) {
+ this.run = run;
+ this.maxFrameSize = maxFrameSize;
+ }
+
+ void updateSize(int newMaxSize){
+ this.maxFrameSize = newMaxSize;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
index cf0d0ad..f013594 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/RunMergingFrameReader.java
@@ -1,23 +1,23 @@
/*
* Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
+ * 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.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package edu.uci.ics.hyracks.dataflow.std.sort;
-import java.nio.ByteBuffer;
import java.util.Comparator;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -25,52 +25,61 @@
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupFrameAccessor;
import edu.uci.ics.hyracks.dataflow.std.util.ReferenceEntry;
import edu.uci.ics.hyracks.dataflow.std.util.ReferencedPriorityQueue;
public class RunMergingFrameReader implements IFrameReader {
private final IHyracksTaskContext ctx;
- private final IFrameReader[] runCursors;
- private final List<ByteBuffer> inFrames;
+ private final List<? extends IFrameReader> runCursors;
+ private final List<? extends IFrame> inFrames;
private final int[] sortFields;
private final IBinaryComparator[] comparators;
private final INormalizedKeyComputer nmkComputer;
private final RecordDescriptor recordDesc;
- private final FrameTupleAppender outFrameAppender;
+ private final int topK;
+ private int tupleCount;
+ private FrameTupleAppender outFrameAppender;
private ReferencedPriorityQueue topTuples;
private int[] tupleIndexes;
- private FrameTupleAccessor[] tupleAccessors;
+ private IFrameTupleAccessor[] tupleAccessors;
- public RunMergingFrameReader(IHyracksTaskContext ctx, IFrameReader[] runCursors, List<ByteBuffer> inFrames,
- int[] sortFields, IBinaryComparator[] comparators, INormalizedKeyComputer nmkComputer,
- RecordDescriptor recordDesc) {
+ public RunMergingFrameReader(IHyracksTaskContext ctx, List<? extends IFrameReader> runs,
+ List<? extends IFrame> inFrames, int[] sortFields, IBinaryComparator[] comparators,
+ INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc) {
+ this(ctx, runs, inFrames, sortFields, comparators, nmkComputer, recordDesc, Integer.MAX_VALUE);
+ }
+
+ public RunMergingFrameReader(IHyracksTaskContext ctx, List<? extends IFrameReader> runs,
+ List<? extends IFrame> inFrames, int[] sortFields, IBinaryComparator[] comparators,
+ INormalizedKeyComputer nmkComputer, RecordDescriptor recordDesc, int topK) {
this.ctx = ctx;
- this.runCursors = runCursors;
+ this.runCursors = runs;
this.inFrames = inFrames;
this.sortFields = sortFields;
this.comparators = comparators;
this.nmkComputer = nmkComputer;
this.recordDesc = recordDesc;
- outFrameAppender = new FrameTupleAppender(ctx.getFrameSize());
+ this.topK = topK;
}
@Override
public void open() throws HyracksDataException {
- tupleAccessors = new FrameTupleAccessor[runCursors.length];
+ tupleCount = 0;
+ tupleAccessors = new IFrameTupleAccessor[runCursors.size()];
+ outFrameAppender = new FrameTupleAppender();
Comparator<ReferenceEntry> comparator = createEntryComparator(comparators);
- topTuples = new ReferencedPriorityQueue(ctx.getFrameSize(), recordDesc, runCursors.length, comparator,
- sortFields, nmkComputer);
- tupleIndexes = new int[runCursors.length];
- for (int i = 0; i < runCursors.length; i++) {
+ topTuples = new ReferencedPriorityQueue(runCursors.size(), comparator, sortFields, nmkComputer);
+ tupleIndexes = new int[runCursors.size()];
+ for (int i = 0; i < runCursors.size(); i++) {
tupleIndexes[i] = 0;
int runIndex = topTuples.peek().getRunid();
- runCursors[runIndex].open();
- if (runCursors[runIndex].nextFrame(inFrames.get(runIndex))) {
- tupleAccessors[runIndex] = new FrameTupleAccessor(ctx.getFrameSize(), recordDesc);
- tupleAccessors[runIndex].reset(inFrames.get(runIndex));
- setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+ runCursors.get(runIndex).open();
+ if (runCursors.get(runIndex).nextFrame(inFrames.get(runIndex))) {
+ tupleAccessors[runIndex] = new GroupFrameAccessor(ctx.getInitialFrameSize(), recordDesc);
+ tupleAccessors[runIndex].reset(inFrames.get(runIndex).getBuffer());
+ setNextTopTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors, topTuples);
} else {
closeRun(runIndex, runCursors, tupleAccessors);
topTuples.pop();
@@ -79,20 +88,21 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
- outFrameAppender.reset(buffer, true);
- while (!topTuples.areRunsExhausted()) {
+ public boolean nextFrame(IFrame outFrame) throws HyracksDataException {
+ outFrameAppender.reset(outFrame, true);
+ while (!topTuples.areRunsExhausted() && tupleCount < topK) {
ReferenceEntry top = topTuples.peek();
int runIndex = top.getRunid();
- FrameTupleAccessor fta = top.getAccessor();
+ IFrameTupleAccessor fta = top.getAccessor();
int tupleIndex = top.getTupleIndex();
if (!outFrameAppender.append(fta, tupleIndex)) {
return true;
+ } else {
+ tupleCount++;
}
-
++tupleIndexes[runIndex];
- setNextTopTuple(runIndex, tupleIndexes, runCursors, tupleAccessors, topTuples);
+ setNextTopTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors, topTuples);
}
if (outFrameAppender.getTupleCount() > 0) {
@@ -103,14 +113,15 @@
@Override
public void close() throws HyracksDataException {
- for (int i = 0; i < runCursors.length; ++i) {
+ for (int i = 0; i < runCursors.size(); ++i) {
closeRun(i, runCursors, tupleAccessors);
}
}
- private void setNextTopTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
- FrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples) throws HyracksDataException {
- boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+ private static void setNextTopTuple(int runIndex, int[] tupleIndexes, List<? extends IFrameReader> runCursors,
+ List<? extends IFrame> inFrames, IFrameTupleAccessor[] tupleAccessors, ReferencedPriorityQueue topTuples)
+ throws HyracksDataException {
+ boolean exists = hasNextTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors);
if (exists) {
topTuples.popAndReplace(tupleAccessors[runIndex], tupleIndexes[runIndex]);
} else {
@@ -119,15 +130,16 @@
}
}
- private boolean hasNextTuple(int runIndex, int[] tupleIndexes, IFrameReader[] runCursors,
- FrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
- if (tupleAccessors[runIndex] == null || runCursors[runIndex] == null) {
+ private static boolean hasNextTuple(int runIndex, int[] tupleIndexes, List<? extends IFrameReader> runCursors,
+ List<? extends IFrame> inFrames, IFrameTupleAccessor[] tupleAccessors) throws HyracksDataException {
+ if (tupleAccessors[runIndex] == null || runCursors.get(runIndex) == null) {
return false;
} else if (tupleIndexes[runIndex] >= tupleAccessors[runIndex].getTupleCount()) {
- ByteBuffer buf = tupleAccessors[runIndex].getBuffer(); // same-as-inFrames.get(runIndex)
- if (runCursors[runIndex].nextFrame(buf)) {
+ IFrame frame = inFrames.get(runIndex);
+ if (runCursors.get(runIndex).nextFrame(frame)) {
tupleIndexes[runIndex] = 0;
- return hasNextTuple(runIndex, tupleIndexes, runCursors, tupleAccessors);
+ tupleAccessors[runIndex].reset(frame.getBuffer());
+ return hasNextTuple(runIndex, tupleIndexes, runCursors, inFrames, tupleAccessors);
} else {
return false;
}
@@ -136,11 +148,12 @@
}
}
- private void closeRun(int index, IFrameReader[] runCursors, IFrameTupleAccessor[] tupleAccessors)
+ private static void closeRun(int index, List<? extends IFrameReader> runCursors,
+ IFrameTupleAccessor[] tupleAccessors)
throws HyracksDataException {
- if (runCursors[index] != null) {
- runCursors[index].close();
- runCursors[index] = null;
+ if (runCursors.get(index) != null) {
+ runCursors.get(index).close();
+ runCursors.set(index, null);
tupleAccessors[index] = null;
}
}
@@ -153,8 +166,8 @@
if (nmk1 != nmk2) {
return ((((long) nmk1) & 0xffffffffL) < (((long) nmk2) & 0xffffffffL)) ? -1 : 1;
}
- FrameTupleAccessor fta1 = (FrameTupleAccessor) tp1.getAccessor();
- FrameTupleAccessor fta2 = (FrameTupleAccessor) tp2.getAccessor();
+ IFrameTupleAccessor fta1 = tp1.getAccessor();
+ IFrameTupleAccessor fta2 = tp2.getAccessor();
byte[] b1 = fta1.getBuffer().array();
byte[] b2 = fta2.getBuffer().array();
int[] tPointers1 = tp1.getTPointers();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
deleted file mode 100644
index 73f99dd..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/Slot.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-/**
- * @author pouria
- * Defines a slot in the memory, which can be a free or used (allocated)
- * slot. Memory is a set of frames, ordered as a list. Each tuple is
- * stored in a slot, where the location of the slot is denoted by a pair
- * of integers:
- * - The index of the frame, in the list of frames in memory. (referred
- * to as frameIx)
- * - The starting offset of the slot, within that specific frame.
- * (referred to as offset)
- */
-public class Slot {
-
- private int frameIx;
- private int offset;
-
- public Slot() {
- this.frameIx = BSTNodeUtil.INVALID_INDEX;
- this.offset = BSTNodeUtil.INVALID_INDEX;
- }
-
- public Slot(int frameIx, int offset) {
- this.frameIx = frameIx;
- this.offset = offset;
- }
-
- public void set(int frameIx, int offset) {
- this.frameIx = frameIx;
- this.offset = offset;
- }
-
- public int getFrameIx() {
- return frameIx;
- }
-
- public void setFrameIx(int frameIx) {
- this.frameIx = frameIx;
- }
-
- public int getOffset() {
- return offset;
- }
-
- public void setOffset(int offset) {
- this.offset = offset;
- }
-
- public boolean isNull() {
- return (frameIx == BSTNodeUtil.INVALID_INDEX) || (offset == BSTNodeUtil.INVALID_INDEX);
- }
-
- public void clear() {
- this.frameIx = BSTNodeUtil.INVALID_INDEX;
- this.offset = BSTNodeUtil.INVALID_INDEX;
- }
-
- public void copy(Slot s) {
- this.frameIx = s.getFrameIx();
- this.offset = s.getOffset();
- }
-
- public String toString() {
- return "(" + frameIx + ", " + offset + ")";
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
deleted file mode 100644
index 1cde75f..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinHeap.java
+++ /dev/null
@@ -1,293 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-/**
- * @author pouria
- * Implements a minimum binary heap, used as selection tree, for sort
- * with replacement. This heap structure can only be used as the min
- * heap (no access to the max element). Elements in the heap are
- * compared based on their run numbers, and sorting key(s):
- * Considering two heap elements A and B:
- * if RunNumber(A) > RunNumber(B) then A is larger than B if
- * RunNumber(A) == RunNumber(B), then A is smaller than B, if and only
- * if the value of the sort key(s) in B is greater than A (based on the
- * sort comparator).
- */
-public class SortMinHeap implements ISelectionTree {
-
- static final int RUN_ID_IX = 0;
- static final int FRAME_IX = 1;
- static final int OFFSET_IX = 2;
- private static final int PNK_IX = 3;
- private static final int ELEMENT_SIZE = 4;
- private static final int INIT_ARRAY_SIZE = 512;
-
- private final int[] sortFields;
- private final IBinaryComparator[] comparators;
- private final RecordDescriptor recordDescriptor;
- private final FrameTupleAccessor fta1;
- private final FrameTupleAccessor fta2;
- private int[] elements;
- private int nextIx;
- private final IMemoryManager memMgr;
- private int[] top; // Used as a temp variable to access the top, to avoid object creation
-
- public SortMinHeap(IHyracksCommonContext ctx, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, IMemoryManager memMgr) {
- this.sortFields = sortFields;
- this.comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- this.comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- this.recordDescriptor = recordDesc;
- fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- this.memMgr = memMgr;
- this.top = new int[ELEMENT_SIZE];
- Arrays.fill(top, -1);
- this.elements = new int[INIT_ARRAY_SIZE];
- Arrays.fill(elements, -1);
- this.nextIx = 0;
- }
-
- /*
- * Assumption (element structure): [RunId][FrameIx][Offset][Poorman NK]
- */
- @Override
- public void getMin(int[] result) throws HyracksDataException {
- if (nextIx == 0) {
- result[0] = result[1] = result[2] = result[3] = -1;
- return;
- }
-
- top = delete(0);
- for (int i = 0; i < top.length; i++) {
- result[i] = top[i];
- }
- }
-
- @Override
- public void peekMin(int[] result) {
- if (nextIx == 0) {
- result[0] = result[1] = result[2] = result[3] = -1;
- return;
- }
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- result[i] = elements[i];
- }
- }
-
- @Override
- public void insert(int[] e) throws HyracksDataException {
- if (nextIx >= elements.length) {
- elements = Arrays.copyOf(elements, elements.length * 2);
- }
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- elements[nextIx + i] = e[i];
- }
- siftUp(nextIx);
- nextIx += ELEMENT_SIZE;
-
- }
-
- @Override
- public void reset() {
- Arrays.fill(elements, -1);
- nextIx = 0;
- }
-
- @Override
- public boolean isEmpty() {
- return (nextIx < ELEMENT_SIZE);
- }
-
- public int _debugGetSize() {
- return (nextIx > 0 ? (nextIx - 1) / 4 : 0);
- }
-
- private int[] delete(int nix) throws HyracksDataException {
- int[] nv = Arrays.copyOfRange(elements, nix, nix + ELEMENT_SIZE);
- int[] lastElem = removeLast();
-
- if (nextIx == 0) {
- return nv;
- }
-
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- elements[nix + i] = lastElem[i];
- }
- int pIx = getParent(nix);
- if (pIx > -1 && (compare(lastElem, Arrays.copyOfRange(elements, pIx, pIx + ELEMENT_SIZE)) < 0)) {
- siftUp(nix);
- } else {
- siftDown(nix);
- }
- return nv;
- }
-
- private int[] removeLast() {
- if (nextIx < ELEMENT_SIZE) { //this is the very last element
- return new int[] { -1, -1, -1, -1 };
- }
- int[] l = Arrays.copyOfRange(elements, nextIx - ELEMENT_SIZE, nextIx);
- Arrays.fill(elements, nextIx - ELEMENT_SIZE, nextIx, -1);
- nextIx -= ELEMENT_SIZE;
- return l;
- }
-
- private void siftUp(int nodeIx) throws HyracksDataException {
- int p = getParent(nodeIx);
- if (p < 0) {
- return;
- }
- while (p > -1 && (compare(nodeIx, p) < 0)) {
- swap(p, nodeIx);
- nodeIx = p;
- p = getParent(nodeIx);
- if (p < 0) { // We are at the root
- return;
- }
- }
- }
-
- private void siftDown(int nodeIx) throws HyracksDataException {
- int mix = getMinOfChildren(nodeIx);
- if (mix < 0) {
- return;
- }
- while (mix > -1 && (compare(mix, nodeIx) < 0)) {
- swap(mix, nodeIx);
- nodeIx = mix;
- mix = getMinOfChildren(nodeIx);
- if (mix < 0) { // We hit the leaf level
- return;
- }
- }
- }
-
- // first < sec : -1
- private int compare(int nodeSIx1, int nodeSIx2) throws HyracksDataException {
- int[] n1 = Arrays.copyOfRange(elements, nodeSIx1, nodeSIx1 + ELEMENT_SIZE);
- int[] n2 = Arrays.copyOfRange(elements, nodeSIx2, nodeSIx2 + ELEMENT_SIZE);
- return (compare(n1, n2));
- }
-
- // first < sec : -1
- private int compare(int[] n1, int[] n2) throws HyracksDataException {
- // Compare Run Numbers
- if (n1[RUN_ID_IX] != n2[RUN_ID_IX]) {
- return (n1[RUN_ID_IX] < n2[RUN_ID_IX] ? -1 : 1);
- }
-
- // Compare Poor man Normalized Keys
- if (n1[PNK_IX] != n2[PNK_IX]) {
- return ((((long) n1[PNK_IX]) & 0xffffffffL) < (((long) n2[PNK_IX]) & 0xffffffffL)) ? -1 : 1;
- }
-
- return compare(getFrame(n1[FRAME_IX]), getFrame(n2[FRAME_IX]), n1[OFFSET_IX], n2[OFFSET_IX]);
- }
-
- private int compare(ByteBuffer fr1, ByteBuffer fr2, int r1StartOffset, int r2StartOffset)
- throws HyracksDataException {
- byte[] b1 = fr1.array();
- byte[] b2 = fr2.array();
- fta1.reset(fr1);
- fta2.reset(fr2);
- int headerLen = BSTNodeUtil.HEADER_SIZE;
- r1StartOffset += headerLen;
- r2StartOffset += headerLen;
- for (int f = 0; f < comparators.length; ++f) {
- int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : fr1.getInt(r1StartOffset + (fIdx - 1) * 4);
- int f1End = fr1.getInt(r1StartOffset + fIdx * 4);
- int s1 = r1StartOffset + fta1.getFieldSlotsLength() + f1Start;
- int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
- int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
- int s2 = r2StartOffset + fta2.getFieldSlotsLength() + f2Start;
- int l2 = f2End - f2Start;
-
- int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-
- if (c != 0) {
- return c;
- }
- }
- return 0;
- }
-
- private int getMinOfChildren(int nix) throws HyracksDataException { // returns index of min child
- int lix = getLeftChild(nix);
- if (lix < 0) {
- return -1;
- }
- int rix = getRightChild(nix);
- if (rix < 0) {
- return lix;
- }
- return ((compare(lix, rix) < 0) ? lix : rix);
- }
-
- //Assumption: n1Ix and n2Ix are starting indices of two elements
- private void swap(int n1Ix, int n2Ix) {
- int[] temp = Arrays.copyOfRange(elements, n1Ix, n1Ix + ELEMENT_SIZE);
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- elements[n1Ix + i] = elements[n2Ix + i];
- elements[n2Ix + i] = temp[i];
- }
- }
-
- private int getLeftChild(int ix) {
- int lix = (2 * ELEMENT_SIZE) * (ix / ELEMENT_SIZE) + ELEMENT_SIZE;
- return ((lix < nextIx) ? lix : -1);
- }
-
- private int getRightChild(int ix) {
- int rix = (2 * ELEMENT_SIZE) * (ix / ELEMENT_SIZE) + (2 * ELEMENT_SIZE);
- return ((rix < nextIx) ? rix : -1);
- }
-
- private int getParent(int ix) {
- if (ix <= 0) {
- return -1;
- }
- return ((ix - ELEMENT_SIZE) / (2 * ELEMENT_SIZE)) * ELEMENT_SIZE;
- }
-
- private ByteBuffer getFrame(int frameIx) {
- return (memMgr.getFrame(frameIx));
- }
-
- @Override
- public void getMax(int[] result) {
- throw new IllegalStateException("getMax() method not applicable to Min Heap");
- }
-
- @Override
- public void peekMax(int[] result) {
- throw new IllegalStateException("getMax() method not applicable to Min Heap");
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
deleted file mode 100644
index 12aa8a1..0000000
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/SortMinMaxHeap.java
+++ /dev/null
@@ -1,448 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.dataflow.std.sort;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-
-/**
- * @author pouria
- * Implements a MinMax binary heap, used as the selection tree, in
- * sorting with replacement. Check SortMinHeap for details on comparing
- * elements.
- */
-public class SortMinMaxHeap implements ISelectionTree {
- static final int RUN_ID_IX = 0;
- static final int FRAME_IX = 1;
- static final int OFFSET_IX = 2;
- private static final int PNK_IX = 3;
- private static final int NOT_EXIST = -1;
- private static final int ELEMENT_SIZE = 4;
- private static final int INIT_ARRAY_SIZE = 512;
-
- private final int[] sortFields;
- private final IBinaryComparator[] comparators;
- private final RecordDescriptor recordDescriptor;
- private final FrameTupleAccessor fta1;
- private final FrameTupleAccessor fta2;
-
- private int[] elements;
- private int nextIx;
-
- private final IMemoryManager memMgr;
-
- public SortMinMaxHeap(IHyracksCommonContext ctx, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
- RecordDescriptor recordDesc, IMemoryManager memMgr) {
- this.sortFields = sortFields;
- this.comparators = new IBinaryComparator[comparatorFactories.length];
- for (int i = 0; i < comparatorFactories.length; ++i) {
- this.comparators[i] = comparatorFactories[i].createBinaryComparator();
- }
- this.recordDescriptor = recordDesc;
- fta1 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- fta2 = new FrameTupleAccessor(ctx.getFrameSize(), recordDescriptor);
- this.memMgr = memMgr;
- this.elements = new int[INIT_ARRAY_SIZE];
- Arrays.fill(elements, -1);
- this.nextIx = 0;
- }
-
- @Override
- public void insert(int[] element) throws HyracksDataException {
- if (nextIx >= elements.length) {
- elements = Arrays.copyOf(elements, elements.length * 2);
- }
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- elements[nextIx + i] = element[i];
- }
- nextIx += ELEMENT_SIZE;
- bubbleUp(nextIx - ELEMENT_SIZE);
- }
-
- @Override
- public void getMin(int[] result) throws HyracksDataException {
- if (nextIx == 0) {
- result[0] = result[1] = result[2] = result[3] = -1;
- return;
- }
-
- int[] topElem = delete(0);
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- result[x] = topElem[x];
- }
- }
-
- @Override
- public void reset() {
- Arrays.fill(elements, -1);
- nextIx = 0;
- }
-
- @Override
- public boolean isEmpty() {
- return (nextIx < ELEMENT_SIZE);
- }
-
- @Override
- public void peekMin(int[] result) {
- if (nextIx == 0) {
- result[0] = result[1] = result[2] = result[3] = -1;
- return;
- }
-
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- result[x] = elements[x];
- }
- }
-
- @Override
- public void getMax(int[] result) throws HyracksDataException {
- if (nextIx == ELEMENT_SIZE) {
- int[] topElement = removeLast();
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- result[x] = topElement[x];
- }
- return;
- }
-
- if (nextIx > ELEMENT_SIZE) {
- int lc = getLeftChild(0);
- int rc = getRightChild(0);
- int maxIx = lc;
-
- if (rc != -1) {
- maxIx = compare(lc, rc) < 0 ? rc : lc;
- }
-
- int[] maxElem = delete(maxIx);
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- result[x] = maxElem[x];
- }
- return;
- }
-
- result[0] = result[1] = result[2] = result[3] = -1;
-
- }
-
- @Override
- public void peekMax(int[] result) throws HyracksDataException {
- if (nextIx == ELEMENT_SIZE) {
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- result[i] = elements[i];
- }
- return;
- }
- if (nextIx > ELEMENT_SIZE) {
- int lc = getLeftChild(0);
- int rc = getRightChild(0);
- int maxIx = lc;
-
- if (rc != -1) {
- maxIx = compare(lc, rc) < 0 ? rc : lc;
- }
-
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- result[x] = elements[maxIx + x];
- }
-
- return;
- }
- result[0] = result[1] = result[2] = result[3] = -1;
- }
-
- private int[] delete(int delIx) throws HyracksDataException {
- int s = nextIx;
- if (nextIx > ELEMENT_SIZE) {
- int[] delEntry = Arrays.copyOfRange(elements, delIx, delIx + ELEMENT_SIZE);
- int[] last = removeLast();
- if (delIx != (s - ELEMENT_SIZE)) {
- for (int x = 0; x < ELEMENT_SIZE; x++) {
- elements[delIx + x] = last[x];
- }
- trickleDown(delIx);
- }
- return delEntry;
- } else if (nextIx == ELEMENT_SIZE) {
- return (removeLast());
- }
- return null;
- }
-
- private int[] removeLast() {
- if (nextIx < ELEMENT_SIZE) { //this is the very last element
- return new int[] { -1, -1, -1, -1 };
- }
- int[] l = Arrays.copyOfRange(elements, nextIx - ELEMENT_SIZE, nextIx);
- Arrays.fill(elements, nextIx - ELEMENT_SIZE, nextIx, -1);
- nextIx -= ELEMENT_SIZE;
- return l;
- }
-
- private void bubbleUp(int ix) throws HyracksDataException {
- int p = getParentIx(ix);
- if (isAtMinLevel(ix)) {
- if (p != NOT_EXIST && compare(p, ix) < 0) {
- swap(ix, p);
- bubbleUpMax(p);
- } else {
- bubbleUpMin(ix);
- }
- } else { // i is at max level
- if (p != NOT_EXIST && compare(ix, p) < 0) {
- swap(ix, p);
- bubbleUpMin(p);
- } else {
- bubbleUpMax(ix);
- }
- }
- }
-
- private void bubbleUpMax(int ix) throws HyracksDataException {
- int gp = getGrandParent(ix);
- if (gp != NOT_EXIST && compare(gp, ix) < 0) {
- swap(ix, gp);
- bubbleUpMax(gp);
- }
- }
-
- private void bubbleUpMin(int ix) throws HyracksDataException {
- int gp = getGrandParent(ix);
- if (gp != NOT_EXIST && compare(ix, gp) < 0) {
- swap(ix, gp);
- bubbleUpMin(gp);
- }
- }
-
- private void trickleDown(int ix) throws HyracksDataException {
- if (isAtMinLevel(ix)) {
- trickleDownMin(ix);
- } else {
- trickleDownMax(ix);
- }
- }
-
- private void trickleDownMax(int ix) throws HyracksDataException {
- int maxIx = getMaxOfDescendents(ix);
- if (maxIx == NOT_EXIST) {
- return;
- }
- if (maxIx > getLeftChild(ix) && maxIx > getRightChild(ix)) { // A grand
- // children
- if (compare(ix, maxIx) < 0) {
- swap(maxIx, ix);
- int p = getParentIx(maxIx);
- if (p != NOT_EXIST && compare(maxIx, p) < 0) {
- swap(maxIx, p);
- }
- trickleDownMax(maxIx);
- }
- } else { // A children
- if (compare(ix, maxIx) < 0) {
- swap(ix, maxIx);
- }
- }
- }
-
- private void trickleDownMin(int ix) throws HyracksDataException {
- int minIx = getMinOfDescendents(ix);
- if (minIx == NOT_EXIST) {
- return;
- }
- if (minIx > getLeftChild(ix) && minIx > getRightChild(ix)) { // A grand
- // children
- if (compare(minIx, ix) < 0) {
- swap(minIx, ix);
- int p = getParentIx(minIx);
- if (p != NOT_EXIST && compare(p, minIx) < 0) {
- swap(minIx, p);
- }
- trickleDownMin(minIx);
- }
- } else { // A children
- if (compare(minIx, ix) < 0) {
- swap(ix, minIx);
- }
- }
- }
-
- // Min among children and grand children
- private int getMinOfDescendents(int ix) throws HyracksDataException {
- int lc = getLeftChild(ix);
- if (lc == NOT_EXIST) {
- return NOT_EXIST;
- }
- int rc = getRightChild(ix);
- if (rc == NOT_EXIST) {
- return lc;
- }
- int min = (compare(lc, rc) < 0) ? lc : rc;
- int[] lgc = getLeftGrandChildren(ix);
- int[] rgc = getRightGrandChildren(ix);
- for (int k = 0; k < 2; k++) {
- if (lgc[k] != NOT_EXIST && compare(lgc[k], min) < 0) {
- min = lgc[k];
- }
- if (rgc[k] != NOT_EXIST && compare(rgc[k], min) < 0) {
- min = rgc[k];
- }
- }
- return min;
- }
-
- // Max among children and grand children
- private int getMaxOfDescendents(int ix) throws HyracksDataException {
- int lc = getLeftChild(ix);
- if (lc == NOT_EXIST) {
- return NOT_EXIST;
- }
- int rc = getRightChild(ix);
- if (rc == NOT_EXIST) {
- return lc;
- }
- int max = (compare(lc, rc) < 0) ? rc : lc;
- int[] lgc = getLeftGrandChildren(ix);
- int[] rgc = getRightGrandChildren(ix);
- for (int k = 0; k < 2; k++) {
- if (lgc[k] != NOT_EXIST && compare(max, lgc[k]) < 0) {
- max = lgc[k];
- }
- if (rgc[k] != NOT_EXIST && compare(max, rgc[k]) < 0) {
- max = rgc[k];
- }
- }
- return max;
- }
-
- private void swap(int n1Ix, int n2Ix) {
- int[] temp = Arrays.copyOfRange(elements, n1Ix, n1Ix + ELEMENT_SIZE);
- for (int i = 0; i < ELEMENT_SIZE; i++) {
- elements[n1Ix + i] = elements[n2Ix + i];
- elements[n2Ix + i] = temp[i];
- }
- }
-
- private int getParentIx(int i) {
- if (i < ELEMENT_SIZE) {
- return NOT_EXIST;
- }
- return ((i - ELEMENT_SIZE) / (2 * ELEMENT_SIZE)) * ELEMENT_SIZE;
- }
-
- private int getGrandParent(int i) {
- int p = getParentIx(i);
- return p != -1 ? getParentIx(p) : NOT_EXIST;
- }
-
- private int getLeftChild(int i) {
- int lc = (2 * ELEMENT_SIZE) * (i / ELEMENT_SIZE) + ELEMENT_SIZE;
- return (lc < nextIx ? lc : -1);
- }
-
- private int[] getLeftGrandChildren(int i) {
- int lc = getLeftChild(i);
- return lc != NOT_EXIST ? new int[] { getLeftChild(lc), getRightChild(lc) } : new int[] { NOT_EXIST, NOT_EXIST };
- }
-
- private int getRightChild(int i) {
- int rc = (2 * ELEMENT_SIZE) * (i / ELEMENT_SIZE) + (2 * ELEMENT_SIZE);
- return (rc < nextIx ? rc : -1);
- }
-
- private int[] getRightGrandChildren(int i) {
- int rc = getRightChild(i);
- return rc != NOT_EXIST ? new int[] { getLeftChild(rc), getRightChild(rc) } : new int[] { NOT_EXIST, NOT_EXIST };
- }
-
- private boolean isAtMinLevel(int i) {
- int l = getLevel(i);
- return l % 2 == 0 ? true : false;
- }
-
- private int getLevel(int i) {
- if (i < ELEMENT_SIZE) {
- return 0;
- }
-
- int cnv = i / ELEMENT_SIZE;
- int l = (int) Math.floor(Math.log(cnv) / Math.log(2));
- if (cnv == (((int) Math.pow(2, (l + 1))) - 1)) {
- return (l + 1);
- }
- return l;
- }
-
- private ByteBuffer getFrame(int frameIx) {
- return (memMgr.getFrame(frameIx));
- }
-
- // first < sec : -1
- private int compare(int nodeSIx1, int nodeSIx2) throws HyracksDataException {
- int[] n1 = Arrays.copyOfRange(elements, nodeSIx1, nodeSIx1 + ELEMENT_SIZE); //tree.get(nodeSIx1);
- int[] n2 = Arrays.copyOfRange(elements, nodeSIx2, nodeSIx2 + ELEMENT_SIZE); //tree.get(nodeSIx2);
- return (compare(n1, n2));
- }
-
- // first < sec : -1
- private int compare(int[] n1, int[] n2) throws HyracksDataException {
- // Compare Run Numbers
- if (n1[RUN_ID_IX] != n2[RUN_ID_IX]) {
- return (n1[RUN_ID_IX] < n2[RUN_ID_IX] ? -1 : 1);
- }
-
- // Compare Poor man Normalized Keys
- if (n1[PNK_IX] != n2[PNK_IX]) {
- return ((((long) n1[PNK_IX]) & 0xffffffffL) < (((long) n2[PNK_IX]) & 0xffffffffL)) ? -1 : 1;
- }
-
- return compare(getFrame(n1[FRAME_IX]), getFrame(n2[FRAME_IX]), n1[OFFSET_IX], n2[OFFSET_IX]);
- }
-
- private int compare(ByteBuffer fr1, ByteBuffer fr2, int r1StartOffset, int r2StartOffset)
- throws HyracksDataException {
- byte[] b1 = fr1.array();
- byte[] b2 = fr2.array();
- fta1.reset(fr1);
- fta2.reset(fr2);
- int headerLen = BSTNodeUtil.HEADER_SIZE;
- r1StartOffset += headerLen;
- r2StartOffset += headerLen;
- for (int f = 0; f < comparators.length; ++f) {
- int fIdx = sortFields[f];
- int f1Start = fIdx == 0 ? 0 : fr1.getInt(r1StartOffset + (fIdx - 1) * 4);
- int f1End = fr1.getInt(r1StartOffset + fIdx * 4);
- int s1 = r1StartOffset + fta1.getFieldSlotsLength() + f1Start;
- int l1 = f1End - f1Start;
- int f2Start = fIdx == 0 ? 0 : fr2.getInt(r2StartOffset + (fIdx - 1) * 4);
- int f2End = fr2.getInt(r2StartOffset + fIdx * 4);
- int s2 = r2StartOffset + fta2.getFieldSlotsLength() + f2Start;
- int l2 = f2End - f2Start;
-
- int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
-
- if (c != 0) {
- return c;
- }
- }
- return 0;
- }
-}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
new file mode 100644
index 0000000..ee43993
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TopKSorterOperatorDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.value.*;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+
+import java.util.List;
+
+public class TopKSorterOperatorDescriptor extends AbstractSorterOperatorDescriptor {
+
+ private final int topK;
+
+ public TopKSorterOperatorDescriptor(IOperatorDescriptorRegistry spec, int framesLimit, int topK, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) {
+ super(spec, framesLimit, sortFields, firstKeyNormalizerFactory, comparatorFactories, recordDescriptor);
+ this.topK = topK;
+ }
+
+ @Override
+ public SortActivity getSortActivity(ActivityId id) {
+ return new SortActivity(id) {
+ @Override
+ protected AbstractSortRunGenerator getRunGenerator(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider) {
+ return new HybridTopKSortRunGenerator(ctx, framesLimit, topK, sortFields, firstKeyNormalizerFactory,
+ comparatorFactories, recordDescriptors[0]);
+
+ }
+ };
+ }
+
+ @Override
+ public MergeActivity getMergeActivity(ActivityId id) {
+ return new MergeActivity(id) {
+ @Override
+ protected ExternalSortRunMerger getSortRunMerger(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, IFrameWriter writer, ISorter sorter, List<RunAndMaxFrameSizePair> runs, IBinaryComparator[] comparators,
+ INormalizedKeyComputer nmkComputer, int necessaryFrames) {
+ return new ExternalSortRunMerger(ctx, sorter, runs, sortFields, comparators,
+ nmkComputer, recordDescriptors[0], necessaryFrames, topK, writer);
+ }
+ };
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
new file mode 100644
index 0000000..8f8518c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -0,0 +1,269 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+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.sort.buffermanager.ITupleBufferAccessor;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparable;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparableFactory;
+import edu.uci.ics.hyracks.dataflow.std.structures.MaxHeap;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class TupleSorterHeapSort implements ITupleSorter {
+
+ private static final Logger LOGGER = Logger.getLogger(TupleSorterHeapSort.class.getName());
+
+ class HeapEntryFactory implements IResetableComparableFactory<HeapEntry> {
+ @Override
+ public IResetableComparable<HeapEntry> createResetableComparable() {
+ return new HeapEntry();
+ }
+ }
+
+ class HeapEntry implements IResetableComparable<HeapEntry> {
+ int nmk;
+ TuplePointer tuplePointer;
+
+ public HeapEntry() {
+ tuplePointer = new TuplePointer();
+ nmk = 0;
+ }
+
+ @Override
+ public int compareTo(HeapEntry o) {
+ if (nmk != o.nmk) {
+ return ((((long) nmk) & 0xffffffffL) < (((long) o.nmk) & 0xffffffffL)) ? -1 : 1;
+ }
+ bufferAccessor1.reset(tuplePointer);
+ bufferAccessor2.reset(o.tuplePointer);
+ byte[] b1 = bufferAccessor1.getTupleBuffer().array();
+ byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+
+ for (int f = 0; f < comparators.length; ++f) {
+ int fIdx = sortFields[f];
+ int s1 = bufferAccessor1.getAbsFieldStartOffset(fIdx);
+ int l1 = bufferAccessor1.getFieldLength(fIdx);
+
+ int s2 = bufferAccessor2.getAbsFieldStartOffset(fIdx);
+ int l2 = bufferAccessor2.getFieldLength(fIdx);
+ int c;
+ try {
+ c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ if (c != 0) {
+ return c;
+ }
+ }
+ return 0;
+ }
+
+ public void reset(int nmkey) {
+ nmk = nmkey;
+ }
+
+ @Override
+ public void reset(HeapEntry other) {
+ nmk = other.nmk;
+ tuplePointer.reset(other.tuplePointer);
+ }
+ }
+
+ private final ITupleBufferManager bufferManager;
+ private final ITupleBufferAccessor bufferAccessor1;
+ private final ITupleBufferAccessor bufferAccessor2;
+ private final int topK;
+ private final FrameTupleAppender outputAppender;
+ private final IFrame outputFrame;
+ private final int[] sortFields;
+ private final INormalizedKeyComputer nkc;
+ private final IBinaryComparator[] comparators;
+
+ private HeapEntry maxEntry;
+ private HeapEntry newEntry;
+
+ private MaxHeap heap;
+ private boolean isSorted;
+
+ public TupleSorterHeapSort(IHyracksTaskContext ctx, ITupleBufferManager bufferManager, int topK, int[] sortFields,
+ INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories)
+ throws HyracksDataException {
+ this.bufferManager = bufferManager;
+ this.bufferAccessor1 = bufferManager.getTupleAccessor();
+ this.bufferAccessor2 = bufferManager.getTupleAccessor();
+ this.topK = topK;
+ this.outputFrame = new VSizeFrame(ctx);
+ this.outputAppender = new FrameTupleAppender();
+ this.sortFields = sortFields;
+ this.nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory.createNormalizedKeyComputer();
+ this.comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+
+ this.heap = new MaxHeap(new HeapEntryFactory(), topK);
+ this.maxEntry = new HeapEntry();
+ this.newEntry = new HeapEntry();
+ this.isSorted = false;
+ }
+
+ @Override
+ public int getTupleCount() {
+ return heap.getNumEntries();
+ }
+
+ @Override
+ public boolean insertTuple(IFrameTupleAccessor frameTupleAccessor, int index) throws HyracksDataException {
+ if (isSorted) {
+ throw new HyracksDataException(
+ "The Heap haven't be reset after sorting, the order of using this class is not correct.");
+ }
+ int nmkey = getPNK(frameTupleAccessor, index);
+ if (heap.getNumEntries() >= topK) {
+ heap.peekMax(maxEntry);
+ if (compareTuple(frameTupleAccessor, index, nmkey, maxEntry) >= 0) {
+ return true;
+ }
+ }
+
+ newEntry.reset(nmkey);
+ if (!bufferManager.insertTuple(frameTupleAccessor, index, newEntry.tuplePointer)) {
+ return false;
+ }
+ if (heap.getNumEntries() < topK) {
+ heap.insert(newEntry);
+ } else {
+ bufferManager.deleteTuple(maxEntry.tuplePointer);
+ heap.replaceMax(newEntry);
+ }
+ return true;
+ }
+
+ private int getPNK(IFrameTupleAccessor fta, int tIx) {
+ if (nkc == null) {
+ return 0;
+ }
+ int sfIdx = sortFields[0];
+ return nkc.normalize(fta.getBuffer().array(), fta.getAbsoluteFieldStartOffset(tIx, sfIdx),
+ fta.getFieldLength(tIx, sfIdx));
+ }
+
+ private int compareTuple(IFrameTupleAccessor frameTupleAccessor, int tid, int nmkey, HeapEntry maxEntry)
+ throws HyracksDataException {
+ if (nmkey != maxEntry.nmk) {
+ return ((((long) nmkey) & 0xffffffffL) < (((long) maxEntry.nmk) & 0xffffffffL)) ? -1 : 1;
+ }
+ bufferAccessor2.reset(maxEntry.tuplePointer);
+ byte[] b1 = frameTupleAccessor.getBuffer().array();
+ byte[] b2 = bufferAccessor2.getTupleBuffer().array();
+
+ for (int f = 0; f < comparators.length; ++f) {
+ int fIdx = sortFields[f];
+ int s1 = frameTupleAccessor.getAbsoluteFieldStartOffset(tid, fIdx);
+ int l1 = frameTupleAccessor.getFieldLength(tid, fIdx);
+
+ int s2 = bufferAccessor2.getAbsFieldStartOffset(fIdx);
+ int l2 = bufferAccessor2.getFieldLength(fIdx);
+ int c = comparators[f].compare(b1, s1, l1, b2, s2, l2);
+ if (c != 0) {
+ return c;
+ }
+ }
+ return 0;
+ }
+
+ @Override
+ public boolean hasRemaining() {
+ return getTupleCount() > 0;
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ bufferManager.reset();
+ heap.reset();
+ isSorted = false;
+ }
+
+ @Override
+ @SuppressWarnings("deprecation")
+ public void sort() throws HyracksDataException {
+ IResetableComparable[] entries = heap.getEntries();
+ int count = heap.getNumEntries();
+ Arrays.sort(entries, 0, count, entryComparator);
+ isSorted = true;
+ }
+
+ private static final Comparator<IResetableComparable> entryComparator = new Comparator<IResetableComparable>() {
+ @Override
+ public int compare(IResetableComparable o1, IResetableComparable o2) {
+ return o1.compareTo(o2);
+ }
+ };
+
+ @Override
+ public void close() {
+ heap = null;
+ bufferManager.close();
+ isSorted = false;
+ }
+
+ @Override
+ @SuppressWarnings("deprecation")
+ public int flush(IFrameWriter writer) throws HyracksDataException {
+ outputAppender.reset(outputFrame, true);
+ int maxFrameSize = outputFrame.getFrameSize();
+ int numEntries = heap.getNumEntries();
+ IResetableComparable[] entries = heap.getEntries();
+ int io = 0;
+ for (int i = 0; i < numEntries; i++) {
+ HeapEntry minEntry = (HeapEntry) entries[i];
+ bufferAccessor1.reset(minEntry.tuplePointer);
+ int flushed = FrameUtils
+ .appendToWriter(writer, outputAppender, bufferAccessor1.getTupleBuffer().array(),
+ bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
+ if (flushed > 0) {
+ maxFrameSize = Math.max(maxFrameSize, flushed);
+ io++;
+ }
+ }
+ maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
+ outputAppender.flush(writer, true);
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info(
+ "Flushed records:" + numEntries + "; Flushed through " + (io + 1) + " frames");
+ }
+ return maxFrameSize;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
new file mode 100644
index 0000000..26da494
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/EnumFreeSlotPolicy.java
@@ -0,0 +1,22 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+public enum EnumFreeSlotPolicy {
+ SMALLEST_FIT,
+ LAST_FIT,
+ BIGGEST_FIT,
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
new file mode 100644
index 0000000..085a1e8
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirst.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparable;
+import edu.uci.ics.hyracks.dataflow.std.structures.IResetableComparableFactory;
+import edu.uci.ics.hyracks.dataflow.std.structures.MaxHeap;
+
+public class FrameFreeSlotBiggestFirst implements IFrameFreeSlotPolicy {
+ private static final int INVALID = -1;
+
+ class SpaceEntryFactory implements IResetableComparableFactory {
+ @Override
+ public IResetableComparable createResetableComparable() {
+ return new SpaceEntry();
+ }
+ }
+
+ class SpaceEntry implements IResetableComparable<SpaceEntry> {
+ int space;
+ int id;
+
+ SpaceEntry() {
+ space = INVALID;
+ id = INVALID;
+ }
+
+ @Override
+ public int compareTo(SpaceEntry o) {
+ if (o.space != space) {
+ if (o.space == INVALID) {
+ return 1;
+ }
+ if (space == INVALID) {
+ return -1;
+ }
+ return space < o.space ? -1 : 1;
+ }
+ return 0;
+ }
+
+ @Override
+ public void reset(SpaceEntry other) {
+ space = other.space;
+ id = other.id;
+ }
+
+ void reset(int space, int id) {
+ this.space = space;
+ this.id = id;
+ }
+ }
+
+ private MaxHeap heap;
+ private SpaceEntry tempEntry;
+
+ public FrameFreeSlotBiggestFirst(int initialCapacity) {
+ heap = new MaxHeap(new SpaceEntryFactory(), initialCapacity);
+ tempEntry = new SpaceEntry();
+ }
+
+ @Override
+ public int popBestFit(int tobeInsertedSize) {
+ if (!heap.isEmpty()) {
+ heap.peekMax(tempEntry);
+ if (tempEntry.space >= tobeInsertedSize) {
+ heap.getMax(tempEntry);
+ return tempEntry.id;
+ }
+ }
+ return -1;
+ }
+
+ @Override
+ public void pushNewFrame(int frameID, int freeSpace) {
+ tempEntry.reset(freeSpace, frameID);
+ heap.insert(tempEntry);
+ }
+
+ @Override
+ public void reset() {
+ heap.reset();
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
new file mode 100644
index 0000000..0bfcf38
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFit.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.util.Arrays;
+
+public class FrameFreeSlotLastFit implements IFrameFreeSlotPolicy {
+ private static int INITIAL_CAPACITY = 10;
+
+ private class FrameSpace {
+ int frameId;
+ int freeSpace;
+
+ FrameSpace(int frameId, int freeSpace) {
+ reset(frameId, freeSpace);
+ }
+
+ void reset(int frameId, int freeSpace) {
+ this.frameId = frameId;
+ this.freeSpace = freeSpace;
+ }
+ }
+
+ private FrameSpace[] frameSpaces;
+ private int size;
+
+ public FrameFreeSlotLastFit(int maxFrames) {
+ frameSpaces = new FrameSpace[maxFrames];
+ size = 0;
+ }
+
+ public FrameFreeSlotLastFit() {
+ this(INITIAL_CAPACITY);
+ }
+
+ @Override
+ public int popBestFit(int tobeInsertedSize) {
+ for (int i = size - 1; i >= 0; i--) {
+ if (frameSpaces[i].freeSpace >= tobeInsertedSize) {
+ FrameSpace ret = frameSpaces[i];
+ System.arraycopy(frameSpaces, i + 1, frameSpaces, i, size - i - 1);
+ frameSpaces[--size] = ret;
+ return ret.frameId;
+ }
+ }
+ return -1;
+ }
+
+ @Override
+ public void pushNewFrame(int frameID, int freeSpace) {
+ if (size >= frameSpaces.length) {
+ frameSpaces = Arrays.copyOf(frameSpaces, size * 2);
+ }
+ if (frameSpaces[size] == null) {
+ frameSpaces[size++] = new FrameSpace(frameID, freeSpace);
+ } else {
+ frameSpaces[size++].reset(frameID, freeSpace);
+ }
+ }
+
+ @Override
+ public void reset() {
+ size = 0;
+ for (int i = frameSpaces.length - 1; i >= 0; i--) {
+ frameSpaces[i] = null;
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
new file mode 100644
index 0000000..69e1911
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotSmallestFit.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.TreeMap;
+
+public class FrameFreeSlotSmallestFit implements IFrameFreeSlotPolicy {
+
+ private TreeMap<Integer, LinkedList<Integer>> freeSpaceIndex;
+
+ public FrameFreeSlotSmallestFit() {
+ freeSpaceIndex = new TreeMap<>();
+ }
+
+ @Override
+ public int popBestFit(int tobeInsertedSize) {
+ Map.Entry<Integer, LinkedList<Integer>> entry = freeSpaceIndex.ceilingEntry(tobeInsertedSize);
+ if (entry == null) {
+ return -1;
+ }
+ int id = entry.getValue().removeFirst();
+ if (entry.getValue().isEmpty()) {
+ freeSpaceIndex.remove(entry.getKey());
+ }
+ return id;
+ }
+
+ @Override
+ public void pushNewFrame(int frameID, int freeSpace) {
+ Map.Entry<Integer, LinkedList<Integer>> entry = freeSpaceIndex.ceilingEntry(freeSpace);
+ if (entry == null || entry.getKey() != freeSpace) {
+ LinkedList<Integer> linkedList = new LinkedList<>();
+ linkedList.add(frameID);
+ freeSpaceIndex.put(freeSpace, linkedList);
+ } else {
+ entry.getValue().add(frameID);
+ }
+ }
+
+ @Override
+ public void reset() {
+ freeSpaceIndex.clear();
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
new file mode 100644
index 0000000..9a52efa
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameBufferManager.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFrameBufferManager {
+
+ /**
+ * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+ *
+ * @throws edu.uci.ics.hyracks.api.exceptions.HyracksDataException
+ */
+ void reset() throws HyracksDataException;
+
+ /**
+ * @param frameIndex
+ * @return the specified frame, from the set of memory buffers, being
+ * managed by this memory manager
+ */
+ ByteBuffer getFrame(int frameIndex);
+
+ /**
+ * Get the startOffset of the specific frame inside buffer
+ *
+ * @param frameIndex
+ * @return the start offset of the frame returned by {@link #getFrame(int)} method.
+ */
+ int getFrameStartOffset(int frameIndex);
+
+ /**
+ * Get the size of the specific frame inside buffer
+ *
+ * @param frameIndex
+ * @return the length of the specific frame
+ */
+ int getFrameSize(int frameIndex);
+
+ /**
+ * @return the number of frames in this buffer
+ */
+ int getNumFrames();
+
+ /**
+ * Writes the whole frame into the buffer.
+ *
+ * @param frame source frame
+ * @return the id of the inserted frame. if failed to return it will be -1.
+ */
+ int insertFrame(ByteBuffer frame) throws HyracksDataException;
+
+ void close();
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
new file mode 100644
index 0000000..57a8094
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFrameFreeSlotPolicy.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+public interface IFrameFreeSlotPolicy {
+
+ /**
+ * Find the best fit frame id which can hold the data, and then pop it out from the index.
+ * Return -1 is failed to find any.
+ *
+ * @param tobeInsertedSize the actual size of the data which should include
+ * the meta data like the field offset and the tuple
+ * count extra size
+ * @return the best fit frame id
+ */
+ int popBestFit(int tobeInsertedSize);
+
+ /**
+ * Register the new free slot into the index
+ *
+ * @param frameID
+ * @param freeSpace
+ */
+ void pushNewFrame(int frameID, int freeSpace);
+
+ /**
+ * Clear all the existing free slot information.
+ */
+ void reset();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
new file mode 100644
index 0000000..1e5be25
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/IFramePool.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IFramePool {
+
+ int getMinFrameSize();
+
+ int getMemoryBudgetBytes();
+
+ /**
+ * Get a frame of given size. <br>
+ * Returns {@code null} if failed to allocate the required size of frame
+ *
+ * @param frameSize the actual size of the frame.
+ * @return the allocated frame
+ * @throws HyracksDataException
+ */
+ ByteBuffer allocateFrame(int frameSize) throws HyracksDataException;
+
+ /**
+ * Reset the counters to initial status. This method should not release the pre-allocated resources.
+ */
+ void reset();
+
+ /**
+ * Release the pre-allocated resources.
+ */
+ void close();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
new file mode 100644
index 0000000..49a664c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferAccessor.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface ITupleBufferAccessor {
+
+ void reset(TuplePointer tuplePointer);
+
+ ByteBuffer getTupleBuffer();
+
+ int getTupleStartOffset();
+
+ int getTupleLength();
+
+ int getAbsFieldStartOffset(int fieldId);
+
+ int getFieldLength(int fieldId);
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
new file mode 100644
index 0000000..6f94563
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface ITupleBufferManager {
+ /**
+ * Reset the counters and flags to initial status. This method should not release the pre-allocated resources
+ *
+ * @throws edu.uci.ics.hyracks.api.exceptions.HyracksDataException
+ */
+ void reset() throws HyracksDataException;
+
+ /**
+ * @return the number of tuples in this buffer
+ */
+ int getNumTuples();
+
+ boolean insertTuple(IFrameTupleAccessor accessor, int idx, TuplePointer tuplePointer) throws HyracksDataException;
+
+ void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException;
+
+ void close();
+
+ ITupleBufferAccessor getTupleAccessor();
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
new file mode 100644
index 0000000..834ba2a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFrameMemoryManager.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+public class VariableFrameMemoryManager implements IFrameBufferManager {
+
+ private class PhysicalFrameOffset {
+ IFrame physicalFrame;
+ int physicalOffset;
+
+ PhysicalFrameOffset(IFrame frame, int offset) {
+ physicalFrame = frame;
+ physicalOffset = offset;
+ }
+ }
+
+ private class LogicalFrameStartSize {
+ ByteBuffer logicalFrame;
+ int logicalStart;
+ int logicalSize;
+
+ LogicalFrameStartSize(ByteBuffer frame, int start, int size) {
+ logicalFrame = frame;
+ logicalStart = start;
+ logicalSize = size;
+ }
+ }
+
+ private final IFramePool framePool;
+ private List<PhysicalFrameOffset> physicalFrameOffsets;
+ private List<LogicalFrameStartSize> logicalFrameStartSizes;
+ private final IFrameFreeSlotPolicy freeSlotPolicy;
+
+ public VariableFrameMemoryManager(IFramePool framePool, IFrameFreeSlotPolicy freeSlotPolicy) {
+ this.framePool = framePool;
+ this.freeSlotPolicy = freeSlotPolicy;
+ int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
+ this.physicalFrameOffsets = new ArrayList<>(maxFrames);
+ this.logicalFrameStartSizes = new ArrayList<>(maxFrames);
+ }
+
+ private int findAvailableFrame(int frameSize) throws HyracksDataException {
+ int frameId = freeSlotPolicy.popBestFit(frameSize);
+ if (frameId >= 0) {
+ return frameId;
+ }
+ ByteBuffer buffer = framePool.allocateFrame(frameSize);
+ if (buffer != null) {
+ IntSerDeUtils.putInt(buffer.array(), FrameHelper.getTupleCountOffset(buffer.capacity()), 0);
+ physicalFrameOffsets.add(new PhysicalFrameOffset(new FixedSizeFrame(buffer), 0));
+ return physicalFrameOffsets.size() - 1;
+ }
+ return -1;
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ physicalFrameOffsets.clear();
+ logicalFrameStartSizes.clear();
+ freeSlotPolicy.reset();
+ framePool.reset();
+ }
+
+ @Override
+ public ByteBuffer getFrame(int frameIndex) {
+ return logicalFrameStartSizes.get(frameIndex).logicalFrame;
+ }
+
+ @Override
+ public int getFrameStartOffset(int frameIndex) {
+ return logicalFrameStartSizes.get(frameIndex).logicalStart;
+ }
+
+ @Override
+ public int getFrameSize(int frameIndex) {
+ return logicalFrameStartSizes.get(frameIndex).logicalSize;
+ }
+
+ @Override
+ public int getNumFrames() {
+ return logicalFrameStartSizes.size();
+ }
+
+ @Override
+ public int insertFrame(ByteBuffer frame) throws HyracksDataException {
+ int frameSize = frame.capacity();
+ int physicalFrameId = findAvailableFrame(frameSize);
+ if (physicalFrameId < 0) {
+ return -1;
+ }
+ ByteBuffer buffer = physicalFrameOffsets.get(physicalFrameId).physicalFrame.getBuffer();
+ int offset = physicalFrameOffsets.get(physicalFrameId).physicalOffset;
+ System.arraycopy(frame.array(), 0, buffer.array(), offset, frameSize);
+ if (offset + frameSize < buffer.capacity()) {
+ freeSlotPolicy.pushNewFrame(physicalFrameId, buffer.capacity() - offset - frameSize);
+ }
+ physicalFrameOffsets.get(physicalFrameId).physicalOffset = offset + frameSize;
+ logicalFrameStartSizes.add(new LogicalFrameStartSize(buffer, offset, frameSize));
+ return logicalFrameStartSizes.size() - 1;
+ }
+
+ @Override
+ public void close() {
+ physicalFrameOffsets.clear();
+ logicalFrameStartSizes.clear();
+ freeSlotPolicy.reset();
+ framePool.close();
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
new file mode 100644
index 0000000..0d936b4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePool.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.context.IHyracksFrameMgrContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class VariableFramePool implements IFramePool {
+ public static final int UNLIMITED_MEMORY = -1;
+
+ private final IHyracksFrameMgrContext ctx;
+ private final int minFrameSize;
+ private final int memBudget;
+
+ private int allocateMem;
+ private ArrayList<ByteBuffer> buffers; // the unused slots were sorted by size increasingly.
+ private BitSet used; // the merged one also marked as used.
+
+ /**
+ * The constructor of the VariableFramePool.
+ *
+ * @param ctx
+ * @param memBudgetInBytes the given memory budgets to allocate the frames. If it less than 0, it will be treated as unlimited budgets
+ */
+ public VariableFramePool(IHyracksFrameMgrContext ctx, int memBudgetInBytes) {
+ this.ctx = ctx;
+ this.minFrameSize = ctx.getInitialFrameSize();
+ this.allocateMem = 0;
+ if (memBudgetInBytes == UNLIMITED_MEMORY) {
+ this.memBudget = Integer.MAX_VALUE;
+ this.buffers = new ArrayList<>();
+ this.used = new BitSet();
+ } else {
+ this.memBudget = memBudgetInBytes;
+ this.buffers = new ArrayList<>(memBudgetInBytes / minFrameSize);
+ this.used = new BitSet(memBudgetInBytes / minFrameSize);
+ }
+ }
+
+ @Override
+ public int getMinFrameSize() {
+ return minFrameSize;
+ }
+
+ @Override
+ public int getMemoryBudgetBytes() {
+ return memBudget;
+ }
+
+ @Override
+ public ByteBuffer allocateFrame(int frameSize) throws HyracksDataException {
+ int frameId = findExistingFrame(frameSize);
+ if (frameId >= 0) {
+ return reuseFrame(frameId);
+ }
+ if (haveEnoughFreeSpace(frameSize)) {
+ return createNewFrame(frameSize);
+ }
+ return mergeExistingFrames(frameSize);
+
+ }
+
+ private boolean haveEnoughFreeSpace(int frameSize) {
+ return frameSize + allocateMem <= memBudget;
+ }
+
+ private static int getFirstUnUsedPos(BitSet used) {
+ return used.nextClearBit(0);
+ }
+
+ private static int getLastUnUsedPos(BitSet used, int lastPos) {
+ return used.previousClearBit(lastPos);
+ }
+
+ private static int binarySearchUnusedBuffer(ArrayList<ByteBuffer> buffers, BitSet used, int frameSize) {
+ int l = getFirstUnUsedPos(used); // to skip the merged null buffers
+ int h = getLastUnUsedPos(used, (buffers.size() - 1)) + 1; // to skip the newly created buffers
+ if (l >= h) {
+ return -1;
+ }
+ int highest = h;
+ int mid = (l + h) / 2;
+ while (l < h) {
+ ByteBuffer buffer = buffers.get(mid);
+ if (buffer.capacity() == frameSize) {
+ break;
+ }
+ if (buffer.capacity() < frameSize) {
+ l = mid + 1;
+ } else {
+ h = mid;
+ }
+ mid = (l + h) / 2;
+ }
+ mid = used.nextClearBit(mid);
+ return mid < highest ? mid : -1;
+ }
+
+ private int findExistingFrame(int frameSize) {
+ return binarySearchUnusedBuffer(buffers, used, frameSize);
+ }
+
+ private ByteBuffer reuseFrame(int id) {
+ used.set(id);
+ buffers.get(id).clear();
+ return buffers.get(id);
+ }
+
+ private ByteBuffer createNewFrame(int frameSize) throws HyracksDataException {
+ buffers.add(ctx.allocateFrame(frameSize));
+ allocateMem += frameSize;
+ return reuseFrame(buffers.size() - 1);
+ }
+
+ /**
+ * The merging sequence is from the smallest to the largest order.
+ * Once the buffer get merged, it will be remove from the list in order to free the object.
+ * And the index spot of it will be marked as used.
+ *
+ * @param frameSize
+ * @return
+ * @throws HyracksDataException
+ */
+ private ByteBuffer mergeExistingFrames(int frameSize) throws HyracksDataException {
+ int mergedSize = memBudget - allocateMem;
+ int highBound = getLastUnUsedPos(used, buffers.size() - 1) + 1;
+ for (int i = getFirstUnUsedPos(used); i < highBound; ++i) {
+ if (!used.get(i)) {
+ mergedSize += deAllocateFrame(i);
+ if (mergedSize >= frameSize) {
+ return createNewFrame(mergedSize);
+ }
+ }
+ }
+ return null;
+ }
+
+ private int deAllocateFrame(int id) {
+ ByteBuffer frame = buffers.get(id);
+ ctx.deallocateFrames(frame.capacity());
+ buffers.set(id, null);
+ used.set(id);
+ allocateMem -= frame.capacity();
+ return frame.capacity();
+ }
+
+ @Override
+ public void reset() {
+ removeEmptySpot(buffers);
+ Collections.sort(buffers, sizeByteBufferComparator);
+ used.clear();
+ }
+
+ private static void removeEmptySpot(List<ByteBuffer> buffers) {
+ for (int i = 0; i < buffers.size(); ) {
+ if (buffers.get(i) == null) {
+ buffers.remove(i);
+ } else {
+ i++;
+ }
+ }
+ }
+
+ @Override
+ public void close() {
+ buffers.clear();
+ used.clear();
+ allocateMem = 0;
+ }
+
+ private static Comparator<ByteBuffer> sizeByteBufferComparator = new Comparator<ByteBuffer>() {
+ @Override
+ public int compare(ByteBuffer o1, ByteBuffer o2) {
+ if (o1.capacity() == o2.capacity()) {
+ return 0;
+ }
+ return o1.capacity() < o2.capacity() ? -1 : 1;
+ }
+ };
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
new file mode 100644
index 0000000..0b077c2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
@@ -0,0 +1,203 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.DeletableFrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.IAppendDeletableFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class VariableTupleMemoryManager implements ITupleBufferManager {
+
+ private final static Logger LOG = Logger.getLogger(VariableTupleMemoryManager.class.getName());
+
+ private final int MIN_FREE_SPACE;
+ private final IFramePool pool;
+ private final IFrameFreeSlotPolicy policy;
+ private final IAppendDeletableFrameTupleAccessor accessor;
+ private final ArrayList<ByteBuffer> frames;
+ private final RecordDescriptor recordDescriptor;
+ private int numTuples;
+ private int statsReOrg;
+
+ public VariableTupleMemoryManager(IFramePool framePool, RecordDescriptor recordDescriptor) {
+ this.pool = framePool;
+ int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
+ this.policy = new FrameFreeSlotLastFit(maxFrames);
+ this.accessor = new DeletableFrameTupleAppender(recordDescriptor);
+ this.frames = new ArrayList<>();
+ this.MIN_FREE_SPACE = calculateMinFreeSpace(recordDescriptor);
+ this.recordDescriptor = recordDescriptor;
+ this.numTuples = 0;
+ this.statsReOrg = 0;
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ pool.reset();
+ policy.reset();
+ frames.clear();
+ numTuples = 0;
+ }
+
+ @Override
+ public int getNumTuples() {
+ return numTuples;
+ }
+
+ @Override
+ public boolean insertTuple(IFrameTupleAccessor fta, int idx, TuplePointer tuplePointer)
+ throws HyracksDataException {
+ int requiredFreeSpace = calculatePhysicalSpace(fta, idx);
+ int frameId = findAvailableFrame(requiredFreeSpace);
+ if (frameId < 0) {
+ if (canBeInsertedAfterCleanUpFragmentation(requiredFreeSpace)) {
+ reOrganizeFrames();
+ frameId = findAvailableFrame(requiredFreeSpace);
+ statsReOrg++;
+ } else {
+ return false;
+ }
+ }
+ assert frameId >= 0;
+ accessor.reset(frames.get(frameId));
+ assert accessor.getContiguousFreeSpace() >= requiredFreeSpace;
+ int tid = accessor.append(fta, idx);
+ assert tid >= 0;
+ tuplePointer.reset(frameId, tid);
+ if (accessor.getContiguousFreeSpace() > MIN_FREE_SPACE) {
+ policy.pushNewFrame(frameId, accessor.getContiguousFreeSpace());
+ }
+ numTuples++;
+ return true;
+ }
+
+ private void reOrganizeFrames() {
+ policy.reset();
+ for (int i = 0; i < frames.size(); i++) {
+ accessor.reset(frames.get(i));
+ accessor.reOrganizeBuffer();
+ policy.pushNewFrame(i, accessor.getContiguousFreeSpace());
+ }
+ }
+
+ private boolean canBeInsertedAfterCleanUpFragmentation(int requiredFreeSpace) {
+ for (int i = 0; i < frames.size(); i++) {
+ accessor.reset(frames.get(i));
+ if (accessor.getTotalFreeSpace() >= requiredFreeSpace) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private int findAvailableFrame(int requiredFreeSpace) throws HyracksDataException {
+ int frameId = policy.popBestFit(requiredFreeSpace);
+ if (frameId >= 0) {
+ return frameId;
+ }
+
+ int frameSize = calculateMinFrameSizeToPlaceTuple(requiredFreeSpace, pool.getMinFrameSize());
+ ByteBuffer buffer = pool.allocateFrame(frameSize);
+ if (buffer != null) {
+ accessor.clear(buffer);
+ frames.add(buffer);
+ return frames.size() - 1;
+ }
+ return -1;
+ }
+
+ private static int calculateMinFrameSizeToPlaceTuple(int requiredFreeSpace, int minFrameSize) {
+ return (1 + (requiredFreeSpace + 4 - 1) / minFrameSize) * minFrameSize;
+ }
+
+ private static int calculatePhysicalSpace(IFrameTupleAccessor fta, int idx) {
+ // 4 bytes to store the offset
+ return 4 + fta.getTupleLength(idx);
+ }
+
+ private static int calculateMinFreeSpace(RecordDescriptor recordDescriptor) {
+ // + 4 for the tuple offset
+ return recordDescriptor.getFieldCount() * 4 + 4;
+ }
+
+ @Override
+ public void deleteTuple(TuplePointer tuplePointer) throws HyracksDataException {
+ accessor.reset(frames.get(tuplePointer.frameIndex));
+ accessor.delete(tuplePointer.tupleIndex);
+ numTuples--;
+ }
+
+ @Override
+ public void close() {
+ pool.close();
+ policy.reset();
+ frames.clear();
+ numTuples = 0;
+ if (LOG.isLoggable(Level.FINE)) {
+ LOG.fine("VariableTupleMemoryManager has reorganized " + statsReOrg + " times");
+ }
+ statsReOrg = 0;
+ }
+
+ @Override
+ public ITupleBufferAccessor getTupleAccessor() {
+ return new ITupleBufferAccessor() {
+ private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
+ recordDescriptor);
+ private int tid;
+
+ @Override
+ public void reset(TuplePointer tuplePointer) {
+ bufferAccessor.reset(frames.get(tuplePointer.frameIndex));
+ tid = tuplePointer.tupleIndex;
+ }
+
+ @Override
+ public ByteBuffer getTupleBuffer() {
+ return bufferAccessor.getBuffer();
+ }
+
+ @Override
+ public int getTupleStartOffset() {
+ return bufferAccessor.getTupleStartOffset(tid);
+ }
+
+ @Override
+ public int getTupleLength() {
+ return bufferAccessor.getTupleLength(tid);
+ }
+
+ @Override
+ public int getAbsFieldStartOffset(int fieldId) {
+ return bufferAccessor.getAbsoluteFieldStartOffset(tid, fieldId);
+ }
+
+ @Override
+ public int getFieldLength(int fieldId) {
+ return bufferAccessor.getFieldLength(tid, fieldId);
+ }
+ };
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
new file mode 100644
index 0000000..9b03a77
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
@@ -0,0 +1,244 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+
+/**
+ * This is a special frame which is used in TupleMemoryBuffer.
+ * This frame has a special structure to organize the deleted spaces.
+ * Specifically, the endOffset of the deleted tuple will be set as negative number.
+ * And we add a special <code>deleted_space</code> field at the last 4 bytes to remember how many bytes has been deleted.
+ */
+public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAccessor {
+
+ private static final int SIZE_DELETED_SPACE = 4;
+ private final RecordDescriptor recordDescriptor;
+ private ByteBuffer buffer;
+ private int tupleCountOffset;
+ private int tupleCount;
+ private int freeDataEndOffset;
+ private int deletedSpace;
+ private byte[] array; // to speed up the array visit a little
+
+ public DeletableFrameTupleAppender(RecordDescriptor recordDescriptor) {
+ this.recordDescriptor = recordDescriptor;
+ }
+
+ private int getTupleCountOffset() {
+ return FrameHelper.getTupleCountOffset(buffer.capacity()) - SIZE_DELETED_SPACE;
+ }
+
+ private int getFreeDataEndOffset() {
+ return tupleCount == 0 ? 0 : Math.abs(IntSerDeUtils.getInt(array, tupleCountOffset - tupleCount * 4));
+ }
+
+ private void setFreeDataEndOffset(int offset) {
+ assert (offset >= 0);
+ IntSerDeUtils.putInt(array, tupleCountOffset - tupleCount * 4, offset);
+ }
+
+ private void setTupleCount(int count) {
+ IntSerDeUtils.putInt(array, tupleCountOffset, count);
+ }
+
+ private void setDeleteSpace(int count) {
+ IntSerDeUtils.putInt(array, buffer.capacity() - SIZE_DELETED_SPACE, count);
+ }
+
+ private int getPhysicalTupleCount() {
+ return IntSerDeUtils.getInt(array, tupleCountOffset);
+ }
+
+ private int getDeletedSpace() {
+ return IntSerDeUtils.getInt(array, buffer.capacity() - SIZE_DELETED_SPACE);
+ }
+
+ @Override
+ public void clear(ByteBuffer buffer) throws HyracksDataException {
+ this.buffer = buffer;
+ this.array = buffer.array();
+ tupleCountOffset = getTupleCountOffset();
+ setTupleCount(0);
+ setDeleteSpace(0);
+ resetCounts();
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ this.array = buffer.array();
+ tupleCountOffset = getTupleCountOffset();
+ resetCounts();
+ }
+
+ private void resetCounts() {
+ deletedSpace = getDeletedSpace();
+ tupleCount = getPhysicalTupleCount();
+ freeDataEndOffset = getFreeDataEndOffset();
+ }
+
+ /**
+ * Append the record into the frame. This method will not validate the space, please make sure space is enough
+ * by calling {@link #getContiguousFreeSpace()}
+ *
+ * @param tupleAccessor
+ * @param tIndex
+ * @return
+ * @throws HyracksDataException
+ */
+ @Override
+ public int append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException {
+ byte[] src = tupleAccessor.getBuffer().array();
+ int tStartOffset = tupleAccessor.getTupleStartOffset(tIndex);
+ int length = tupleAccessor.getTupleLength(tIndex);
+ System.arraycopy(src, tStartOffset, array, freeDataEndOffset, length);
+ setTupleCount(++tupleCount);
+ freeDataEndOffset += length;
+ setFreeDataEndOffset(freeDataEndOffset);
+ return tupleCount - 1;
+ }
+
+ @Override
+ public void delete(int tupleIndex) {
+ int endOffset = getTupleEndOffset(tupleIndex);
+ if (endOffset > 0) {
+ setTupleEndOffset(tupleIndex, -endOffset);
+ deletedSpace += endOffset - getTupleStartOffset(tupleIndex);
+ setDeleteSpace(deletedSpace);
+ }
+ }
+
+ @Override
+ public void reOrganizeBuffer() {
+ if (deletedSpace <= 0) {
+ return;
+ }
+ reclaimDeletedEnding();
+
+ freeDataEndOffset = 0;
+ int endOffset = 0;
+ for (int i = 0; i < tupleCount; i++) {
+ int startOffset = Math.abs(endOffset);
+ endOffset = getTupleEndOffset(i);
+ if (endOffset >= 0) {
+ int length = endOffset - startOffset;
+ assert ( length >= 0);
+ if (freeDataEndOffset != startOffset) {
+ System.arraycopy(array, startOffset, array, freeDataEndOffset, length);
+ }
+ freeDataEndOffset += length;
+ }
+ setTupleEndOffset(i, freeDataEndOffset);
+ }
+ setFreeDataEndOffset(freeDataEndOffset);
+ deletedSpace = 0;
+ setDeleteSpace(0);
+ }
+
+ private void reclaimDeletedEnding() {
+ for (int i = tupleCount - 1; i >= 0; i--) {
+ int endOffset = getTupleEndOffset(i);
+ if (endOffset < 0) {
+ tupleCount--;
+ } else {
+ break;
+ }
+ }
+ setTupleCount(tupleCount);
+ }
+
+ @Override
+ public int getTotalFreeSpace() {
+ return getContiguousFreeSpace() + deletedSpace;
+ }
+
+ @Override
+ public int getContiguousFreeSpace() {
+ return getTupleCountOffset() - tupleCount * 4 - freeDataEndOffset;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return recordDescriptor.getFieldCount();
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return recordDescriptor.getFieldCount() * 4;
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return IntSerDeUtils.getInt(array, getTupleStartOffset(tupleIndex) + fIdx * 4);
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return fIdx == 0 ? 0 : IntSerDeUtils.getInt(array, getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ int endOffset = getTupleEndOffset(tupleIndex);
+ if (endOffset < 0) {
+ return endOffset + getTupleStartOffset(tupleIndex);
+ }
+ return endOffset - getTupleStartOffset(tupleIndex);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return IntSerDeUtils.getInt(array, tupleCountOffset - 4 * (tupleIndex + 1));
+ }
+
+ private void setTupleEndOffset(int tupleIndex, int offset) {
+ IntSerDeUtils.putInt(array, tupleCountOffset - 4 * (tupleIndex + 1), offset);
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ int offset = tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(array, tupleCountOffset - 4 * tupleIndex);
+ return Math.abs(offset);
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return tupleCount;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java
new file mode 100644
index 0000000..12ba72f
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupFrameAccessor.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+/**
+ * This {@code GroupFrameAccessor} access a group of logical frames which are stored in one physical
+ * continuous ByteBuffer. It is used in a RunFileReader which can read several frames at once, and we
+ * can use this accessor to parse the returned data as one frame. In the caller's view there is only
+ * one frame which simply the caller's work.
+ */
+public class GroupFrameAccessor implements IFrameTupleAccessor {
+
+ private class InnerFrameInfo implements Comparable<Integer> {
+ int start;
+ int length;
+ int tupleCount;
+
+ InnerFrameInfo(int start, int length, int tupleCount) {
+ this.start = start;
+ this.length = length;
+ this.tupleCount = tupleCount;
+ }
+
+ @Override
+ public int compareTo(Integer o) {
+ return -o.compareTo(tupleCount);
+ }
+ }
+
+ private final RecordDescriptor recordDescriptor;
+ private final int minFrameSize;
+ private final FrameTupleAccessor frameTupleAccessor;
+ private int lastTupleIndex;
+ private int lastFrameId;
+ private ByteBuffer buffer;
+ private List<InnerFrameInfo> innerFrameInfos;
+
+ public GroupFrameAccessor(int minFrameSize, RecordDescriptor recordDescriptor) {
+ this.minFrameSize = minFrameSize;
+ this.recordDescriptor = (recordDescriptor);
+ this.frameTupleAccessor = new FrameTupleAccessor(recordDescriptor);
+ this.innerFrameInfos = new ArrayList<>();
+ }
+
+ @Override
+ public int getFieldCount() {
+ return recordDescriptor.getFieldCount();
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return frameTupleAccessor.getFieldSlotsLength();
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return frameTupleAccessor.getFieldEndOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return frameTupleAccessor.getFieldStartOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return frameTupleAccessor.getFieldLength(resetSubTupleAccessor(tupleIndex), fIdx);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ return frameTupleAccessor.getTupleLength(resetSubTupleAccessor(tupleIndex));
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return frameTupleAccessor.getTupleEndOffset(resetSubTupleAccessor(tupleIndex));
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return frameTupleAccessor.getTupleStartOffset(resetSubTupleAccessor(tupleIndex));
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return frameTupleAccessor.getAbsoluteFieldStartOffset(resetSubTupleAccessor(tupleIndex), fIdx);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return innerFrameInfos.size() > 0 ? innerFrameInfos.get(innerFrameInfos.size() - 1).tupleCount : 0;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ this.lastTupleIndex = -1;
+ this.lastFrameId = -1;
+ parseGroupedBuffer(0, buffer.limit());
+ }
+
+ private void parseGroupedBuffer(int start, int stop) {
+ this.innerFrameInfos.clear();
+ int i = start;
+ while (i < stop) {
+ int unitSize = FrameHelper.deserializeNumOfMinFrame(buffer, i) * minFrameSize;
+ if (unitSize == 0) { // run consumed.
+ break;
+ }
+ if (i + unitSize > stop) { // contains future partial run, stop here
+ break;
+ }
+ frameTupleAccessor.reset(buffer, i, unitSize);
+ this.innerFrameInfos
+ .add(new InnerFrameInfo(i, unitSize, getTupleCount() + frameTupleAccessor.getTupleCount()));
+ i += unitSize;
+ }
+ buffer.position(i); // reading stops here.
+ }
+
+ private int resetSubTupleAccessor(int tupleIndex) {
+ assert tupleIndex < getTupleCount();
+ if (innerFrameInfos.size() == 1) {
+ return tupleIndex;
+ }
+ if (tupleIndex == lastTupleIndex) {
+ return lastFrameId > 0 ? lastTupleIndex - innerFrameInfos.get(lastFrameId - 1).tupleCount : lastTupleIndex;
+ }
+ int subFrameId = Collections.binarySearch(innerFrameInfos, tupleIndex);
+ if (subFrameId >= 0) {
+ subFrameId++;
+ } else {
+ subFrameId = -subFrameId - 1;
+ }
+ frameTupleAccessor.reset(buffer, innerFrameInfos.get(subFrameId).start, innerFrameInfos.get(subFrameId).length);
+ lastTupleIndex = tupleIndex;
+ lastFrameId = subFrameId;
+ return lastFrameId > 0 ? lastTupleIndex - innerFrameInfos.get(lastFrameId - 1).tupleCount : lastTupleIndex;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java
new file mode 100644
index 0000000..b273f5b
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/GroupVSizeFrame.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class GroupVSizeFrame extends VSizeFrame {
+
+ public GroupVSizeFrame(IHyracksCommonContext ctx, int frameSize)
+ throws HyracksDataException {
+ super(ctx, frameSize);
+ }
+
+ @Override
+ public void reset() throws HyracksDataException {
+ if (buffer.position() > 0 && buffer.hasRemaining()) {
+ movePartialFutureToStartPosition();
+ } else {
+ buffer.clear();
+ }
+ }
+
+ private void movePartialFutureToStartPosition() {
+ assert buffer.hasArray();
+ if (!FrameHelper.hasBeenCleared(buffer, buffer.position())) {
+ buffer.compact();
+ FrameHelper.clearRemainingFrame(buffer, buffer.position()); // mark it to make reset idempotent
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
new file mode 100644
index 0000000..01744f9
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Basically it a union of the {@link IFrameTupleAccessor} and {@link IFrameTupleAppender}.
+ * Moreover, it has the delete function as well.
+ * This is a special TupleAccessor used for TopK sorting.
+ * In HeapSort, or other Tuple-based operators, we need to append the tuple, access the arbitrary previously
+ * inserted tuple, and delete the previously inserted tuple.
+ */
+public interface IAppendDeletableFrameTupleAccessor extends IFrameTupleAccessor {
+
+ /**
+ * Prepare to write on this buffer
+ *
+ * @param buffer
+ * @throws HyracksDataException
+ */
+ void clear(ByteBuffer buffer) throws HyracksDataException;
+
+ /**
+ * Append tuple content to this buffer. Return the new tid as a handle to the caller.
+ *
+ * @param tupleAccessor
+ * @param tIndex
+ * @return
+ * @throws HyracksDataException
+ */
+ int append(IFrameTupleAccessor tupleAccessor, int tIndex) throws HyracksDataException;
+
+ /**
+ * Remove the certain tuple by tid
+ *
+ * @param tid
+ */
+ void delete(int tid);
+
+ /**
+ * Reorganize the space to remove the unused space and make the free space contiguous.
+ */
+ void reOrganizeBuffer();
+
+ /**
+ * @return how many free space in total in the buffer, including the fragmented space
+ */
+ int getTotalFreeSpace();
+
+ /**
+ * @return how many contiguous free space in the buffer.
+ */
+ int getContiguousFreeSpace();
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java
new file mode 100644
index 0000000..5d8b252
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstractHeap.java
@@ -0,0 +1,156 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import java.util.Arrays;
+
+import edu.uci.ics.hyracks.dataflow.std.util.MathUtil;
+
+public abstract class AbstractHeap implements IHeap<IResetableComparable> {
+ protected static final int NOT_EXIST = -1;
+ protected static final int MAX_INITIAL_CAPACITY = 1024;
+ protected IResetableComparable[] entries;
+ protected IResetableComparable tempEntry;
+ protected IResetableComparableFactory factory;
+ protected int numEntry;
+
+ public AbstractHeap(IResetableComparableFactory factory, int capacity) {
+ capacity = Math.min(MAX_INITIAL_CAPACITY, Math.max(1, capacity));
+ this.entries = new IResetableComparable[capacity];
+ this.numEntry = 0;
+ this.tempEntry = factory.createResetableComparable();
+ this.factory = factory;
+ }
+
+ @Override
+ public void insert(IResetableComparable element) {
+ if (numEntry >= entries.length) {
+ entries = Arrays.copyOf(entries, entries.length * 2);
+ }
+ if (entries[numEntry] == null) {
+ entries[numEntry] = factory.createResetableComparable();
+ }
+ entries[numEntry++].reset(element);
+ bubbleUp(numEntry - 1);
+ }
+
+ protected abstract void bubbleUp(int i);
+
+ protected abstract void trickleDown(int i);
+
+ protected void swap(int cid, int pid) {
+ tempEntry.reset(entries[cid]);
+ entries[cid].reset(entries[pid]);
+ entries[pid].reset(tempEntry);
+ }
+
+ protected int compareTo(int i, int j) {
+ return entries[i].compareTo(entries[j]);
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return numEntry == 0;
+ }
+
+ @Override
+ public void reset() {
+ for (int i = 0; i < numEntry; i++) {
+ entries[i] = null;
+ }
+ numEntry = 0;
+ }
+
+ /**
+ * By getting the entries it can manipulate the entries which may violate the Heap property.
+ * Use with care.
+ *
+ * @return
+ */
+ @Deprecated
+ public IResetableComparable[] getEntries() {
+ return entries;
+ }
+
+ @Override
+ public int getNumEntries() {
+ return numEntry;
+ }
+
+ protected int getLevel(int cid) {
+ return MathUtil.log2Floor(cid + 1);
+ }
+
+ static int getParentId(int cid) {
+ return cid < 1 ? NOT_EXIST : (cid - 1) / 2;
+ }
+
+ static int getLeftChild(int id, int numEntry) {
+ int cid = id * 2 + 1;
+ return cid >= numEntry ? NOT_EXIST : cid;
+ }
+
+ protected int getLeftChild(int id) {
+ return getLeftChild(id, numEntry);
+ }
+
+ static int getRightChild(int id, int numEntry) {
+ int cid = id * 2 + 2;
+ return cid >= numEntry ? NOT_EXIST : cid;
+ }
+
+ protected int getRightChild(int id) {
+ return getRightChild(id, numEntry);
+ }
+
+ protected int getGrandParentId(int id) {
+ int pid = getParentId(id);
+ return pid == NOT_EXIST ? NOT_EXIST : getParentId(pid);
+ }
+
+ protected boolean isDirectChild(int id, int childId) {
+ return id == getParentId(childId);
+ }
+
+ protected int getMinChild(int id) {
+ int min = NOT_EXIST;
+ if (id != NOT_EXIST) {
+ min = getLeftChild(id, numEntry);
+ if (min != NOT_EXIST) {
+ int rightCid = getRightChild(id, numEntry);
+ if (rightCid != NOT_EXIST) {
+ min = compareTo(rightCid, min) < 0 ? rightCid : min;
+ }
+ }
+ }
+ return min;
+ }
+
+ protected int getMaxChild(int id) {
+ int max = NOT_EXIST;
+ if (id != NOT_EXIST) {
+ max = getLeftChild(id, numEntry);
+ if (max != NOT_EXIST) {
+ int rightCid = getRightChild(id, numEntry);
+ if (rightCid != NOT_EXIST) {
+ max = compareTo(rightCid, max) > 0 ? rightCid : max;
+ }
+ }
+ }
+ return max;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java
new file mode 100644
index 0000000..f650f12
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IHeap.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IHeap<T> {
+ /**
+ * Inserts a new element into the selectionTree
+ *
+ * @param element to be inserted
+ */
+ void insert(T element);
+
+ /**
+ * @return True of the selection tree does not have any element, false
+ * otherwise
+ */
+ boolean isEmpty();
+
+ /**
+ * Removes all the elements in the tree
+ */
+ void reset();
+
+ /**
+ * Return the number of the inserted tuples
+ *
+ * @return
+ */
+ int getNumEntries();
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java
new file mode 100644
index 0000000..d932a0e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMaxHeap.java
@@ -0,0 +1,43 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IMaxHeap<T> extends IHeap<T> {
+ /**
+ * Removes and returns the largest element in the tree.
+ * Make sure the heap is not empty (by {@link #isEmpty()}) before calling this method
+ *
+ * @param result
+ */
+ void getMax(T result);
+
+ /**
+ * Returns (and does NOT remove) the largest element in the tree
+ *
+ * @param result is the object that will eventually contain maximum entry
+ * pointer
+ */
+ void peekMax(T result);
+
+ /**
+ * Removes the current max and insert a new element.
+ * Normally it is a faster way to call getMax() && insert() together
+ *
+ * @param newElement
+ */
+ void replaceMax(T newElement);
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java
new file mode 100644
index 0000000..784c492
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinHeap.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IMinHeap<T> extends IHeap<T> {
+ /**
+ * Removes and returns the smallest element in the tree.
+ * Make sure the heap is not empty (by {@link #isEmpty()}) before calling this method
+ *
+ * @param result
+ */
+ void getMin(T result);
+
+ /**
+ * Returns (and does NOT remove) the smallest element in the tree
+ *
+ * @param result is the object that will eventually contain minimum entry
+ * pointer
+ */
+ void peekMin(T result);
+
+ /**
+ * Removes the current min and insert a new element.
+ * Normally it is a faster way to call getMin() && insert() together
+ *
+ * @param newElement
+ */
+ void replaceMin(T newElement);
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
similarity index 70%
rename from hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java
rename to hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
index 43538d7..8225d88 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameConstants.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IMinMaxHeap.java
@@ -3,21 +3,16 @@
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* you may obtain a copy of the License from
- *
+ *
* http://www.apache.org/licenses/LICENSE-2.0
- *
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.dataflow.common.comm.io;
+package edu.uci.ics.hyracks.dataflow.std.structures;
-public interface FrameConstants {
- public static final int SIZE_LEN = 4;
-
- public static final boolean DEBUG_FRAME_IO = false;
-
- public static final int FRAME_FIELD_MAGIC = 0x12345678;
+public interface IMinMaxHeap<T> extends IMinHeap<T>, IMaxHeap<T> {
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java
new file mode 100644
index 0000000..7608128
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetable.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetable<T> {
+ void reset(T other);
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java
new file mode 100644
index 0000000..88e2e64
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparable.java
@@ -0,0 +1,19 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetableComparable<T> extends IResetable<T>, Comparable<T>{
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java
new file mode 100644
index 0000000..a5a635e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/IResetableComparableFactory.java
@@ -0,0 +1,20 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public interface IResetableComparableFactory<T> {
+ IResetableComparable<T> createResetableComparable();
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java
new file mode 100644
index 0000000..1cfb4e0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeap.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MaxHeap extends AbstractHeap implements IMaxHeap<IResetableComparable> {
+
+ public MaxHeap(IResetableComparableFactory factory, int capacity) {
+ super(factory, capacity);
+ }
+
+ @Override
+ protected void bubbleUp(int i) {
+ int pid = getParentId(i);
+ if (pid != NOT_EXIST && compareTo(pid, i) < 0) {
+ swap(pid, i);
+ bubbleUp(pid);
+ }
+ }
+
+ @Override
+ protected void trickleDown(int i) {
+ int maxChild = getMaxChild(i);
+ if (maxChild != NOT_EXIST && compareTo(i, maxChild) < 0) {
+ swap(maxChild, i);
+ trickleDown(maxChild);
+ }
+ }
+
+ @Override
+ public void getMax(IResetableComparable result) {
+ result.reset(entries[0]);
+ numEntry--;
+ if (numEntry > 0) {
+ entries[0].reset(entries[numEntry]);
+ trickleDown(0);
+ }
+ }
+
+ @Override
+ public void peekMax(IResetableComparable result) {
+ result.reset(entries[0]);
+ }
+
+ @Override
+ public void replaceMax(IResetableComparable newElement) {
+ entries[0].reset(newElement);
+ trickleDown(0);
+ }
+}
+
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java
new file mode 100644
index 0000000..bfeda33
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeap.java
@@ -0,0 +1,62 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MinHeap extends AbstractHeap implements IMinHeap<IResetableComparable> {
+
+ public MinHeap(IResetableComparableFactory factory, int capacity) {
+ super(factory, capacity);
+ }
+
+ @Override
+ protected void bubbleUp(int i) {
+ int pid = getParentId(i);
+ if (pid != NOT_EXIST && compareTo(pid, i) > 0) {
+ swap(pid, i);
+ bubbleUp(pid);
+ }
+ }
+
+ @Override
+ protected void trickleDown(int i) {
+ int minChild = getMinChild(i);
+ if (minChild != NOT_EXIST && compareTo(i, minChild) > 0) {
+ swap(minChild, i);
+ trickleDown(minChild);
+ }
+ }
+
+ @Override
+ public void getMin(IResetableComparable result) {
+ result.reset(entries[0]);
+ numEntry--;
+ if (numEntry > 0) {
+ entries[0].reset(entries[numEntry]);
+ trickleDown(0);
+ }
+ }
+
+ @Override
+ public void peekMin(IResetableComparable result) {
+ result.reset(entries[0]);
+ }
+
+ @Override
+ public void replaceMin(IResetableComparable newElement) {
+ entries[0].reset(newElement);
+ trickleDown(0);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java
new file mode 100644
index 0000000..e9782e2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeap.java
@@ -0,0 +1,217 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+public class MinMaxHeap extends AbstractHeap implements IMinMaxHeap<IResetableComparable> {
+
+ public MinMaxHeap(IResetableComparableFactory factory, int capacity) {
+ super(factory, capacity);
+ }
+
+ @Override
+ protected void bubbleUp(int cid) {
+ int pid = getParentId(cid);
+ if (isAtMinLevel(cid)) {
+ if (pid != NOT_EXIST && entries[pid].compareTo(entries[cid]) < 0) {
+ swap(cid, pid);
+ bubbleUpMax(pid);
+ } else {
+ bubbleUpMin(cid);
+ }
+ } else { // isAtMaxLevel
+ if (pid != NOT_EXIST && entries[pid].compareTo(entries[cid]) > 0) {
+ swap(cid, pid);
+ bubbleUpMin(pid);
+ } else {
+ bubbleUpMax(cid);
+ }
+ }
+ }
+
+ private void bubbleUpMin(int id) {
+ int gp = getGrandParentId(id);
+ if (gp != NOT_EXIST && entries[gp].compareTo(entries[id]) > 0) {
+ swap(gp, id);
+ bubbleUpMin(gp);
+ }
+ }
+
+ private void bubbleUpMax(int id) {
+ int gp = getGrandParentId(id);
+ if (gp != NOT_EXIST && entries[gp].compareTo(entries[id]) < 0) {
+ swap(gp, id);
+ bubbleUpMax(gp);
+ }
+ }
+
+ private boolean isAtMinLevel(int cid) {
+ return getLevel(cid) % 2 == 0;
+ }
+
+ /**
+ * Make sure to check the {@link #isEmpty()} before calling this function.
+ *
+ * @param result
+ */
+ @Override
+ public void getMin(IResetableComparable result) {
+ result.reset(entries[0]);
+ numEntry--;
+ if (numEntry > 0) {
+ entries[0].reset(entries[numEntry]);
+ trickleDown(0);
+ }
+ }
+
+ @Override
+ public void getMax(IResetableComparable result) {
+ int max = getMaxChild(0);
+ if (max == NOT_EXIST) {
+ getMin(result);
+ return;
+ }
+ result.reset(entries[max]);
+ numEntry--;
+ if (numEntry > max) {
+ entries[max].reset(entries[numEntry]);
+ trickleDown(max);
+ }
+ }
+
+ @Override
+ protected void trickleDown(int id) {
+ if (isAtMinLevel(id)) {
+ trickleDownMin(id);
+ } else {
+ trickleDownMax(id);
+ }
+ }
+
+ private void trickleDownMax(int id) {
+ int maxId = getMaxOfDescendents(id);
+ if (maxId == NOT_EXIST) {
+ return;
+ }
+ if (isDirectChild(id, maxId)) {
+ if (entries[id].compareTo(entries[maxId]) < 0) {
+ swap(id, maxId);
+ }
+ } else {
+ if (entries[id].compareTo(entries[maxId]) < 0) {
+ swap(id, maxId);
+ int pid = getParentId(maxId);
+ if (entries[maxId].compareTo(entries[pid]) < 0) {
+ swap(pid, maxId);
+ }
+ trickleDownMax(maxId);
+ }
+ }
+ }
+
+ private void trickleDownMin(int id) {
+ int minId = getMinOfDescendents(id);
+ if (minId == NOT_EXIST) {
+ return;
+ }
+ if (isDirectChild(id, minId)) {
+ if (entries[id].compareTo(entries[minId]) > 0) {
+ swap(id, minId);
+ }
+ } else { // is grand child
+ if (entries[id].compareTo(entries[minId]) > 0) {
+ swap(id, minId);
+ int pid = getParentId(minId);
+ if (entries[minId].compareTo(entries[pid]) > 0) {
+ swap(pid, minId);
+ }
+ trickleDownMin(minId);
+ }
+ }
+ }
+
+ private int getMaxOfDescendents(int id) {
+ int max = getMaxChild(id);
+ if (max != NOT_EXIST) {
+ int leftMax = getMaxChild(getLeftChild(id));
+ if (leftMax != NOT_EXIST) {
+ max = entries[leftMax].compareTo(entries[max]) > 0 ? leftMax : max;
+ int rightMax = getMaxChild(getRightChild(id));
+ if (rightMax != NOT_EXIST) {
+ max = entries[rightMax].compareTo(entries[max]) > 0 ? rightMax : max;
+ }
+ }
+ }
+ return max;
+ }
+
+ private int getMinOfDescendents(int id) {
+ int min = getMinChild(id);
+ if (min != NOT_EXIST) {
+ int leftMin = getMinChild(getLeftChild(id));
+ if (leftMin != NOT_EXIST) {
+ min = entries[leftMin].compareTo(entries[min]) < 0 ? leftMin : min;
+ int rightMin = getMinChild(getRightChild(id));
+ if (rightMin != NOT_EXIST) {
+ min = entries[rightMin].compareTo(entries[min]) < 0 ? rightMin : min;
+ }
+ }
+ }
+ return min;
+ }
+
+ public boolean isEmpty() {
+ return numEntry == 0;
+ }
+
+ /**
+ * Make sure to call the {@link #isEmpty()} before calling this function
+ *
+ * @param result is the object that will eventually contain minimum entry
+ */
+ @Override
+ public void peekMin(IResetableComparable result) {
+ result.reset(entries[0]);
+ }
+
+ @Override
+ public void peekMax(IResetableComparable result) {
+ int maxChild = getMaxChild(0);
+ if (maxChild == NOT_EXIST) {
+ peekMin(result);
+ return;
+ }
+ result.reset(entries[maxChild]);
+ }
+
+ @Override
+ public void replaceMin(IResetableComparable newElement) {
+ entries[0].reset(newElement);
+ trickleDown(0);
+ }
+
+ @Override
+ public void replaceMax(IResetableComparable newElement) {
+ int maxChild = getMaxChild(0);
+ if (maxChild == NOT_EXIST) {
+ replaceMin(newElement);
+ return;
+ }
+ entries[maxChild].reset(newElement);
+ bubbleUp(maxChild);
+ trickleDown(maxChild);
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
index 1ea8393..3837004 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/SerializableHashTable.java
@@ -41,7 +41,7 @@
public SerializableHashTable(int tableSize, final IHyracksTaskContext ctx) throws HyracksDataException {
this.ctx = ctx;
- int frameSize = ctx.getFrameSize();
+ int frameSize = ctx.getInitialFrameSize();
int residual = tableSize * INT_SIZE * 2 % frameSize == 0 ? 0 : 1;
int headerSize = tableSize * INT_SIZE * 2 / frameSize + residual;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
index 8ef478c..a42e06d 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/structures/TuplePointer.java
@@ -14,7 +14,48 @@
*/
package edu.uci.ics.hyracks.dataflow.std.structures;
-public class TuplePointer {
+public class TuplePointer implements IResetable<TuplePointer> {
+ public static final int INVALID_ID = -1;
public int frameIndex;
public int tupleIndex;
+
+ public TuplePointer() {
+ this(INVALID_ID, INVALID_ID);
+ }
+
+ public TuplePointer(int frameId, int tupleId) {
+ reset(frameId, tupleId);
+ }
+
+ public void reset(TuplePointer other) {
+ reset(other.frameIndex, other.tupleIndex);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o)
+ return true;
+ if (o == null || getClass() != o.getClass())
+ return false;
+
+ TuplePointer that = (TuplePointer) o;
+
+ if (frameIndex != that.frameIndex)
+ return false;
+ return tupleIndex == that.tupleIndex;
+
+ }
+
+ @Override
+ public int hashCode() {
+ int result = frameIndex;
+ result = 31 * result + tupleIndex;
+ return result;
+ }
+
+ public void reset(int frameId, int tupleId) {
+ this.frameIndex = frameId;
+ this.tupleIndex = tupleId;
+ }
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
index aff4273..09aacd6 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/DeserializedOperatorNodePushable.java
@@ -36,7 +36,7 @@
RecordDescriptor inRecordDesc) {
this.ctx = ctx;
this.delegate = delegate;
- deserializer = inRecordDesc == null ? null : new FrameDeserializer(ctx.getFrameSize(), inRecordDesc);
+ deserializer = inRecordDesc == null ? null : new FrameDeserializer(inRecordDesc);
}
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java
new file mode 100644
index 0000000..c89ee25
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/MathUtil.java
@@ -0,0 +1,50 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+public class MathUtil {
+ /**
+ * Fast way to calculate the log2(x). Note: x should be >= 1.
+ *
+ * @param n
+ * @return
+ */
+ public static int log2Floor(int n) {
+ assert n >= 1;
+ int log = 0;
+ if (n > 0xffff) {
+ n >>>= 16;
+ log = 16;
+ }
+
+ if (n > 0xff) {
+ n >>>= 8;
+ log |= 8;
+ }
+
+ if (n > 0xf) {
+ n >>>= 4;
+ log |= 4;
+ }
+
+ if (n > 0b11) {
+ n >>>= 2;
+ log |= 2;
+ }
+
+ return log + (n >>> 1);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
index c06b50c..d6adcf4 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferenceEntry.java
@@ -14,12 +14,13 @@
*/
package edu.uci.ics.hyracks.dataflow.std.util;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
public class ReferenceEntry {
private final int runid;
- private FrameTupleAccessor acccessor;
+ private IFrameTupleAccessor acccessor;
private int tupleIndex;
private int[] tPointers;
@@ -38,11 +39,11 @@
return runid;
}
- public FrameTupleAccessor getAccessor() {
+ public IFrameTupleAccessor getAccessor() {
return acccessor;
}
- public void setAccessor(FrameTupleAccessor fta) {
+ public void setAccessor(IFrameTupleAccessor fta) {
this.acccessor = fta;
}
@@ -62,15 +63,14 @@
initTPointer(acccessor, tupleIndex, keyFields, nmkComputer);
}
- private void initTPointer(FrameTupleAccessor fta, int tupleIndex, int[] keyFields,
+ private void initTPointer(IFrameTupleAccessor fta, int tupleIndex, int[] keyFields,
INormalizedKeyComputer nmkComputer) {
this.tupleIndex = tupleIndex;
byte[] b1 = fta.getBuffer().array();
for (int f = 0; f < keyFields.length; ++f) {
int fIdx = keyFields[f];
- tPointers[2 * f + 1] = fta.getTupleStartOffset(tupleIndex) + fta.getFieldSlotsLength()
- + fta.getFieldStartOffset(tupleIndex, fIdx);
- tPointers[2 * f + 2] = fta.getFieldEndOffset(tupleIndex, fIdx) - fta.getFieldStartOffset(tupleIndex, fIdx);
+ tPointers[2 * f + 1] = fta.getAbsoluteFieldStartOffset(tupleIndex, fIdx);
+ tPointers[2 * f + 2] = fta.getFieldLength(tupleIndex, fIdx);
if (f == 0) {
if (nmkComputer != null) {
tPointers[0] = nmkComputer.normalize(b1, tPointers[1], tPointers[2]);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
index 225f583..e994f04 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/util/ReferencedPriorityQueue.java
@@ -18,13 +18,10 @@
import java.util.BitSet;
import java.util.Comparator;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.dataflow.value.INormalizedKeyComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
public class ReferencedPriorityQueue {
- private final int frameSize;
- private final RecordDescriptor recordDescriptor;
private final ReferenceEntry entries[];
private final int size;
private final BitSet runAvail;
@@ -34,10 +31,8 @@
private final INormalizedKeyComputer nmkComputer;
private final int[] keyFields;
- public ReferencedPriorityQueue(int frameSize, RecordDescriptor recordDescriptor, int initSize,
- Comparator<ReferenceEntry> comparator, int[] keyFields, INormalizedKeyComputer nmkComputer) {
- this.frameSize = frameSize;
- this.recordDescriptor = recordDescriptor;
+ public ReferencedPriorityQueue(int initSize, Comparator<ReferenceEntry> comparator, int[] keyFields,
+ INormalizedKeyComputer nmkComputer) {
if (initSize < 1)
throw new IllegalArgumentException();
this.comparator = comparator;
@@ -55,7 +50,7 @@
/**
* Retrieve the top entry without removing it
- *
+ *
* @return the top entry
*/
public ReferenceEntry peek() {
@@ -65,17 +60,14 @@
/**
* compare the new entry with entries within the queue, to find a spot for
* this new entry
- *
- * @param entry
+ *
+ * @param fta
* @return runid of this entry
* @throws IOException
*/
- public int popAndReplace(FrameTupleAccessor fta, int tIndex) {
+ public int popAndReplace(IFrameTupleAccessor fta, int tIndex) {
ReferenceEntry entry = entries[0];
- if (entry.getAccessor() == null) {
- entry.setAccessor(new FrameTupleAccessor(frameSize, recordDescriptor));
- }
- entry.getAccessor().reset(fta.getBuffer());
+ entry.setAccessor(fta);
entry.setTupleIndex(tIndex, keyFields, nmkComputer);
add(entry);
@@ -84,9 +76,8 @@
/**
* Push entry into priority queue
- *
- * @param e
- * the new Entry
+ *
+ * @param e the new Entry
*/
private void add(ReferenceEntry e) {
ReferenceEntry min = entries[0];
@@ -127,7 +118,7 @@
/**
* Pop is called only when a run is exhausted
- *
+ *
* @return
*/
public ReferenceEntry pop() {
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java
new file mode 100644
index 0000000..ee2b008
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/Utility.java
@@ -0,0 +1,23 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort;
+
+public class Utility {
+
+ public static String repeatString(char ch, int times) {
+ return new String(new char[times]).replace('\0', ch);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java
new file mode 100644
index 0000000..5a59ab6
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/Common.java
@@ -0,0 +1,26 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
+
+public class Common {
+ static int MIN_FRAME_SIZE = 256;
+ static int NUM_MIN_FRAME = 15;
+ static int BUDGET = NUM_MIN_FRAME * MIN_FRAME_SIZE;
+
+ static FrameManager commonFrameManager = new FrameManager(MIN_FRAME_SIZE);
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
new file mode 100644
index 0000000..a2d0f1e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBestFitUsingTreeMapTest.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotBestFitUsingTreeMapTest {
+
+ static int size = 10;
+
+ FrameFreeSlotSmallestFit policy;
+
+ @Before
+ public void intial() {
+ policy = new FrameFreeSlotSmallestFit();
+ }
+
+ @Test
+ public void testAll() {
+
+ for (int i = 0; i < size; i++) {
+ policy.pushNewFrame(i, i);
+ assertEquals(i, policy.popBestFit(i));
+ }
+ assertEquals(-1, policy.popBestFit(0));
+
+ for (int i = 0; i < size; i++) {
+ policy.pushNewFrame(i, i);
+ }
+ for (int i = 0; i < size; i++) {
+ assertEquals(i, policy.popBestFit(i));
+ }
+
+ }
+
+ @Test
+ public void testReset(){
+ testAll();
+ policy.reset();
+ testAll();
+ }
+
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
new file mode 100644
index 0000000..f3b923e
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotBiggestFirstTest.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static junit.framework.Assert.assertEquals;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotBiggestFirstTest {
+
+ static int size = 10;
+
+ FrameFreeSlotBiggestFirst policy;
+
+ @Before
+ public void intial() {
+ policy = new FrameFreeSlotBiggestFirst(size);
+ }
+
+ @Test
+ public void testAll() {
+
+ for (int i = 0; i < size; i++) {
+ policy.pushNewFrame(i, i);
+ assertEquals(i, policy.popBestFit(i));
+ }
+ assertEquals(-1, policy.popBestFit(0));
+
+ for (int i = 0; i < size; i++) {
+ policy.pushNewFrame(i, i);
+ }
+ for (int i = 0; i < size; i++) {
+ assertEquals(size - i - 1, policy.popBestFit(0));
+ }
+
+ for (int i = 0; i < size; i++) {
+ policy.pushNewFrame(i, i);
+ }
+ for (int i = 0; i < size / 2; i++) {
+ assertEquals(size - i - 1, policy.popBestFit(size / 2));
+ }
+ assertEquals(-1, policy.popBestFit(size / 2));
+ for (int i = 0; i < size / 2; i++) {
+ assertEquals(size / 2 - i - 1, policy.popBestFit(0));
+ }
+
+ }
+
+ @Test
+ public void testReset() {
+ testAll();
+ policy.reset();
+ testAll();
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
new file mode 100644
index 0000000..94a8493
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/FrameFreeSlotLastFitTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class FrameFreeSlotLastFitTest {
+
+ FrameFreeSlotLastFit zeroPolicy;
+ FrameFreeSlotLastFit unifiedPolicy;
+ FrameFreeSlotLastFit ascPolicy;
+ FrameFreeSlotLastFit dscPolicy;
+
+ static final int size = 10;
+ static final int medium = 5;
+
+ @Before
+ public void setUp() throws Exception {
+ zeroPolicy = new FrameFreeSlotLastFit(0);
+ unifiedPolicy = new FrameFreeSlotLastFit(size);
+ ascPolicy = new FrameFreeSlotLastFit(size);
+ dscPolicy = new FrameFreeSlotLastFit(size);
+ }
+
+ @Test
+ public void testPushAndPop() throws Exception {
+ for (int i = 0; i < size; i++) {
+ unifiedPolicy.pushNewFrame(i, medium);
+ }
+ for (int i = 0; i < size; i++) {
+ assertTrue(unifiedPolicy.popBestFit(medium) == size - i - 1);
+ }
+ assertTrue(unifiedPolicy.popBestFit(0) == -1);
+
+ for (int i = 0; i < size / 2; i++) {
+ ascPolicy.pushNewFrame(i, i);
+ assertEquals(ascPolicy.popBestFit(medium), -1);
+ dscPolicy.pushNewFrame(i, size - i - 1);
+ assertEquals(dscPolicy.popBestFit(medium), i);
+ }
+
+ for (int i = size / 2; i < size; i++) {
+ ascPolicy.pushNewFrame(i, i);
+ assertEquals(ascPolicy.popBestFit(medium), i);
+ dscPolicy.pushNewFrame(i, size - i - 1);
+ assertEquals(dscPolicy.popBestFit(medium), -1);
+ }
+
+ ascPolicy.reset();
+ for (int i = 0; i < size; i++) {
+ ascPolicy.pushNewFrame(size - i, size - i);
+ }
+
+ for (int i = 0; i < size; i++) {
+ assertEquals(size - i, ascPolicy.popBestFit(size - i));
+ }
+ }
+
+ @Test
+ public void testReset() throws Exception {
+ testPushAndPop();
+
+ zeroPolicy.reset();
+ unifiedPolicy.reset();
+ ascPolicy.reset();
+ dscPolicy.reset();
+ testPushAndPop();
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
new file mode 100644
index 0000000..e9ac2ec
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramePoolTest.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.commonFrameManager;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class VariableFramePoolTest {
+
+ VariableFramePool pool;
+ @Before
+ public void setUp() throws Exception {
+
+ pool = new VariableFramePool(commonFrameManager, BUDGET);
+ }
+
+ @Test
+ public void testGetMinFrameSize() throws Exception {
+ assertEquals(MIN_FRAME_SIZE, commonFrameManager.getInitialFrameSize());
+ assertEquals(MIN_FRAME_SIZE, pool.getMinFrameSize());
+ }
+
+ @Test
+ public void testGetMemoryBudgetBytes() throws Exception {
+ assertEquals(BUDGET, pool.getMemoryBudgetBytes());
+ }
+
+ @Test
+ public void testAllocateUniformFrameShouldSuccess() throws Exception {
+ testAllocateAllSpacesWithMinFrames();
+ testAllocateShouldFailAfterAllSpaceGetUsed();
+ pool.reset();
+ testAllocateAllSpacesWithMinFrames();
+ pool.close();
+ }
+
+ @Test
+ public void testResetShouldReuseExistingFrames() throws HyracksDataException {
+ Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+ pool.reset();
+ Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+ assertEquals(set1, set2);
+ pool.close();
+ }
+
+ @Test
+ public void testCloseShouldNotReuseExistingFrames() throws HyracksDataException {
+ Set<?> set1 = testAllocateAllSpacesWithMinFrames();
+ pool.close();
+ Set<?> set2 = testAllocateAllSpacesWithMinFrames();
+ assertFalse(set1.equals(set2));
+ pool.close();
+ }
+
+ @Test
+ public void testShouldReturnLargerFramesIfFitOneIsUsed() throws HyracksDataException {
+ Set<?> set = testAllocateVariableFrames();
+ pool.reset();
+ testShouldFindTheMatchFrames(set);
+ pool.reset();
+
+ // allocate seq: 1, 1, 2, 3, 4
+ ByteBuffer placeBuffer = pool.allocateFrame(MIN_FRAME_SIZE);
+ assertTrue(set.contains(new ByteBufferPtr(placeBuffer)));
+ for (int i = 1; i <= 4 ; i++) {
+ ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ }
+ assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+ pool.close();
+ }
+
+ @Test
+ public void testShouldMergeIfNoLargerFrames() throws HyracksDataException {
+ Set<?> set = testAllocateAllSpacesWithMinFrames();
+ pool.reset();
+ int chunks = 5;
+ for (int i = 0; i < NUM_MIN_FRAME; i+= chunks) {
+ ByteBuffer buffer = pool.allocateFrame(chunks * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+ }
+ }
+
+ @Test
+ public void testUseMiddleSizeFrameAndNeedToMergeSmallAndBigger() throws HyracksDataException {
+ Set<?> set = testAllocateVariableFrames();
+ pool.reset();
+ // allocate seq: 3, 6, 1;
+ ByteBuffer buffer = pool.allocateFrame(3 * MIN_FRAME_SIZE);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ buffer = pool.allocateFrame(6 * MIN_FRAME_SIZE);
+ assertFalse(set.contains(new ByteBufferPtr(buffer)));
+ buffer = pool.allocateFrame(1 * MIN_FRAME_SIZE);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ assertEquals(5 * MIN_FRAME_SIZE, buffer.capacity());
+ pool.reset();
+ }
+
+ private void testAllocateShouldFailAfterAllSpaceGetUsed() throws HyracksDataException {
+ for (int i = 0; i < NUM_MIN_FRAME; i++) {
+ assertNull(pool.allocateFrame(MIN_FRAME_SIZE));
+ }
+ }
+
+ private HashSet<ByteBufferPtr> testAllocateAllSpacesWithMinFrames() throws HyracksDataException {
+ HashSet<ByteBufferPtr> set = new HashSet<>();
+ for (int i = 0; i < NUM_MIN_FRAME; i++) {
+ ByteBuffer buffer = pool.allocateFrame(MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(!set.contains(new ByteBufferPtr(buffer)));
+ set.add(new ByteBufferPtr(buffer));
+ }
+ return set;
+ }
+
+ /**
+ * Pool will become 1,2,3,4,5
+ *
+ * @throws HyracksDataException
+ */
+ private Set<ByteBufferPtr> testAllocateVariableFrames() throws HyracksDataException {
+ int budget = BUDGET;
+ int allocate = 0;
+ int i = 1;
+ Set<ByteBufferPtr> set = new HashSet<>();
+ while (budget - allocate >= i * MIN_FRAME_SIZE) {
+ ByteBuffer buffer = pool.allocateFrame(i * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ set.add(new ByteBufferPtr(buffer));
+ allocate += i++ * MIN_FRAME_SIZE;
+ }
+ return set;
+ }
+
+ private void testShouldFindTheMatchFrames(Set<?> set) throws HyracksDataException {
+ pool.reset();
+ List<Integer> list = Arrays.asList(1, 2, 3, 4, 5);
+
+ for (int i = 0; i < list.size(); i++) {
+ ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+ }
+ pool.reset();
+ for (int i = list.size() - 1; i >= 0; i--) {
+ ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+ }
+
+ Collections.shuffle(list);
+ pool.reset();
+ for (int i = 0; i < list.size(); i++) {
+ ByteBuffer buffer = pool.allocateFrame(list.get(i) * MIN_FRAME_SIZE);
+ assertNotNull(buffer);
+ assertTrue(set.contains(new ByteBufferPtr(buffer)));
+ assertEquals(list.get(i) * MIN_FRAME_SIZE, buffer.capacity());
+ }
+
+ }
+
+ public static class ByteBufferPtr {
+ ByteBuffer bytebuffer;
+
+ public ByteBufferPtr(ByteBuffer buffer) {
+ bytebuffer = buffer;
+ }
+
+ @Override
+ public int hashCode() {
+ return bytebuffer.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ return this.bytebuffer == ((ByteBufferPtr) obj).bytebuffer;
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
new file mode 100644
index 0000000..f56a62d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableFramesMemoryManagerTest.java
@@ -0,0 +1,170 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.BUDGET;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.MIN_FRAME_SIZE;
+import static edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.Common.NUM_MIN_FRAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAppender;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class VariableFramesMemoryManagerTest {
+ VariableFrameMemoryManager framesMemoryManager;
+ FrameTupleAccessor fta;
+ Random random;
+ List<IFrame> frameList;
+
+ @Before
+ public void setUp() throws Exception {
+ VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, BUDGET);
+ FrameFreeSlotLastFit policy = new FrameFreeSlotLastFit(NUM_MIN_FRAME);
+ framesMemoryManager = new VariableFrameMemoryManager(framePool, policy);
+ RecordDescriptor recordDescriptor = new RecordDescriptor(new ISerializerDeserializer[] { null });
+ fta = new FrameTupleAccessor(recordDescriptor);
+ random = new Random(System.currentTimeMillis());
+ frameList = new ArrayList<>();
+ }
+
+ @Test
+ public void testNormalIncomingFrames() throws HyracksDataException {
+ HashMap<Integer, Integer> tupleSet = prepareTuples();
+ for (IFrame frame : frameList) {
+ assertTrue(framesMemoryManager.insertFrame(frame.getBuffer()) >=0);
+ }
+ assertEquals(NUM_MIN_FRAME, framesMemoryManager.getNumFrames());
+ assertEveryTupleInFTAIsInFrameMemoryManager(tupleSet, framesMemoryManager);
+ }
+
+ @Test
+ public void testRandomTuplesAreAllStoredInBuffer() throws HyracksDataException {
+ Map<Integer, Integer> tupleSet = prepareRandomTuples();
+ for (IFrame frame : frameList) {
+ if (framesMemoryManager.insertFrame(frame.getBuffer()) < 0) {
+ fta.reset(frame.getBuffer());
+ for (int i = 0; i < fta.getTupleCount(); ++i) {
+ int id = parseTuple(fta.getBuffer(),
+ fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+ tupleSet.remove(id);
+ // System.out.println(
+ // "can't appended id:" + id + ",frameSize:" + frame.getInitialFrameSize());
+ }
+ }
+ }
+ assertEveryTupleInFTAIsInFrameMemoryManager(tupleSet, framesMemoryManager);
+ framesMemoryManager.reset();
+ }
+
+ @Test
+ public void testResetShouldWork() throws HyracksDataException {
+ testNormalIncomingFrames();
+ framesMemoryManager.reset();
+ testRandomTuplesAreAllStoredInBuffer();
+ framesMemoryManager.reset();
+ testRandomTuplesAreAllStoredInBuffer();
+ }
+
+ @Test
+ public void testCloseShouldAlsoWork() throws HyracksDataException {
+ testRandomTuplesAreAllStoredInBuffer();
+ framesMemoryManager.close();
+ testRandomTuplesAreAllStoredInBuffer();
+ framesMemoryManager.close();
+ testRandomTuplesAreAllStoredInBuffer();
+ }
+
+ private HashMap<Integer, Integer> prepareRandomTuples() throws HyracksDataException {
+ frameList.clear();
+ HashMap<Integer, Integer> set = new HashMap<>(NUM_MIN_FRAME);
+ int[] fieldSlot = { 0 };
+ int id = 0;
+ int size = 0;
+ while (size < BUDGET) {
+ int tupleLength = random.nextInt(BUDGET / 3) + 4;
+ IFrame frame = new FixedSizeFrame(Common.commonFrameManager
+ .allocateFrame(FrameHelper.calcAlignedFrameSizeToStore(1, tupleLength, MIN_FRAME_SIZE)));
+ IFrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(frame, true);
+ // System.out.println("id:" + id + ",frameSize:" + frame.getInitialFrameSize() / MIN_FRAME_SIZE);
+ ByteBuffer buffer = ByteBuffer.allocate(tupleLength);
+ buffer.putInt(0, id);
+ assertTrue(appender.append(fieldSlot, buffer.array(), 0, buffer.capacity()));
+ set.put(id++, tupleLength);
+ size += frame.getFrameSize();
+ frameList.add(frame);
+ }
+ return set;
+ }
+
+ private HashMap<Integer, Integer> prepareTuples() throws HyracksDataException {
+ frameList.clear();
+ HashMap<Integer, Integer> set = new HashMap<>(NUM_MIN_FRAME);
+ for (int i = 0; i < NUM_MIN_FRAME; ++i) {
+ IFrame frame = new FixedSizeFrame(Common.commonFrameManager.allocateFrame(MIN_FRAME_SIZE));
+ IFrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(frame, true);
+
+ int[] fieldSlot = { 0 };
+ ByteBuffer buffer = ByteBuffer.allocate(MIN_FRAME_SIZE / 2);
+ buffer.putInt(0, i);
+ appender.append(fieldSlot, buffer.array(), 0, buffer.capacity());
+ set.put(i, buffer.capacity());
+ frameList.add(frame);
+ }
+ return set;
+ }
+
+ private void assertEveryTupleInFTAIsInFrameMemoryManager(Map<Integer, Integer> tupleSet,
+ VariableFrameMemoryManager framesMemoryManager) {
+ for (int i = 0; i < framesMemoryManager.getNumFrames(); ++i) {
+ fta.reset(framesMemoryManager.getFrame(i), framesMemoryManager.getFrameStartOffset(i),
+ framesMemoryManager.getFrameSize(i));
+ for (int t = 0; t < fta.getTupleCount(); t++) {
+ int id = parseTuple(fta.getBuffer(), fta.getTupleStartOffset(t) + fta.getFieldSlotsLength() + fta
+ .getFieldStartOffset(t, 0));
+ // System.out.println("frameid:" + i + ",tuple:" + t + ",has id:" + id + ",length:" +
+ // (fta.getTupleEndOffset(t) - fta.getTupleStartOffset(t) - fta.getFieldSlotsLength()));
+ assertTrue(tupleSet.remove(id) == fta.getTupleEndOffset(t) - fta.getTupleStartOffset(t) - fta
+ .getFieldSlotsLength());
+ }
+ }
+ assertTrue(tupleSet.isEmpty());
+ }
+
+ private int parseTuple(ByteBuffer buffer, int fieldStartOffset) {
+ return buffer.getInt(fieldStartOffset);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
new file mode 100644
index 0000000..0c54607
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
@@ -0,0 +1,230 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.buffermanager;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.Utility;
+import edu.uci.ics.hyracks.dataflow.std.structures.TuplePointer;
+
+public class VariableTupleMemoryManagerTest {
+ ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+ RecordDescriptor recordDescriptor = new RecordDescriptor(fieldsSerDer);
+ ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+ VariableTupleMemoryManager tupleMemoryManager;
+ FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
+ Random random = new Random(System.currentTimeMillis());
+
+ @Before
+ public void setup() {
+ VariableFramePool framePool = new VariableFramePool(Common.commonFrameManager, Common.BUDGET);
+ tupleMemoryManager = new VariableTupleMemoryManager(framePool, recordDescriptor);
+ }
+
+ @Test
+ public void testInsertTupleToMemoryManager() throws HyracksDataException {
+ int iTuplePerFrame = 3;
+ Map<Integer, Integer> mapPrepare = prepareFixedSizeTuples(iTuplePerFrame);
+ Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+ assertEachTupleInFTAIsInBuffer(mapPrepare, mapInserted);
+ }
+
+ @Test
+ public void testReset() throws HyracksDataException {
+ testInsertVariableSizeTupleToMemoryManager();
+ tupleMemoryManager.reset();
+ testInsertTupleToMemoryManager();
+ tupleMemoryManager.reset();
+ testInsertVariableSizeTupleToMemoryManager();
+ }
+
+ @Test
+ public void testDeleteTupleInMemoryManager() throws HyracksDataException {
+ int iTuplePerFrame = 3;
+ Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
+ Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+ deleteRandomSelectedTuples(map, mapInserted, 1);
+ assertEachTupleInFTAIsInBuffer(map, mapInserted);
+ }
+
+ @Test
+ public void testReOrganizeSpace() throws HyracksDataException {
+ int iTuplePerFrame = 3;
+ Map<Integer, Integer> map = prepareFixedSizeTuples(iTuplePerFrame);
+ Map<Integer, Integer> copyMap = new HashMap<>(map);
+ Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferShouldAllSuccess();
+ ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
+ inFTA.reset(buffer);
+ Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferShouldAllSuccess();
+ Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
+ mergedMap.putAll(mapInserted2);
+ assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
+ }
+
+ @Test
+ public void testReOrganizeVariableSizeTuple() throws HyracksDataException {
+ Map<Integer, Integer> map = prepareVariableSizeTuples();
+ Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+ Map<Integer, Integer> copyMap = new HashMap<>(map);
+
+ ByteBuffer buffer = deleteRandomSelectedTuples(map, mapInserted, map.size() / 2);
+ inFTA.reset(buffer);
+
+ Map<TuplePointer, Integer> mapInserted2 = insertInFTAToBufferCouldFailForLargerTuples(copyMap);
+ Map<TuplePointer, Integer> mergedMap = new HashMap<>(mapInserted);
+ mergedMap.putAll(mapInserted2);
+
+ assertEachTupleInFTAIsInBuffer(copyMap, mergedMap);
+ }
+
+ @Test
+ public void testInsertVariableSizeTupleToMemoryManager() throws HyracksDataException {
+ Map<Integer, Integer> map = prepareVariableSizeTuples();
+ Map<TuplePointer, Integer> mapInserted = insertInFTAToBufferCouldFailForLargerTuples(map);
+ assertEachTupleInFTAIsInBuffer(map, mapInserted);
+ }
+
+ private void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
+ ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+ for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
+ accessor.reset(entry.getKey());
+ int dataLength = map.get(entry.getValue());
+ assertEquals((int) entry.getValue(),
+ IntSerDeUtils.getInt(accessor.getTupleBuffer().array(), accessor.getAbsFieldStartOffset(0)));
+ assertEquals(dataLength, accessor.getTupleLength());
+ }
+ assertEquals(map.size(), mapInserted.size());
+ }
+
+ private Map<Integer, Integer> prepareFixedSizeTuples(int tuplePerFrame) throws HyracksDataException {
+ Map<Integer, Integer> dataSet = new HashMap<>();
+ ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+ FixedSizeFrame frame = new FixedSizeFrame(buffer);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(frame, true);
+
+ int sizePerTuple = (Common.MIN_FRAME_SIZE - 1 - 4 - tuplePerFrame * 4 - 4) / tuplePerFrame;
+ int sizeChar = sizePerTuple - fieldsSerDer.length * 4 - 4 - 4;
+ assert (sizeChar > 0);
+ for (int i = 0; i < Common.NUM_MIN_FRAME * tuplePerFrame; i++) {
+ tupleBuilder.reset();
+ tupleBuilder.addField(fieldsSerDer[0], i);
+ tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', sizeChar));
+ assertTrue(appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+ tupleBuilder.getSize()));
+ dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+ }
+ inFTA.reset(buffer);
+ return dataSet;
+ }
+
+ private Map<Integer, Integer> prepareVariableSizeTuples() throws HyracksDataException {
+ Map<Integer, Integer> dataSet = new HashMap<>();
+ ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+ FixedSizeFrame frame = new FixedSizeFrame(buffer);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(frame, true);
+
+ for (int i = 0; true; i++) {
+ tupleBuilder.reset();
+ tupleBuilder.addField(fieldsSerDer[0], i);
+ tupleBuilder.addField(fieldsSerDer[1], Utility.repeatString('a', i));
+ if (!appender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0,
+ tupleBuilder.getSize())) {
+ break;
+ }
+ dataSet.put(i, tupleBuilder.getSize() + tupleBuilder.getFieldEndOffsets().length * 4);
+ }
+ inFTA.reset(buffer);
+ return dataSet;
+ }
+
+ private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() throws HyracksDataException {
+ Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
+ for (int i = 0; i < inFTA.getTupleCount(); i++) {
+ TuplePointer tuplePointer = new TuplePointer();
+ assertTrue(tupleMemoryManager.insertTuple(inFTA, i, tuplePointer));
+ tuplePointerIntegerMap.put(tuplePointer,
+ IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+ }
+ return tuplePointerIntegerMap;
+ }
+
+ private Map<TuplePointer, Integer> insertInFTAToBufferCouldFailForLargerTuples(Map<Integer, Integer> map)
+ throws HyracksDataException {
+ Map<TuplePointer, Integer> tuplePointerIdMap = new HashMap<>();
+ int i = 0;
+ for (; i < inFTA.getTupleCount(); i++) {
+ TuplePointer tuplePointer = new TuplePointer();
+ if (!tupleMemoryManager.insertTuple(inFTA, i, tuplePointer)) {
+ break;
+ }
+ tuplePointerIdMap.put(tuplePointer,
+ IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+ }
+ for (; i < inFTA.getTupleCount(); i++) {
+ map.remove(IntSerDeUtils.getInt(inFTA.getBuffer().array(), inFTA.getAbsoluteFieldStartOffset(i, 0)));
+ }
+ return tuplePointerIdMap;
+ }
+
+ private ByteBuffer deleteRandomSelectedTuples(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted,
+ int minNumOfRecordTobeDeleted)
+ throws HyracksDataException {
+ ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
+ FixedSizeFrame frame = new FixedSizeFrame(buffer);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(frame, true);
+
+ assert (minNumOfRecordTobeDeleted < mapInserted.size());
+ int countDeleted = minNumOfRecordTobeDeleted + random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
+
+ ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+ for (int i = 0; i < countDeleted; i++) {
+ Iterator<Map.Entry<TuplePointer, Integer>> iter = mapInserted.entrySet().iterator();
+ assert (iter.hasNext());
+ Map.Entry<TuplePointer, Integer> pair = iter.next();
+ accessor.reset(pair.getKey());
+ appender.append(accessor.getTupleBuffer().array(), accessor.getTupleStartOffset(),
+ accessor.getTupleLength());
+ map.remove(pair.getValue());
+ tupleMemoryManager.deleteTuple(pair.getKey());
+ iter.remove();
+ }
+ return buffer;
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
new file mode 100644
index 0000000..eece886
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
@@ -0,0 +1,233 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.sort.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+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.util.IntSerDeUtils;
+import edu.uci.ics.hyracks.dataflow.std.sort.Utility;
+
+public class DeletableFrameTupleAppenderTest {
+ DeletableFrameTupleAppender appender;
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE,
+ };
+ RecordDescriptor recordDescriptor = new RecordDescriptor(fields);
+ ArrayTupleBuilder builder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
+ static final char TEST_CH = 'x';
+
+ int cap = 256;
+
+ @Before
+ public void initial() throws HyracksDataException {
+ appender = new DeletableFrameTupleAppender(recordDescriptor);
+ }
+
+ @Test
+ public void testClear() throws Exception {
+ ByteBuffer buffer = ByteBuffer.allocate(cap);
+ appender.clear(buffer);
+ assertTrue(appender.getBuffer() == buffer);
+ assertTrue(appender.getTupleCount() == 0);
+ assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+ }
+
+ ByteBuffer makeAFrame(int capacity, int count, int deletedBytes) throws HyracksDataException {
+ ByteBuffer buffer = ByteBuffer.allocate(capacity);
+ int metaOffset = capacity - 4;
+ buffer.putInt(metaOffset, deletedBytes);
+ metaOffset -= 4;
+ buffer.putInt(metaOffset, count);
+ metaOffset -= 4;
+ for (int i = 0; i < count; i++, metaOffset -= 4) {
+ makeARecord(builder, i);
+ for (int x = 0; x < builder.getFieldEndOffsets().length; x++) {
+ buffer.putInt(builder.getFieldEndOffsets()[x]);
+ }
+ buffer.put(builder.getByteArray(), 0, builder.getSize());
+ assert (metaOffset > buffer.position());
+ buffer.putInt(metaOffset, buffer.position());
+
+ }
+ return buffer;
+ }
+
+ void makeARecord(ArrayTupleBuilder builder, int i) throws HyracksDataException {
+ builder.reset();
+ builder.addField(fields[0], i + 1);
+ builder.addField(fields[1], Utility.repeatString(TEST_CH, i + 1));
+ }
+
+ int assertTupleIsExpected(int i, int dataOffset) {
+ int tupleLength = 2 * 4 + 4 + 2 + i + 1;
+ assertEquals(dataOffset, appender.getTupleStartOffset(i));
+ assertEquals(tupleLength, appender.getTupleLength(i));
+
+ assertEquals(dataOffset + 2 * 4, appender.getAbsoluteFieldStartOffset(i, 0));
+ assertEquals(4, appender.getFieldLength(i, 0));
+ assertEquals(i + 1,
+ IntSerDeUtils.getInt(appender.getBuffer().array(), appender.getAbsoluteFieldStartOffset(i, 0)));
+ assertEquals(dataOffset + 2 * 4 + 4, appender.getAbsoluteFieldStartOffset(i, 1));
+ assertEquals(2 + i + 1, appender.getFieldLength(i, 1));
+ return tupleLength;
+ }
+
+ @Test
+ public void testReset() throws Exception {
+ ByteBuffer buffer = ByteBuffer.allocate(cap);
+ appender.reset(buffer);
+ assertTrue(appender.getBuffer() == buffer);
+ assertTrue(appender.getTupleCount() == 0);
+ assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+
+ int count = 10;
+ int deleted = 7;
+ buffer = makeAFrame(cap, count, deleted);
+ int pos = buffer.position();
+ appender.reset(buffer);
+ assertTrue(appender.getBuffer() == buffer);
+ assertTrue(appender.getTupleCount() == count);
+ assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4 - count * 4 - pos);
+ assertTrue(appender.getTotalFreeSpace() == appender.getContiguousFreeSpace() + deleted);
+
+ int dataOffset = 0;
+ for (int i = 0; i < count; i++) {
+ dataOffset += assertTupleIsExpected(i, dataOffset);
+ }
+ }
+
+ @Test
+ public void testAppend() throws Exception {
+ int count = 10;
+ ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+ DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+ accessor.reset(bufferRead);
+ ByteBuffer bufferWrite = ByteBuffer.allocate(cap);
+ appender.clear(bufferWrite);
+ for (int i = 0; i < accessor.getTupleCount(); i++) {
+ appender.append(accessor, i);
+ }
+ for (int i = 0; i < bufferRead.capacity(); i++) {
+ assertEquals(bufferRead.get(i), bufferWrite.get(i));
+ }
+ }
+
+ @Test
+ public void testDelete() throws Exception {
+ int count = 10;
+ int deleteSpace = 0;
+ ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
+ appender.reset(buffer);
+
+ int freeSpace = appender.getContiguousFreeSpace();
+ for (int i = 0; i < appender.getTupleCount(); i++) {
+ deleteSpace += assertDeleteSucceed(i, freeSpace, deleteSpace);
+ int innerOffset = deleteSpace;
+ for (int j = i + 1; j < appender.getTupleCount(); j++) {
+ innerOffset += assertTupleIsExpected(j, innerOffset);
+ }
+ }
+ }
+
+ @Test
+ public void testResetAfterDelete() throws Exception {
+ testDelete();
+ appender.reset(appender.getBuffer());
+ assertEquals(cap - appender.getTupleCount() * 4 - 4 - 4, appender.getTotalFreeSpace());
+
+ }
+
+ int assertDeleteSucceed(int i, int freeSpaceBeforeDelete, int deleteSpace) {
+ int startOffset = appender.getTupleStartOffset(i);
+ int endOffset = appender.getTupleEndOffset(i);
+ int tupleLength = appender.getTupleLength(i);
+
+ appender.delete(i);
+
+ assertEquals(startOffset, appender.getTupleStartOffset(i));
+ assertEquals(-endOffset, appender.getTupleEndOffset(i));
+ assertEquals(-tupleLength, appender.getTupleLength(i));
+ assertEquals(freeSpaceBeforeDelete, appender.getContiguousFreeSpace());
+ assertEquals(deleteSpace + tupleLength + freeSpaceBeforeDelete, appender.getTotalFreeSpace());
+ return tupleLength;
+ }
+
+ @Test
+ public void testAppendAndDelete() throws Exception {
+ int cap = 1024;
+ int count = 10;
+ int deleteSpace = 0;
+ ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
+ int dataOffset = buffer.position();
+ appender.reset(buffer);
+
+ int freeSpace = appender.getContiguousFreeSpace();
+ int[] deleteSet = new int[] { 1, 3, 5 };
+ for (int i = 0; i < deleteSet.length; i++) {
+ deleteSpace += assertDeleteSucceed(deleteSet[i], freeSpace, deleteSpace);
+ }
+
+ ByteBuffer bufferRead = makeAFrame(cap, count * 2, 0);
+ DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+ accessor.reset(bufferRead);
+
+ for (int i = count; i < accessor.getTupleCount(); i++) {
+ int id = appender.append(accessor, i);
+ dataOffset += assertTupleIsExpected(i, dataOffset);
+ assertEquals(i, id);
+ }
+
+ appender.reOrganizeBuffer();
+ dataOffset = 0;
+ for (int i = 0; i < appender.getTupleCount(); i++) {
+ if (ArrayUtils.contains(deleteSet, i)) {
+ continue;
+ }
+ dataOffset += assertTupleIsExpected(i, dataOffset);
+ }
+ }
+
+ @Test
+ public void testReOrganizeBuffer() throws Exception {
+ int count = 10;
+ testDelete();
+ appender.reOrganizeBuffer();
+ ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+ DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
+ accessor.reset(bufferRead);
+ for (int i = 0; i < accessor.getTupleCount(); i++) {
+ appender.append(accessor, i);
+ }
+ for (int i = 0; i < bufferRead.capacity(); i++) {
+ assertEquals(bufferRead.get(i), appender.getBuffer().get(i));
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java
new file mode 100644
index 0000000..672e078
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/AbstracHeapTest.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Random;
+
+public class AbstracHeapTest {
+ Random random = new Random(System.currentTimeMillis());
+
+ class IntFactory implements IResetableComparableFactory<Int> {
+ @Override
+ public IResetableComparable<Int> createResetableComparable() {
+ return new Int();
+ }
+ }
+
+ class Int implements IResetableComparable<Int> {
+ int i;
+
+ public Int() {
+ i = 0;
+ }
+
+ public Int(int i) {
+ this.i = i;
+ }
+
+ @Override
+ public void reset(Int other) {
+ i = other.i;
+ }
+
+ @Override
+ public int compareTo(Int o) {
+ return Integer.compare(i, o.i);
+ }
+ }
+
+ protected void assertGetMinHeapIsSorted(IMinHeap minHeap) {
+ int count = minHeap.getNumEntries();
+ Int minI = new Int();
+ Int peekI = new Int();
+ int preI = Integer.MIN_VALUE;
+ while (!minHeap.isEmpty()) {
+ count--;
+ minHeap.peekMin(peekI);
+ minHeap.getMin(minI);
+ assertTrue(peekI.compareTo(minI) == 0);
+ assertTrue(preI <= minI.i);
+ preI = minI.i;
+ }
+ assertEquals(0, count);
+ }
+
+ protected void assertGetMaxHeapIsSorted(IMaxHeap maxHeap) {
+ int count = maxHeap.getNumEntries();
+ Int maxI = new Int();
+ Int peekI = new Int();
+ int preI = Integer.MAX_VALUE;
+ while (!maxHeap.isEmpty()) {
+ count--;
+ maxHeap.peekMax(peekI);
+ maxHeap.getMax(maxI);
+ assertTrue(peekI.compareTo(maxI) == 0);
+ assertTrue(preI >= maxI.i);
+ preI = maxI.i;
+ }
+ assertEquals(0, count);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java
new file mode 100644
index 0000000..3f2404c
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MaxHeapTest.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MaxHeapTest extends AbstracHeapTest {
+ @Test
+ public void testInitialMinHeap() {
+ int capacity = 10;
+ MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+ assertTrue(maxHeap.isEmpty());
+ assertEquals(0, maxHeap.getNumEntries());
+ }
+
+ @Test
+ public void testInsertSmallAmountElements() {
+ int capacity = 10;
+ MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ maxHeap.insert(new Int(capacity - i));
+ }
+ assertEquals(capacity, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+
+ assertGetMaxHeapIsSorted(maxHeap);
+
+ for (int i = 0; i < capacity; i++) {
+ maxHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+ assertGetMaxHeapIsSorted(maxHeap);
+ }
+
+ @Test
+ public void testInsertLargerThanCapacityElements() {
+ int capacity = 10;
+ MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ maxHeap.insert(new Int(capacity - i));
+ }
+ assertEquals(capacity, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+ assertGetMaxHeapIsSorted(maxHeap);
+
+ for (int i = 0; i < capacity * 10; i++) {
+ maxHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity * 10, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+ assertGetMaxHeapIsSorted(maxHeap);
+
+ }
+
+ @Test
+ public void testReplaceMax() {
+ int capacity = 10;
+ MaxHeap maxHeap = new MaxHeap(new IntFactory(), capacity);
+ for (int i = capacity; i < capacity * 2; i++) {
+ maxHeap.insert(new Int(i));
+ }
+ assertEquals(capacity, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+
+ for (int i = 0; i < capacity; i++) {
+ maxHeap.replaceMax(new Int(i));
+ }
+ assertEquals(capacity, maxHeap.getNumEntries());
+ assertFalse(maxHeap.isEmpty());
+
+ Int maxI = new Int();
+ Int peekI = new Int();
+ int i = 0;
+ while (!maxHeap.isEmpty()) {
+ maxHeap.peekMax(peekI);
+ maxHeap.getMax(maxI);
+ assertTrue(peekI.compareTo(maxI) == 0);
+ assertEquals( i++, capacity - 1 - maxI.i);
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java
new file mode 100644
index 0000000..bcdabd1
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinHeapTest.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MinHeapTest extends AbstracHeapTest{
+
+ @Test
+ public void testInitialMinHeap() {
+ int capacity = 10;
+ MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+ assertTrue(minHeap.isEmpty());
+ assertEquals(0, minHeap.getNumEntries());
+ }
+
+ @Test
+ public void testInsertSmallAmountElements() {
+ int capacity = 10;
+ MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ minHeap.insert(new Int(capacity - i));
+ }
+ assertEquals(capacity, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+
+ assertGetMinHeapIsSorted(minHeap);
+
+ for (int i = 0; i < capacity; i++) {
+ minHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+ assertGetMinHeapIsSorted(minHeap);
+ }
+
+ @Test
+ public void testInsertLargerThanCapacityElements() {
+ int capacity = 10;
+ MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ minHeap.insert(new Int(capacity - i));
+ }
+ assertEquals(capacity, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+ assertGetMinHeapIsSorted(minHeap);
+
+ for (int i = 0; i < capacity * 10; i++) {
+ minHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity * 10, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+ assertGetMinHeapIsSorted(minHeap);
+
+ }
+
+ @Test
+ public void testReplaceMin() {
+ int capacity = 10;
+ MinHeap minHeap = new MinHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ minHeap.insert(new Int(i));
+ }
+ assertEquals(capacity, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+
+ for (int i = capacity; i < capacity * 2; i++) {
+ minHeap.replaceMin(new Int(i));
+ }
+ assertEquals(capacity, minHeap.getNumEntries());
+ assertFalse(minHeap.isEmpty());
+
+ Int minI = new Int();
+ Int peekI = new Int();
+ int i = 0;
+ while (!minHeap.isEmpty()) {
+ minHeap.peekMin(peekI);
+ minHeap.getMin(minI);
+ assertTrue(peekI.compareTo(minI) == 0);
+ assertEquals(i++ + capacity, minI.i);
+ }
+ }
+
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java
new file mode 100644
index 0000000..1f16e08
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/structures/MinMaxHeapTest.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.structures;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+public class MinMaxHeapTest extends AbstracHeapTest {
+
+ @Test
+ public void testInitialMinMaxHeap() {
+ int capacity = 10;
+ MinMaxHeap minHeap = new MinMaxHeap(new IntFactory(), capacity);
+ assertTrue(minHeap.isEmpty());
+ assertEquals(0, minHeap.getNumEntries());
+ }
+
+ @Test
+ public void testInsertElements() {
+ int capacity = 10;
+ MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity * 10; i++) {
+ minMaxHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity * 10, minMaxHeap.getNumEntries());
+ assertFalse(minMaxHeap.isEmpty());
+ assertGetMinHeapIsSorted(minMaxHeap);
+
+ for (int i = 0; i < capacity * 10; i++) {
+ minMaxHeap.insert(new Int(random.nextInt()));
+ }
+ assertEquals(capacity * 10, minMaxHeap.getNumEntries());
+ assertGetMaxHeapIsSorted(minMaxHeap);
+ }
+
+ @Test
+ public void testReplaceMin() {
+ int capacity = 10;
+ MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ minMaxHeap.insert(new Int(i));
+ }
+ assertEquals(capacity, minMaxHeap.getNumEntries());
+ assertFalse(minMaxHeap.isEmpty());
+
+ for (int i = capacity; i < capacity * 2; i++) {
+ minMaxHeap.replaceMin(new Int(i));
+ }
+ assertEquals(capacity, minMaxHeap.getNumEntries());
+ assertFalse(minMaxHeap.isEmpty());
+
+ Int minI = new Int();
+ Int peekI = new Int();
+ int i = 0;
+ while (!minMaxHeap.isEmpty()) {
+ minMaxHeap.peekMin(peekI);
+ minMaxHeap.getMin(minI);
+ assertTrue(peekI.compareTo(minI) == 0);
+ assertEquals(i++ + capacity, minI.i);
+ }
+ }
+
+ @Test
+ public void testReplaceMax() {
+ int capacity = 10;
+ MinMaxHeap minMaxHeap = new MinMaxHeap(new IntFactory(), capacity);
+ for (int i = 0; i < capacity; i++) {
+ minMaxHeap.insert(new Int(i + capacity));
+ }
+ assertEquals(capacity, minMaxHeap.getNumEntries());
+ assertFalse(minMaxHeap.isEmpty());
+
+ Int maxI = new Int();
+ for (int i = capacity; i < capacity * 2; i++) {
+ minMaxHeap.peekMax(maxI);
+ minMaxHeap.replaceMax(new Int(i - capacity));
+ }
+ assertEquals(capacity, minMaxHeap.getNumEntries());
+ assertFalse(minMaxHeap.isEmpty());
+
+ System.out.println();
+ Int peekI = new Int();
+ int i = 0;
+ while (!minMaxHeap.isEmpty()) {
+ minMaxHeap.peekMax(peekI);
+ minMaxHeap.getMax(maxI);
+ assertTrue(peekI.compareTo(maxI) == 0);
+ assertEquals(capacity - i - 1, maxI.i);
+ i++;
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java
new file mode 100644
index 0000000..332ac98
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/test/java/edu/uci/ics/hyracks/dataflow/std/util/MathTest.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.dataflow.std.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.Random;
+
+import org.junit.Test;
+
+public class MathTest {
+
+ @Test
+ public void testLog2() {
+ Random random = new Random(System.currentTimeMillis());
+ for (int i = 0; i < 31; i++) {
+ assertTrue(MathUtil.log2Floor((int) Math.pow(2, i)) == i);
+ for(int x = 0; x < 10; x++){
+ float extra = random.nextFloat();
+ while (extra >= 1.0){
+ extra = random.nextFloat();
+ }
+ assertTrue(MathUtil.log2Floor((int) Math.pow(2, i + extra)) == i);
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
index ed4cf0c..c869362 100644
--- a/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
+++ b/hyracks/hyracks-examples/btree-example/btreehelper/src/main/java/edu/uci/ics/hyracks/examples/btree/helper/DataGenOperatorDescriptor.java
@@ -16,10 +16,10 @@
package edu.uci.ics.hyracks.examples.btree.helper;
import java.io.DataOutput;
-import java.nio.ByteBuffer;
import java.util.HashSet;
import java.util.Random;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -28,7 +28,6 @@
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
@@ -61,8 +60,7 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
- final ByteBuffer outputFrame = ctx.allocateFrame();
- final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
final RecordDescriptor recDesc = recordDescriptors[0];
final ArrayTupleBuilder tb = new ArrayTupleBuilder(recDesc.getFields().length);
final Random rnd = new Random(randomSeed);
@@ -79,7 +77,6 @@
public void initialize() throws HyracksDataException {
writer.open();
try {
- appender.reset(outputFrame, true);
for (int i = 0; i < numRecords; i++) {
tb.reset();
for (int j = 0; j < recDesc.getFieldCount(); j++) {
@@ -87,14 +84,13 @@
}
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(outputFrame, writer);
- appender.reset(outputFrame, true);
+ appender.flush(writer, true);
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
}
}
}
- FrameUtils.flushFrame(outputFrame, writer);
+ appender.flush(writer, true);
} catch (Exception e) {
writer.fail();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore b/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore
new file mode 100644
index 0000000..be303ea
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/.gitignore
@@ -0,0 +1,3 @@
+primary*/
+secondary*/
+inv*/
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
index 29be04b..5268150 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/comm/SerializationDeserializationTest.java
@@ -24,8 +24,10 @@
import org.junit.Test;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameReader;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IDataWriter;
import edu.uci.ics.hyracks.api.dataflow.IOpenableDataReader;
@@ -48,12 +50,12 @@
private final IHyracksTaskContext ctx;
private static final int FRAME_SIZE = 32768;
private RecordDescriptor rDes;
- private List<ByteBuffer> buffers;
+ private List<IFrame> buffers;
public SerDeserRunner(RecordDescriptor rDes) throws HyracksException {
ctx = TestUtils.create(FRAME_SIZE);
this.rDes = rDes;
- buffers = new ArrayList<ByteBuffer>();
+ buffers = new ArrayList<>();
}
public IOpenableDataWriter<Object[]> createWriter() throws HyracksDataException {
@@ -64,8 +66,8 @@
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
- ByteBuffer toBuf = ctx.allocateFrame();
- toBuf.put(buffer);
+ IFrame toBuf = new VSizeFrame(ctx);
+ toBuf.getBuffer().put(buffer);
buffers.add(toBuf);
}
@@ -89,12 +91,12 @@
}
@Override
- public boolean nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
if (i < buffers.size()) {
- ByteBuffer buf = buffers.get(i);
- buf.flip();
- buffer.put(buf);
- buffer.flip();
+ IFrame buf = buffers.get(i);
+ buf.getBuffer().flip();
+ frame.getBuffer().put(buf.getBuffer());
+ frame.getBuffer().flip();
++i;
return true;
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 1150cf3..30f89e2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -15,10 +15,11 @@
package edu.uci.ics.hyracks.tests.integration;
import java.io.BufferedReader;
+import java.io.BufferedWriter;
import java.io.File;
import java.io.FileReader;
+import java.io.FileWriter;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
@@ -34,6 +35,7 @@
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
import edu.uci.ics.hyracks.api.dataset.ResultSetId;
@@ -45,6 +47,7 @@
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -138,12 +141,11 @@
hcc.waitForCompletion(jobId);
}
+
protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
int nReaders = 1;
- ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
- resultBuffer.clear();
- IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
@@ -151,16 +153,18 @@
List<String> resultRecords = new ArrayList<String>();
ByteBufferInputStream bbis = new ByteBufferInputStream();
- int readSize = reader.read(resultBuffer);
+ FrameManager resultDisplayFrameMgr = new FrameManager(spec.getFrameSize());
+ VSizeFrame frame = new VSizeFrame(resultDisplayFrameMgr);
+ int readSize = reader.read(frame);
while (readSize > 0) {
try {
- frameTupleAccessor.reset(resultBuffer);
+ frameTupleAccessor.reset(frame.getBuffer());
for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
int start = frameTupleAccessor.getTupleStartOffset(tIndex);
int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
- bbis.setByteBuffer(resultBuffer, start);
+ bbis.setByteBuffer(frame.getBuffer(), start);
byte[] recordBytes = new byte[length];
bbis.read(recordBytes, 0, length);
resultRecords.add(new String(recordBytes, 0, length));
@@ -169,8 +173,7 @@
bbis.close();
}
- resultBuffer.clear();
- readSize = reader.read(resultBuffer);
+ readSize = reader.read(frame);
}
return resultRecords;
}
@@ -198,6 +201,22 @@
return true;
}
+ protected void runTestAndStoreResult(JobSpecification spec, File file) throws Exception {
+ JobId jobId = executeTest(spec);
+
+ BufferedWriter output = new BufferedWriter(new FileWriter(file));
+ List<String> results;
+ for (int i = 0; i < spec.getResultSetIds().size(); i++) {
+ results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+ for(String str : results) {
+ output.write(str);
+ }
+ }
+ output.close();
+
+ hcc.waitForCompletion(jobId);
+ }
+
protected File createTempFile() throws IOException {
File tempFile = File.createTempFile(getClass().getName(), ".tmp", outputFolder.getRoot());
if (LOGGER.isLoggable(Level.INFO)) {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index 970f2fe..602e193 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,7 +16,6 @@
import java.io.File;
import java.io.IOException;
-import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
@@ -33,6 +32,7 @@
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -44,6 +44,7 @@
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
@@ -66,7 +67,8 @@
public TemporaryFolder outputFolder = new TemporaryFolder();
public AbstractMultiNCIntegrationTest() {
- outputFiles = new ArrayList<File>();;
+ outputFiles = new ArrayList<File>();
+ ;
}
@BeforeClass
@@ -124,10 +126,10 @@
int nReaders = 1;
- ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
- resultBuffer.clear();
+ FrameManager resultDisplayFrameMgr = new FrameManager(spec.getFrameSize());
+ VSizeFrame resultFrame = new VSizeFrame(resultDisplayFrameMgr);
- IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor();
IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
@@ -135,16 +137,16 @@
JSONArray resultRecords = new JSONArray();
ByteBufferInputStream bbis = new ByteBufferInputStream();
- int readSize = reader.read(resultBuffer);
+ int readSize = reader.read(resultFrame);
while (readSize > 0) {
try {
- frameTupleAccessor.reset(resultBuffer);
+ frameTupleAccessor.reset(resultFrame.getBuffer());
for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
int start = frameTupleAccessor.getTupleStartOffset(tIndex);
int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
- bbis.setByteBuffer(resultBuffer, start);
+ bbis.setByteBuffer(resultFrame.getBuffer(), start);
byte[] recordBytes = new byte[length];
bbis.read(recordBytes, 0, length);
resultRecords.put(new String(recordBytes, 0, length));
@@ -157,8 +159,7 @@
}
}
- resultBuffer.clear();
- readSize = reader.read(resultBuffer);
+ readSize = reader.read(resultFrame);
}
hcc.waitForCompletion(jobId);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index a2ef99a..9b77ec5 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -42,9 +42,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class OptimizedSortMergeTest extends AbstractIntegrationTest {
@@ -72,17 +73,22 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
- OptimizedExternalSortOperatorDescriptor sorter = new OptimizedExternalSortOperatorDescriptor(spec, 4,
- new int[] { 1, 0 }, new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+ int outputLimit = 5; // larger than the total record numbers.
+ TopKSorterOperatorDescriptor sorter = new TopKSorterOperatorDescriptor(spec, 4,
+ outputLimit, new int[] { 1, 0 }, null, new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
ResultSetId rsId = new ResultSetId(1);
spec.addResultSetId(rsId);
- IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false, false,
- ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ File file = File.createTempFile(getClass().getName(), ".tmp");
+ IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, file.getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -98,6 +104,7 @@
new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
runTest(spec);
+ System.out.println("Result write into :" + file.getAbsolutePath());
}
@Test
@@ -123,11 +130,11 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
- int outputLimit = 200;
- OptimizedExternalSortOperatorDescriptor sorter = new OptimizedExternalSortOperatorDescriptor(spec, 4,
- outputLimit, new int[] { 1, 0 }, new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+ int outputLimit = 20;
+ TopKSorterOperatorDescriptor sorter = new TopKSorterOperatorDescriptor(spec, 4,
+ outputLimit, new int[] { 1, 0 }, null, new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java
new file mode 100644
index 0000000..212a99c
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/VSizeFrameSortMergeTest.java
@@ -0,0 +1,118 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.integration;
+
+import java.io.File;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
+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.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;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+
+public class VSizeFrameSortMergeTest extends AbstractIntegrationTest {
+
+ public static String[] INPUTS = { "data/tpch0.001/orders-part1.tbl", "data/tpch0.001/orders-part2.tbl" };
+
+ FileSplit[] ordersSplits = new FileSplit[] {
+ new FileSplit(NC1_ID, new FileReference(new File(INPUTS[0]))),
+ new FileSplit(NC2_ID, new FileReference(new File(INPUTS[1]))) };
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
+
+ @Test
+ public void sortNormalMergeTest() throws Exception {
+ sortTask(1024, 4);
+ sortTask(256, 4);
+ }
+
+ @Test
+ public void sortLargeMergeTest() throws Exception {
+ sortTask(32, 128);
+ sortTask(16, 256);
+ sortTask(16, 10240);
+ }
+
+ public void sortTask(int frameSize, int frameLimit) throws Exception {
+ JobSpecification spec = new JobSpecification();
+
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);
+ // PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID );
+
+ spec.setFrameSize(frameSize);
+ ExternalSortOperatorDescriptor sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, new int[] { 1, 0 },
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
+
+ File file = File.createTempFile(getClass().getName(), ".tmp");
+
+ IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(
+ new FileSplit[] { new FileSplit(NC1_ID, file.getAbsolutePath()) });
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+ spec.connect(
+ new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] {
+ 1, 0 }, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 },
+ new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory()), sorter, 0, printer, 0);
+
+ spec.addRoot(printer);
+ runTest(spec);
+ System.out.println("Result write into :" + file.getAbsolutePath());
+ }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java
new file mode 100644
index 0000000..c4faa1a
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/AbstractRunGeneratorTest.java
@@ -0,0 +1,279 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+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.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupFrameAccessor;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
+import edu.uci.ics.hyracks.test.support.TestUtils;
+
+public abstract class AbstractRunGeneratorTest {
+ static TestUtils testUtils = new TestUtils();
+ static ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] {
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+ static RecordDescriptor RecordDesc = new RecordDescriptor(SerDers);
+ static Random GRandom = new Random(System.currentTimeMillis());
+ static int[] SortFields = new int[] { 0, 1 };
+ static IBinaryComparatorFactory[] ComparatorFactories = new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+ static void assertMaxFrameSizesAreAllEqualsTo(List<RunAndMaxFrameSizePair> maxSize, int pageSize) {
+ for (int i = 0; i < maxSize.size(); i++) {
+ assertTrue(maxSize.get(i).maxFrameSize == pageSize);
+ }
+ }
+
+ abstract AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+ throws HyracksDataException;
+
+ protected List<RunAndMaxFrameSizePair> testSortRecords(int pageSize, int frameLimit, int numRuns, int minRecordSize,
+ int maxRecordSize, HashMap<Integer, String> specialData) throws HyracksDataException {
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+
+ HashMap<Integer, String> keyValuePair = new HashMap<>();
+ List<IFrame> frameList = new ArrayList<>();
+ prepareData(ctx, frameList, pageSize * frameLimit * numRuns, minRecordSize, maxRecordSize,
+ specialData, keyValuePair);
+ AbstractSortRunGenerator runGenerator = getSortRunGenerator(ctx, frameLimit, keyValuePair.size());
+ runGenerator.open();
+ for (IFrame frame : frameList) {
+ runGenerator.nextFrame(frame.getBuffer());
+ }
+ runGenerator.close();
+ matchResult(ctx, runGenerator.getRuns(), keyValuePair);
+ return runGenerator.getRuns();
+ }
+
+ static void matchResult(IHyracksTaskContext ctx, List<RunAndMaxFrameSizePair> runs,
+ Map<Integer, String> keyValuePair) throws HyracksDataException {
+ IFrame frame = new VSizeFrame(ctx);
+ FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+
+ HashMap<Integer, String> copyMap = new HashMap<>(keyValuePair);
+ assertReadSorted(runs, fta, frame, copyMap);
+
+ HashMap<Integer, String> copyMap2 = new HashMap<>(keyValuePair);
+ int maxFrameSizes = 0;
+ for (RunAndMaxFrameSizePair run : runs) {
+ maxFrameSizes = Math.max(maxFrameSizes, run.maxFrameSize);
+ }
+ GroupVSizeFrame gframe = new GroupVSizeFrame(ctx, maxFrameSizes);
+ GroupFrameAccessor gfta = new GroupFrameAccessor(ctx.getInitialFrameSize(), RecordDesc);
+ assertReadSorted(runs, gfta, gframe, copyMap2);
+ }
+
+ static int assertFTADataIsSorted(IFrameTupleAccessor fta, Map<Integer, String> keyValuePair, int preKey)
+ throws HyracksDataException {
+
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+ for (int i = 0; i < fta.getTupleCount(); i++) {
+ bbis.setByteBuffer(fta.getBuffer(),
+ fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+ int key = (int) RecordDesc.getFields()[0].deserialize(di);
+ bbis.setByteBuffer(fta.getBuffer(),
+ fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 1) + fta.getFieldSlotsLength());
+ String value = (String) RecordDesc.getFields()[1].deserialize(di);
+
+ if (!keyValuePair.get(key).equals(value)) {
+ assertTrue(false);
+ }
+ keyValuePair.remove(key);
+ assertTrue(key >= preKey);
+ preKey = key;
+ }
+ return preKey;
+ }
+
+ static void assertReadSorted(List<RunAndMaxFrameSizePair> runs, IFrameTupleAccessor fta, IFrame frame,
+ Map<Integer, String> keyValuePair) throws HyracksDataException {
+
+ assertTrue(runs.size() > 0);
+ for (RunAndMaxFrameSizePair run : runs) {
+ run.run.open();
+ int preKey = Integer.MIN_VALUE;
+ while (run.run.nextFrame(frame)) {
+ fta.reset(frame.getBuffer());
+ preKey = assertFTADataIsSorted(fta, keyValuePair, preKey);
+ }
+ run.run.close();
+ }
+ assertTrue(keyValuePair.isEmpty());
+ }
+
+ static void prepareData(IHyracksTaskContext ctx, List<IFrame> frameList, int minDataSize, int minRecordSize,
+ int maxRecordSize, Map<Integer, String> specialData, Map<Integer, String> keyValuePair)
+ throws HyracksDataException {
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+ FrameTupleAppender appender = new FrameTupleAppender();
+
+ int datasize = 0;
+ if (specialData != null) {
+ for (Map.Entry<Integer, String> entry : specialData.entrySet()) {
+ tb.reset();
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, entry.getKey());
+ tb.addField(UTF8StringSerializerDeserializer.INSTANCE, entry.getValue());
+
+ VSizeFrame frame = new VSizeFrame(ctx, FrameHelper
+ .calcAlignedFrameSizeToStore(tb.getFieldEndOffsets().length, tb.getSize(), ctx.getInitialFrameSize()));
+ appender.reset(frame, true);
+ assertTrue(appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize()));
+ frameList.add(frame);
+ datasize += frame.getFrameSize();
+ }
+ keyValuePair.putAll(specialData);
+ }
+
+ VSizeFrame frame = new VSizeFrame(ctx, ctx.getInitialFrameSize());
+ appender.reset(frame, true);
+ while (datasize < minDataSize) {
+ tb.reset();
+ int key = GRandom.nextInt(minDataSize + 1);
+ if (!keyValuePair.containsKey(key)) {
+ String value = generateRandomRecord(minRecordSize, maxRecordSize);
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, key);
+ tb.addField(UTF8StringSerializerDeserializer.INSTANCE, value);
+
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ frameList.add(frame);
+ datasize += frame.getFrameSize();
+ frame = new VSizeFrame(ctx, FrameHelper
+ .calcAlignedFrameSizeToStore(tb.getFieldEndOffsets().length, tb.getSize(),
+ ctx.getInitialFrameSize()));
+ appender.reset(frame, true);
+ assertTrue(appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize()));
+ }
+
+ keyValuePair.put(key, value);
+ }
+ }
+ if (appender.getTupleCount() > 0) {
+ frameList.add(frame);
+ }
+
+ }
+
+ static String generateRandomRecord(int minRecordSize, int maxRecordSize)
+ throws HyracksDataException {
+ int size = GRandom.nextInt(maxRecordSize - minRecordSize + 1) + minRecordSize;
+ return generateRandomFixSizedString(size);
+
+ }
+
+ static String generateRandomFixSizedString(int size) {
+ StringBuilder sb = new StringBuilder(size);
+ for (; size >= 0; --size) {
+ char ch = (char) (GRandom.nextInt(26) + 97);
+ sb.append(ch);
+ }
+ return sb.toString();
+ }
+
+ static HashMap<Integer, String> generateBigObject(int pageSize, int times) {
+ HashMap<Integer, String> map = new HashMap<>(1);
+ for (int i = 1; i < times; i++) {
+ map.put(GRandom.nextInt(), generateRandomFixSizedString(pageSize * i));
+ }
+ return map;
+ }
+
+ @Test
+ public void testAllSmallRecords() throws HyracksDataException {
+ int pageSize = 512;
+ int frameLimit = 4;
+ int numRuns = 2;
+ int minRecordSize = pageSize / 8;
+ int maxRecordSize = pageSize / 8;
+ List<RunAndMaxFrameSizePair> maxSize = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize,
+ maxRecordSize, null);
+ assertMaxFrameSizesAreAllEqualsTo(maxSize, pageSize);
+ }
+
+ @Test
+ public void testAllLargeRecords() throws HyracksDataException {
+ int pageSize = 2048;
+ int frameLimit = 4;
+ int numRuns = 2;
+ int minRecordSize = pageSize;
+ int maxRecordSize = (int) (pageSize * 1.8);
+ List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+ null);
+ assertMaxFrameSizesAreAllEqualsTo(size, pageSize * 2);
+ }
+
+ @Test
+ public void testMixedLargeRecords() throws HyracksDataException {
+ int pageSize = 128;
+ int frameLimit = 4;
+ int numRuns = 4;
+ int minRecordSize = 20;
+ int maxRecordSize = pageSize / 2;
+ HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit - 1);
+ List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+ specialPair);
+
+ int max = 0;
+ for (RunAndMaxFrameSizePair run : size) {
+ max = Math.max(max, run.maxFrameSize);
+ }
+ assertTrue(max == pageSize * (frameLimit - 1));
+ }
+
+ @Test(expected = HyracksDataException.class)
+ public void testTooBigRecordWillThrowException() throws HyracksDataException {
+ int pageSize = 1024;
+ int frameLimit = 8;
+ int numRuns = 8;
+ HashMap<Integer, String> specialPair = generateBigObject(pageSize, frameLimit);
+ int minRecordSize = 10;
+ int maxRecordSize = pageSize / 2;
+ List<RunAndMaxFrameSizePair> size = testSortRecords(pageSize, frameLimit, numRuns, minRecordSize, maxRecordSize,
+ specialPair);
+
+ }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java
new file mode 100644
index 0000000..4d7558b
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/ExternalSortRunGeneratorTest.java
@@ -0,0 +1,32 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+
+public class ExternalSortRunGeneratorTest extends AbstractRunGeneratorTest {
+
+ @Override
+ AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+ throws HyracksDataException {
+ return new ExternalSortRunGenerator(ctx, SortFields, null, ComparatorFactories, RecordDesc,
+ Algorithm.MERGE_SORT, frameLimit);
+ }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java
new file mode 100644
index 0000000..00eca70
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HeapSortRunGeneratorTest.java
@@ -0,0 +1,37 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HeapSortRunGenerator;
+
+public class HeapSortRunGeneratorTest extends AbstractRunGeneratorTest {
+ @Override
+ AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+ throws HyracksDataException {
+ return new HeapSortRunGenerator(ctx, frameLimit, numOfInputRecord, SortFields, null, ComparatorFactories,
+ RecordDesc);
+ }
+
+ @Test
+ public void testTopK(){
+
+ }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java
new file mode 100644
index 0000000..f7ecd5e
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/HybridSortRunGenerator.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HybridTopKSortRunGenerator;
+
+public class HybridSortRunGenerator extends AbstractRunGeneratorTest {
+ @Override
+ AbstractSortRunGenerator getSortRunGenerator(IHyracksTaskContext ctx, int frameLimit, int numOfInputRecord)
+ throws HyracksDataException {
+ return new HybridTopKSortRunGenerator(ctx, frameLimit, numOfInputRecord, SortFields, null, ComparatorFactories,
+ RecordDesc);
+ }
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java
new file mode 100644
index 0000000..d5355b8
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/RunMergingFrameReaderTest.java
@@ -0,0 +1,409 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.ComparatorFactories;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.GRandom;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.RecordDesc;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.SortFields;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.generateRandomRecord;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.matchResult;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.prepareData;
+import static edu.uci.ics.hyracks.tests.unit.ExternalSortRunGeneratorTest.testUtils;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameReader;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
+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.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+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.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunAndMaxFrameSizePair;
+import edu.uci.ics.hyracks.dataflow.std.sort.RunMergingFrameReader;
+import edu.uci.ics.hyracks.dataflow.std.sort.util.GroupVSizeFrame;
+
+public class RunMergingFrameReaderTest {
+ static IBinaryComparator[] Comparators = new IBinaryComparator[] {
+ ComparatorFactories[0].createBinaryComparator(),
+ ComparatorFactories[1].createBinaryComparator(),
+ };
+
+ static class TestFrameReader implements IFrameReader {
+
+ private final int pageSize;
+ private final int numFrames;
+ private final int minRecordSize;
+ private final int maxRecordSize;
+ private TreeMap<Integer, String> result = new TreeMap<>();
+ int maxFrameSize;
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+ FrameTupleAppender appender = new FrameTupleAppender();
+ private Iterator<Map.Entry<Integer, String>> iterator;
+ private Map.Entry<Integer, String> lastEntry;
+
+ TestFrameReader(int pageSize, int numFrames, int minRecordSize, int maxRecordSize) {
+ this.pageSize = pageSize;
+ this.numFrames = numFrames;
+ this.minRecordSize = minRecordSize;
+ this.maxRecordSize = maxRecordSize;
+ this.maxFrameSize = pageSize;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ result.clear();
+ int maxTupleSize = prepareSortedData(numFrames * pageSize, minRecordSize, maxRecordSize, null, result);
+ maxFrameSize = FrameHelper.calcAlignedFrameSizeToStore(0, maxTupleSize, pageSize);
+ iterator = result.entrySet().iterator();
+ }
+
+ @Override
+ public boolean nextFrame(IFrame frame) throws HyracksDataException {
+ if (lastEntry == null && !iterator.hasNext()) {
+ return false;
+ }
+ if (lastEntry == null) {
+ lastEntry = iterator.next();
+ }
+ appender.reset(frame, true);
+ while (true) {
+ tb.reset();
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, lastEntry.getKey());
+ tb.addField(UTF8StringSerializerDeserializer.INSTANCE, lastEntry.getValue());
+ if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+ break;
+ } else {
+ if (iterator.hasNext()) {
+ lastEntry = iterator.next();
+ } else {
+ lastEntry = null;
+ break;
+ }
+
+ }
+ }
+ // printFrame(frame.getBuffer());
+ return true;
+ }
+
+ private void printFrame(ByteBuffer buffer) {
+ FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+ fta.reset(buffer);
+ fta.prettyPrint();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ }
+ }
+
+ static int prepareSortedData(int minDataSize, int minRecordSize, int maxRecordSize,
+ Map<Integer, String> specialData, Map<Integer, String> result) throws HyracksDataException {
+
+ ArrayTupleBuilder tb = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+
+ int datasize = 0;
+ int maxtuple = 0;
+ if (specialData != null) {
+ for (Map.Entry<Integer, String> entry : specialData.entrySet()) {
+ tb.reset();
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, entry.getKey());
+ tb.addField(UTF8StringSerializerDeserializer.INSTANCE, entry.getValue());
+ int size = tb.getSize() + tb.getFieldEndOffsets().length * 4;
+ datasize += size;
+ if (size > maxtuple) {
+ maxtuple = size;
+ }
+ }
+ result.putAll(specialData);
+ }
+
+ while (datasize < minDataSize) {
+ String value = generateRandomRecord(minRecordSize, maxRecordSize);
+ tb.reset();
+ int key = GRandom.nextInt(datasize + 1);
+ if (!result.containsKey(key)) {
+ tb.addField(IntegerSerializerDeserializer.INSTANCE, key);
+ tb.addField(UTF8StringSerializerDeserializer.INSTANCE, value);
+ int size = tb.getSize() + tb.getFieldEndOffsets().length * 4;
+ datasize += size;
+ if (size > maxtuple) {
+ maxtuple = size;
+ }
+ if (datasize < minDataSize) {
+ result.put(key, value);
+ }
+ }
+ }
+
+ return maxtuple;
+ }
+
+ @Test
+ public void testOnlyOneRunShouldMerge() throws HyracksDataException {
+ int pageSize = 128;
+ int numRuns = 1;
+ int numFramesPerRun = 1;
+ int minRecordSize = pageSize / 10;
+ int maxRecordSize = pageSize / 8;
+
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+ List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+ List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+ List<IFrame> frameList = new ArrayList<>(numRuns);
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+ RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields, Comparators,
+ null, RecordDesc);
+ testMergeSucceed(ctx, reader, keyValueMapList);
+ }
+
+ @Test
+ public void testNormalRunMerge() throws HyracksDataException {
+
+ int pageSize = 128;
+ int numRuns = 2;
+ int numFramesPerRun = 2;
+ int minRecordSize = pageSize / 10;
+ int maxRecordSize = pageSize / 8;
+
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+ List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+ List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+ List<IFrame> frameList = new ArrayList<>(numRuns);
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+ RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields, Comparators,
+ null, RecordDesc);
+ testMergeSucceed(ctx, reader, keyValueMapList);
+ }
+
+ @Test
+ public void testNormalRunMergeWithTopK() throws HyracksDataException {
+
+ int pageSize = 128;
+ int numRuns = 2;
+ int numFramesPerRun = 2;
+ int minRecordSize = pageSize / 10;
+ int maxRecordSize = pageSize / 8;
+
+ for (int topK = 1; topK < pageSize * numRuns * numFramesPerRun / maxRecordSize / 2; topK++) {
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+ List<Map<Integer, String>> keyValueMapList = new ArrayList<>(numRuns);
+ List<TestFrameReader> readerList = new ArrayList<>(numRuns);
+ List<IFrame> frameList = new ArrayList<>(numRuns);
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMapList);
+
+ RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields,
+ Comparators,
+ null, RecordDesc, topK);
+ int totoalCount = testMergeSucceedInner(ctx, reader, keyValueMapList);
+ int newCount = 0;
+ for (Map<Integer, String> x : keyValueMapList) {
+ newCount += x.size();
+ }
+ assertEquals(topK + newCount, totoalCount);
+ }
+ }
+
+ private void testMergeSucceed(IHyracksTaskContext ctx, RunMergingFrameReader reader,
+ List<Map<Integer, String>> keyValueMapList) throws HyracksDataException {
+
+ testMergeSucceedInner(ctx, reader, keyValueMapList);
+ assertAllKeyValueIsConsumed(keyValueMapList);
+ reader.close();
+ }
+
+ private int testMergeSucceedInner(IHyracksTaskContext ctx, RunMergingFrameReader reader,
+ List<Map<Integer, String>> keyValueMapList) throws HyracksDataException {
+
+ IFrame frame = new VSizeFrame(ctx);
+ reader.open();
+ int count = 0;
+ for (int i = 0; i < keyValueMapList.size(); i++) {
+ keyValueMapList.set(i, new TreeMap<>(keyValueMapList.get(i)));
+ count += keyValueMapList.get(i).size();
+ }
+ while (reader.nextFrame(frame)) {
+ assertFrameIsSorted(frame, keyValueMapList);
+ }
+ return count;
+ }
+
+ @Test
+ public void testOneLargeRunMerge() throws HyracksDataException {
+ int pageSize = 64;
+ int numRuns = 2;
+ int numFramesPerRun = 1;
+ int minRecordSize = pageSize / 10;
+ int maxRecordSize = pageSize / 8;
+
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+ List<Map<Integer, String>> keyValueMap = new ArrayList<>();
+ List<TestFrameReader> readerList = new ArrayList<>();
+ List<IFrame> frameList = new ArrayList<>();
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+ minRecordSize = pageSize;
+ maxRecordSize = pageSize;
+ numFramesPerRun = 4;
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+ minRecordSize = pageSize * 2;
+ maxRecordSize = pageSize * 2;
+ numFramesPerRun = 6;
+ prepareRandomInputRunList(ctx, pageSize, numRuns, numFramesPerRun,
+ minRecordSize, maxRecordSize, readerList, frameList, keyValueMap);
+
+ RunMergingFrameReader reader = new RunMergingFrameReader(ctx, readerList, frameList, SortFields,
+ Comparators,
+ null,
+ RecordDesc);
+ testMergeSucceed(ctx, reader, keyValueMap);
+ }
+
+ @Test
+ public void testRunFileReader() throws HyracksDataException {
+ int pageSize = 128;
+ int numRuns = 4;
+ int numFramesPerRun = 4;
+ int minRecordSize = pageSize / 10;
+ int maxRecordSize = pageSize / 2;
+
+ IHyracksTaskContext ctx = testUtils.create(pageSize);
+ ExternalSortRunGenerator runGenerator = new ExternalSortRunGenerator(ctx, SortFields,
+ null, ComparatorFactories, RecordDesc, Algorithm.MERGE_SORT,
+ numFramesPerRun);
+
+ runGenerator.open();
+ Map<Integer, String> keyValuePair = new HashMap<>();
+ List<IFrame> frameList = new ArrayList<>();
+ prepareData(ctx, frameList, pageSize * numFramesPerRun * numRuns, minRecordSize, maxRecordSize,
+ null, keyValuePair);
+ for (IFrame frame : frameList) {
+ runGenerator.nextFrame(frame.getBuffer());
+ }
+
+ numFramesPerRun = 2;
+ minRecordSize = pageSize;
+ maxRecordSize = pageSize;
+ frameList.clear();
+ prepareData(ctx, frameList, pageSize * numFramesPerRun * numRuns, minRecordSize, maxRecordSize,
+ null, keyValuePair);
+ for (IFrame frame : frameList) {
+ runGenerator.nextFrame(frame.getBuffer());
+ }
+
+ runGenerator.close();
+ List<IFrame> inFrame = new ArrayList<>(runGenerator.getRuns().size());
+ for (RunAndMaxFrameSizePair max : runGenerator.getRuns()) {
+ inFrame.add(new GroupVSizeFrame(ctx, max.maxFrameSize));
+ }
+ matchResult(ctx, runGenerator.getRuns(), keyValuePair);
+ List<IFrameReader> runs = new ArrayList<>();
+ for (RunAndMaxFrameSizePair run : runGenerator.getRuns()) {
+ runs.add(run.run);
+ }
+ RunMergingFrameReader reader = new RunMergingFrameReader(ctx, runs, inFrame, SortFields, Comparators, null,
+ RecordDesc);
+
+ IFrame outFrame = new VSizeFrame(ctx);
+ reader.open();
+ while (reader.nextFrame(outFrame)) {
+ assertFrameIsSorted(outFrame, Arrays.asList(keyValuePair));
+ }
+ reader.close();
+ assertAllKeyValueIsConsumed(Arrays.asList(keyValuePair));
+ }
+
+ private void assertAllKeyValueIsConsumed(List<Map<Integer, String>> keyValueMapList) {
+ for (Map<Integer, String> map : keyValueMapList) {
+ assertTrue(map.isEmpty());
+ }
+ }
+
+ private void assertFrameIsSorted(IFrame frame, List<Map<Integer, String>> keyValueMapList)
+ throws HyracksDataException {
+ FrameTupleAccessor fta = new FrameTupleAccessor(RecordDesc);
+
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+
+ fta.reset(frame.getBuffer());
+ // fta.prettyPrint();
+ int preKey = Integer.MIN_VALUE;
+ for (int i = 0; i < fta.getTupleCount(); i++) {
+ bbis.setByteBuffer(fta.getBuffer(),
+ fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 0) + fta.getFieldSlotsLength());
+ int key = (int) RecordDesc.getFields()[0].deserialize(di);
+ bbis.setByteBuffer(fta.getBuffer(),
+ fta.getTupleStartOffset(i) + fta.getFieldStartOffset(i, 1) + fta.getFieldSlotsLength());
+ String value = (String) RecordDesc.getFields()[1].deserialize(di);
+
+ boolean found = false;
+ for (Map<Integer, String> map : keyValueMapList) {
+ if (map.containsKey(key) && map.get(key).equals(value)) {
+ found = true;
+ map.remove(key);
+ break;
+ }
+ }
+ assertTrue(found);
+ assertTrue(preKey <= key);
+ preKey = key;
+ }
+ }
+
+ static void prepareRandomInputRunList(IHyracksTaskContext ctx, int pageSize, int numRuns,
+ int numFramesPerRun, int minRecordSize, int maxRecordSize,
+ List<TestFrameReader> readerList, List<IFrame> frameList, List<Map<Integer, String>> keyValueMap)
+ throws HyracksDataException {
+ for (int i = 0; i < numRuns; i++) {
+ readerList.add(new TestFrameReader(pageSize, numFramesPerRun, minRecordSize, maxRecordSize));
+ frameList.add(new VSizeFrame(ctx, readerList.get(readerList.size() - 1).maxFrameSize));
+ keyValueMap.add(readerList.get(readerList.size() - 1).result);
+ }
+ }
+
+}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java
new file mode 100644
index 0000000..ae0397b
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/unit/TopKRunGeneratorTest.java
@@ -0,0 +1,208 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.tests.unit;
+
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.ComparatorFactories;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.RecordDesc;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.SerDers;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.SortFields;
+import static edu.uci.ics.hyracks.tests.unit.AbstractRunGeneratorTest.assertFTADataIsSorted;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.junit.Test;
+
+import edu.uci.ics.hyracks.api.comm.FixedSizeFrame;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HybridTopKSortRunGenerator;
+import edu.uci.ics.hyracks.dataflow.std.sort.HeapSortRunGenerator;
+
+public class TopKRunGeneratorTest {
+
+ static final int PAGE_SIZE = 512;
+ static final int NUM_PAGES = 80;
+ static final int SORT_FRAME_LIMIT = 4;
+
+ enum ORDER {
+ INORDER,
+ REVERSE
+ }
+
+ public class InMemorySortDataValidator implements IFrameWriter {
+
+ InMemorySortDataValidator(Map<Integer, String> answer) {
+ this.answer = answer;
+ }
+
+ Map<Integer, String> answer;
+ FrameTupleAccessor accessor;
+ int preKey = Integer.MIN_VALUE;
+
+ @Override
+ public void open() throws HyracksDataException {
+ accessor = new FrameTupleAccessor(RecordDesc);
+ preKey = Integer.MIN_VALUE;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ preKey = assertFTADataIsSorted(accessor, answer, preKey);
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ assertTrue(answer.isEmpty());
+ }
+ }
+
+ @Test
+ public void testReverseOrderedDataShouldNotGenerateAnyRuns() throws HyracksDataException {
+ int topK = 1;
+ IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+ HeapSortRunGenerator sorter = new HeapSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+ SortFields, null, ComparatorFactories, RecordDesc);
+
+ testInMemoryOnly(ctx, topK, ORDER.REVERSE, sorter);
+ }
+
+ @Test
+ public void testAlreadySortedDataShouldNotGenerateAnyRuns() throws HyracksDataException {
+ int topK = SORT_FRAME_LIMIT;
+ IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+ HeapSortRunGenerator sorter = new HeapSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+ SortFields, null, ComparatorFactories, RecordDesc);
+
+ testInMemoryOnly(ctx, topK, ORDER.INORDER, sorter);
+ }
+
+ @Test
+ public void testHybridTopKShouldNotGenerateAnyRuns() throws HyracksDataException {
+ int topK = 1;
+ IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+ AbstractSortRunGenerator sorter = new HybridTopKSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+ SortFields, null, ComparatorFactories, RecordDesc);
+
+ testInMemoryOnly(ctx, topK, ORDER.REVERSE, sorter);
+ }
+
+ @Test
+ public void testHybridTopKShouldSwitchToFrameSorterWhenFlushed() {
+ int topK = 1;
+ IHyracksTaskContext ctx = AbstractRunGeneratorTest.testUtils.create(PAGE_SIZE);
+ AbstractSortRunGenerator sorter = new HybridTopKSortRunGenerator(ctx, SORT_FRAME_LIMIT, topK,
+ SortFields, null, ComparatorFactories, RecordDesc);
+
+ }
+
+ private void testInMemoryOnly(IHyracksTaskContext ctx, int topK, ORDER order, AbstractSortRunGenerator sorter)
+ throws HyracksDataException {
+ Map<Integer, String> keyValuePair = null;
+ switch (order) {
+ case INORDER:
+ keyValuePair = new TreeMap<>();
+ break;
+ case REVERSE:
+ keyValuePair = new TreeMap<>(Collections.reverseOrder());
+ break;
+ }
+
+ List<IFrame> frameList = new ArrayList<>();
+ int minDataSize = PAGE_SIZE * NUM_PAGES * 4 / 5;
+ int minRecordSize = 16;
+ int maxRecordSize = 64;
+
+ AbstractRunGeneratorTest
+ .prepareData(ctx, frameList, minDataSize, minRecordSize, maxRecordSize, null, keyValuePair);
+
+ assert topK > 0;
+
+ ByteBuffer buffer = prepareSortedData(keyValuePair);
+
+ Map<Integer, String> topKAnswer = getTopKAnswer(keyValuePair, topK);
+
+ doSort(sorter, buffer);
+
+ assertEquals(0, sorter.getRuns().size());
+ validateResult(sorter, topKAnswer);
+ }
+
+ private void validateResult(AbstractSortRunGenerator sorter, Map<Integer, String> topKAnswer)
+ throws HyracksDataException {
+
+ InMemorySortDataValidator validator = new InMemorySortDataValidator(topKAnswer);
+ validator.open();
+ sorter.getSorter().flush(validator);
+ validator.close();
+ }
+
+ private void doSort(AbstractSortRunGenerator sorter, ByteBuffer buffer) throws HyracksDataException {
+
+ sorter.open();
+ sorter.nextFrame(buffer);
+ sorter.close();
+ }
+
+ private Map<Integer, String> getTopKAnswer(Map<Integer, String> keyValuePair, int topK) {
+
+ TreeMap<Integer, String> copy = new TreeMap<>(keyValuePair);
+
+ Map<Integer, String> answer = new TreeMap<>();
+ for (Map.Entry<Integer, String> entry : copy.entrySet()) {
+ if (answer.size() < topK) {
+ answer.put(entry.getKey(), entry.getValue());
+ } else {
+ break;
+ }
+ }
+ return answer;
+ }
+
+ private ByteBuffer prepareSortedData(Map<Integer, String> keyValuePair) throws HyracksDataException {
+ ByteBuffer buffer = ByteBuffer.allocate(PAGE_SIZE * NUM_PAGES);
+ IFrame inputFrame = new FixedSizeFrame(buffer);
+ FrameTupleAppender appender = new FrameTupleAppender();
+ appender.reset(inputFrame, true);
+ ArrayTupleBuilder builder = new ArrayTupleBuilder(RecordDesc.getFieldCount());
+
+ for (Map.Entry<Integer, String> entry : keyValuePair.entrySet()) {
+ builder.reset();
+ builder.addField(SerDers[0], entry.getKey());
+ builder.addField(SerDers[1], entry.getValue());
+ appender.append(builder.getFieldEndOffsets(), builder.getByteArray(), 0, builder.getSize());
+ }
+ return buffer;
+ }
+}
diff --git a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
index e46e685..790552c 100644
--- a/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks/hyracks-examples/text-example/texthelper/src/main/java/edu/uci/ics/hyracks/examples/text/WordTupleParserFactory.java
@@ -19,10 +19,10 @@
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.Reader;
-import java.nio.ByteBuffer;
import java.util.Arrays;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -42,9 +42,7 @@
@Override
public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
try {
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(frame, true);
+ FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
DataOutput dos = tb.getDataOutput();
@@ -54,17 +52,10 @@
tb.reset();
utf8StringParser.parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
tb.addFieldEndOffset();
- 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 HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
+ appender.flush(writer, true);
} catch (IOException e) {
throw new HyracksDataException(e);
}
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index a437f37..e22f27f 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -56,7 +56,7 @@
<configuration>
<programs>
<program>
- <mainClass>edu.uci.ics.hyracks.examples.tpch.client.Main</mainClass>
+ <mainClass>edu.uci.ics.hyracks.examples.tpch.client.Sort</mainClass>
<name>tpchclient</name>
</program>
</programs>
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java
new file mode 100644
index 0000000..17f1d3b
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Common.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import java.io.File;
+
+import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+
+public class Common {
+ static RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
+
+ static RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
+ static RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
+
+ static IValueParserFactory[] orderParserFactories = new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE };
+
+
+ static FileSplit[] parseFileSplits(String fileSplits) {
+ String[] splits = fileSplits.split(",");
+ FileSplit[] fSplits = new FileSplit[splits.length];
+ for (int i = 0; i < splits.length; ++i) {
+ String s = splits[i].trim();
+ int idx = s.indexOf(':');
+ if (idx < 0) {
+ throw new IllegalArgumentException("File split " + s + " not well formed");
+ }
+ fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
+ }
+ return fSplits;
+ }
+
+ static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
+ String[] parts = new String[splits.length];
+ for (int i = 0; i < splits.length; ++i) {
+ parts[i] = splits[i].getNodeName();
+ }
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
+ }
+}
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
similarity index 82%
rename from hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
rename to hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
index 1d4e6ce..748d809 100644
--- a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Join.java
@@ -14,7 +14,8 @@
*/
package edu.uci.ics.hyracks.examples.tpch.client;
-import java.io.File;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.*;
+
import java.util.EnumSet;
import org.kohsuke.args4j.CmdLineParser;
@@ -35,7 +36,6 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -64,7 +64,7 @@
import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
-public class Main {
+public class Join {
private static class Options {
@Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
public String host;
@@ -134,20 +134,6 @@
System.err.println(start + " " + end + " " + (end - start));
}
- private static FileSplit[] parseFileSplits(String fileSplits) {
- String[] splits = fileSplits.split(",");
- FileSplit[] fSplits = new FileSplit[splits.length];
- for (int i = 0; i < splits.length; ++i) {
- String s = splits[i].trim();
- int idx = s.indexOf(':');
- if (idx < 0) {
- throw new IllegalArgumentException("File split " + s + " not well formed");
- }
- fSplits[i] = new FileSplit(s.substring(0, idx), new FileReference(new File(s.substring(idx + 1))));
- }
- return fSplits;
- }
-
private static JobSpecification createJob(FileSplit[] customerSplits, FileSplit[] orderSplits,
FileSplit[] resultSplits, int numJoinPartitions, String algo, int graceInputSize, int graceRecordsPerFrame,
double graceFactor, int memSize, int tableSize, boolean hasGroupBy, int frameSize)
@@ -155,37 +141,15 @@
JobSpecification spec = new JobSpecification(frameSize);
IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(customerSplits);
- RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(orderSplits);
- RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
-
- RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider,
new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), Common.ordersDesc);
createPartitionConstraint(spec, ordScanner, orderSplits);
FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider,
@@ -193,14 +157,14 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE }, '|'), custDesc);
+ UTF8StringParserFactory.INSTANCE }, '|'), Common.custDesc);
createPartitionConstraint(spec, custScanner, customerSplits);
IOperatorDescriptor join;
if ("nestedloop".equalsIgnoreCase(algo)) {
join = new NestedLoopJoinOperatorDescriptor(spec, new JoinComparatorFactory(
- PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), custOrderJoinDesc,
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), 0, 1), Common.custOrderJoinDesc,
memSize, false, null);
} else if ("gracehash".equalsIgnoreCase(algo)) {
@@ -215,7 +179,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- custOrderJoinDesc, null);
+ Common.custOrderJoinDesc, null);
} else if ("hybridhash".equalsIgnoreCase(algo)) {
join = new HybridHashJoinOperatorDescriptor(
@@ -229,7 +193,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- custOrderJoinDesc, null);
+ Common.custOrderJoinDesc, null);
} else {
join = new InMemoryHashJoinOperatorDescriptor(
@@ -239,7 +203,7 @@
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- custOrderJoinDesc, 6000000, null);
+ Common.custOrderJoinDesc, 6000000, null);
}
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, numJoinPartitions);
@@ -295,13 +259,7 @@
return spec;
}
- private static void createPartitionConstraint(JobSpecification spec, IOperatorDescriptor op, FileSplit[] splits) {
- String[] parts = new String[splits.length];
- for (int i = 0; i < splits.length; ++i) {
- parts[i] = splits[i].getNodeName();
- }
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, op, parts);
- }
+
static class JoinComparatorFactory implements ITuplePairComparatorFactory {
private static final long serialVersionUID = 1L;
diff --git a/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java
new file mode 100644
index 0000000..7570b0b
--- /dev/null
+++ b/hyracks/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Sort.java
@@ -0,0 +1,165 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.hyracks.examples.tpch.client;
+
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.createPartitionConstraint;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.orderParserFactories;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.ordersDesc;
+import static edu.uci.ics.hyracks.examples.tpch.client.Common.parseFileSplits;
+
+import java.util.EnumSet;
+
+import org.kohsuke.args4j.CmdLineParser;
+import org.kohsuke.args4j.Option;
+
+import edu.uci.ics.hyracks.api.client.HyracksConnection;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.api.job.JobFlag;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory;
+import edu.uci.ics.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.AbstractSorterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.Algorithm;
+import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.TopKSorterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.sort.buffermanager.EnumFreeSlotPolicy;
+
+public class Sort {
+ private static class Options {
+ @Option(name = "-host", usage = "Hyracks Cluster Controller Host name", required = true)
+ public String host;
+
+ @Option(name = "-port", usage = "Hyracks Cluster Controller Port (default: 1098)", required = false)
+ public int port = 1098;
+
+ @Option(name = "-frame-size", usage = "Hyracks frame size (default: 32768)", required = false)
+ public int frameSize = 32768;
+
+ @Option(name = "-frame-limit", usage = "memory limit for sorting (default: 4)", required = false)
+ public int frameLimit = 4;
+
+ @Option(name = "-infile-splits", usage = "Comma separated list of file-splits for the ORDER input. A file-split is <node-name>:<path>", required = true)
+ public String inFileOrderSplits;
+
+ @Option(name = "-outfile-splits", usage = "Comma separated list of file-splits for the output", required = true)
+ public String outFileSplits;
+
+ @Option(name = "-membuffer-alg", usage = "bestfit or lastfit (default: lastfit)", required = false)
+ public String memBufferAlg = "lastfit";
+
+ @Option(name = "-profile", usage = "Enable/Disable profiling. (default: enabled)")
+ public boolean profile = true;
+
+ @Option(name = "-topK", usage = "only output topK for each node. (default: not set)")
+ public int topK = Integer.MAX_VALUE;
+
+ @Option(name = "-heapSort", usage = "using heap sort for topK result. (default: false)")
+ public boolean usingHeapSorter = false;
+ }
+
+ static int[] SortFields = new int[] { 1, 0 };
+ static IBinaryComparatorFactory[] SortFieldsComparatorFactories = new IBinaryComparatorFactory[] {
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) };
+
+ static IBinaryHashFunctionFactory[] orderBinaryHashFunctionFactories = new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) };
+
+ public static void main(String[] args) throws Exception {
+ Options options = new Options();
+ CmdLineParser parser = new CmdLineParser(options);
+ if (args.length == 0) {
+ parser.printUsage(System.err);
+ return;
+ }
+ parser.parseArgument(args);
+
+ IHyracksClientConnection hcc = new HyracksConnection(options.host, options.port);
+
+ JobSpecification job = createJob(parseFileSplits(options.inFileOrderSplits),
+ parseFileSplits(options.outFileSplits),
+ options.memBufferAlg, options.frameLimit, options.frameSize, options.topK, options.usingHeapSorter);
+
+ long start = System.currentTimeMillis();
+ JobId jobId = hcc.startJob(job,
+ options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
+ hcc.waitForCompletion(jobId);
+ long end = System.currentTimeMillis();
+ System.err.println("finished in:" + (end - start) + "ms");
+ }
+
+ private static JobSpecification createJob(FileSplit[] ordersSplits, FileSplit[] outputSplit, String memBufferAlg,
+ int frameLimit, int frameSize, int limit, boolean usingHeapSorter) {
+ JobSpecification spec = new JobSpecification();
+
+ spec.setFrameSize(frameSize);
+ IFileSplitProvider ordersSplitProvider = new ConstantFileSplitProvider(ordersSplits);
+ FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitProvider,
+ new DelimitedDataTupleParserFactory(orderParserFactories, '|'), ordersDesc);
+ createPartitionConstraint(spec, ordScanner, ordersSplits);
+ AbstractSorterOperatorDescriptor sorter;
+ if (usingHeapSorter && limit < Integer.MAX_VALUE) {
+ sorter = new TopKSorterOperatorDescriptor(spec, frameLimit, limit, SortFields, null,
+ SortFieldsComparatorFactories, ordersDesc);
+ } else {
+ if (memBufferAlg.equalsIgnoreCase("bestfit")) {
+ sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields,
+ null, SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT,
+ EnumFreeSlotPolicy.SMALLEST_FIT, limit);
+ } else if (memBufferAlg.equalsIgnoreCase("biggestfit")) {
+ sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields, null,
+ SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT, EnumFreeSlotPolicy.BIGGEST_FIT,
+ limit);
+ } else {
+ sorter = new ExternalSortOperatorDescriptor(spec, frameLimit, SortFields, null,
+ SortFieldsComparatorFactories, ordersDesc, Algorithm.MERGE_SORT, EnumFreeSlotPolicy.LAST_FIT,
+ limit);
+
+ }
+ }
+ createPartitionConstraint(spec, sorter, ordersSplits);
+ IFileSplitProvider outputSplitProvider = new ConstantFileSplitProvider(outputSplit);
+ IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outputSplitProvider, "|");
+ createPartitionConstraint(spec, printer, outputSplit);
+
+ spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
+
+ spec.connect(
+ new MToNPartitioningMergingConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(SortFields, orderBinaryHashFunctionFactories),
+ SortFields, SortFieldsComparatorFactories, new UTF8StringNormalizedKeyComputerFactory()),
+ sorter, 0, printer, 0);
+
+ spec.addRoot(printer);
+ return spec;
+ }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
index 4e48e9b..cb1ca87 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/dataflow/HDFSWriteOperatorDescriptor.java
@@ -76,7 +76,7 @@
private FSDataOutputStream dos;
private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
- private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+ private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
private FrameTupleReference tuple = new FrameTupleReference();
private ITupleWriter tupleWriter;
private ClassLoader ctxCL;
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
index 92cde9d..62cd76a 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs/lib/TextKeyValueParserFactory.java
@@ -15,12 +15,11 @@
package edu.uci.ics.hyracks.hdfs.lib;
-import java.nio.ByteBuffer;
-
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -37,9 +36,7 @@
throws HyracksDataException {
final ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
- final ByteBuffer buffer = ctx.allocateFrame();
- final FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(buffer, true);
+ final FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
return new IKeyValueParser<LongWritable, Text>() {
@@ -53,18 +50,13 @@
throws HyracksDataException {
tb.reset();
tb.addField(value.getBytes(), 0, value.getLength());
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- FrameUtils.flushFrame(buffer, writer);
- appender.reset(buffer, true);
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("tuple cannot be appended into the frame");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
@Override
public void close(IFrameWriter writer) throws HyracksDataException {
- FrameUtils.flushFrame(buffer, writer);
+ appender.flush(writer, false);
}
};
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
index 068cdfc..0d24ee5 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/main/java/edu/uci/ics/hyracks/hdfs2/dataflow/HDFSWriteOperatorDescriptor.java
@@ -75,7 +75,7 @@
private FSDataOutputStream dos;
private RecordDescriptor inputRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);;
- private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRd);
+ private FrameTupleAccessor accessor = new FrameTupleAccessor(inputRd);
private FrameTupleReference tuple = new FrameTupleReference();
private ITupleWriter tupleWriter;
private ClassLoader ctxCL;
diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index e60026e..785258f 100644
--- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -17,7 +17,6 @@
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.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
@@ -64,14 +63,8 @@
dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.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 HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
}
}
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
index 9f3c5dd..9aec4cb 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexBulkLoadOperatorNodePushable.java
@@ -66,7 +66,7 @@
public void open() throws HyracksDataException {
RecordDescriptor recDesc = recDescProvider.getInputRecordDescriptor(
opDesc.getActivityId(), 0);
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(recDesc);
indexHelper.open();
index = indexHelper.getIndexInstance();
try {
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index 092fada..48a395b 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -16,6 +16,8 @@
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.IFrame;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -45,7 +47,7 @@
protected final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
protected FrameTupleAccessor accessor;
protected FrameTupleReference frameTuple;
- protected ByteBuffer writeBuffer;
+ protected IFrame writeBuffer;
protected IIndexAccessor indexAccessor;
protected ITupleFilter tupleFilter;
protected IModificationOperationCallback modCallback;
@@ -63,8 +65,8 @@
@Override
public void open() throws HyracksDataException {
RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
- writeBuffer = ctx.allocateFrame();
+ accessor = new FrameTupleAccessor(inputRecDesc);
+ writeBuffer = new VSizeFrame(ctx);
writer.open();
indexHelper.open();
IIndex index = indexHelper.getIndexInstance();
@@ -134,8 +136,9 @@
}
}
// Pass a copy of the frame to next op.
- System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
- FrameUtils.flushFrame(writeBuffer, writer);
+ writeBuffer.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+ FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
}
@Override
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index a3f4e6f..fd727cf 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -18,6 +18,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
@@ -45,7 +46,6 @@
protected final IIndexDataflowHelper indexHelper;
protected FrameTupleAccessor accessor;
- protected ByteBuffer writeBuffer;
protected FrameTupleAppender appender;
protected ArrayTupleBuilder tb;
protected DataOutput dos;
@@ -103,7 +103,7 @@
@Override
public void open() throws HyracksDataException {
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
+ accessor = new FrameTupleAccessor(inputRecDesc);
writer.open();
indexHelper.open();
index = indexHelper.getIndexInstance();
@@ -126,11 +126,9 @@
try {
searchPred = createSearchPredicate();
- writeBuffer = ctx.allocateFrame();
tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
dos = tb.getDataOutput();
- appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(writeBuffer, true);
+ appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
.createSearchOperationCallback(indexHelper.getResourceID(), ctx);
indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
@@ -162,27 +160,13 @@
dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.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 HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
if (!matched && retainInput && retainNull) {
- if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
- nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
- FrameUtils.flushFrame(writeBuffer, writer);
- appender.reset(writeBuffer, true);
- if (!appender.appendConcat(accessor, tupleIndex, nullTupleBuild.getFieldEndOffsets(),
- nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize())) {
- throw new HyracksDataException("Record size larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendConcatToWriter(writer, appender, accessor, tupleIndex,
+ nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize());
}
}
@@ -205,9 +189,7 @@
@Override
public void close() throws HyracksDataException {
try {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(writeBuffer, writer);
- }
+ appender.flush(writer, true);
try {
cursor.close();
} catch (Exception e) {
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index cb6270f..0db195d 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -15,8 +15,8 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import java.io.DataOutput;
-import java.nio.ByteBuffer;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -60,9 +60,7 @@
try {
indexAccessor.diskOrderScan(cursor);
int fieldCount = treeIndex.getFieldCount();
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(frame, true);
+ FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
DataOutput dos = tb.getDataOutput();
@@ -72,21 +70,15 @@
ITupleReference frameTuple = cursor.getTuple();
for (int i = 0; i < frameTuple.getFieldCount(); i++) {
- dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(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(frame, writer);
- appender.reset(frame, true);
- if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
+ tb.getSize());
}
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(frame, writer);
- }
+ appender.flush(writer, true);
} catch (Exception e) {
writer.fail();
throw new HyracksDataException(e);
diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
index 717d326..f340142 100644
--- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorNodePushable.java
@@ -15,14 +15,13 @@
package edu.uci.ics.hyracks.storage.am.common.dataflow;
import java.io.DataOutput;
-import java.nio.ByteBuffer;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
@@ -70,18 +69,18 @@
.getInteriorFrameFactory().createFrame(), treeIndex.getFreePageManager().getMetaDataFrameFactory()
.createFrame());
// Write the stats output as a single string field.
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(frame, true);
+ FrameTupleAppender appender = new FrameTupleAppender(new VSizeFrame(ctx));
ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
DataOutput dos = tb.getDataOutput();
tb.reset();
UTF8StringSerializerDeserializer.INSTANCE.serialize(stats.toString(), dos);
tb.addFieldEndOffset();
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
- throw new HyracksDataException("Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
+ throw new HyracksDataException(
+ "Record size (" + tb.getSize() + ") larger than frame size (" + appender.getBuffer().capacity()
+ + ")");
}
- FrameUtils.flushFrame(frame, writer);
+ appender.flush(writer, false);
} catch (Exception e) {
writer.fail();
} finally {
diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index 5bf52e4..fd693cf 100644
--- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -30,6 +30,12 @@
protected FrameTupleAppender appender;
+ @Override
+ public void open() throws HyracksDataException {
+ super.open();
+ appender = new FrameTupleAppender(writeBuffer);
+ }
+
public LSMIndexInsertUpdateDeleteOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, int[] fieldPermutation, IRecordDescriptorProvider recordDescProvider, IndexOperation op) {
super(opDesc, ctx, partition, fieldPermutation, recordDescProvider, op);
@@ -85,8 +91,8 @@
break;
}
default: {
- throw new HyracksDataException("Unsupported operation " + op
- + " in tree index InsertUpdateDelete operator");
+ throw new HyracksDataException(
+ "Unsupported operation " + op + " in tree index InsertUpdateDelete operator");
}
}
} catch (HyracksDataException e) {
@@ -97,8 +103,9 @@
}
if (nextFlushTupleIndex == 0) {
// No partial flushing was necessary. Forward entire frame.
- System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
- FrameUtils.flushFrame(writeBuffer, writer);
+ writeBuffer.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+ FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
} else {
// Flush remaining partial frame.
flushPartialFrame(nextFlushTupleIndex, tupleCount);
@@ -106,17 +113,9 @@
}
private void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
- if (appender == null) {
- appender = new FrameTupleAppender(ctx.getFrameSize());
- }
- appender.reset(writeBuffer, true);
for (int i = startTupleIndex; i < endTupleIndex; i++) {
- if (!appender.append(accessor, i)) {
- throw new HyracksDataException("Record size ("
- + (accessor.getTupleEndOffset(i) - accessor.getTupleStartOffset(i))
- + ") larger than frame size (" + appender.getBuffer().capacity() + ")");
- }
+ FrameUtils.appendToWriter(writer, appender, accessor, i);
}
- FrameUtils.flushFrame(writeBuffer, writer);
+ appender.flush(writer, true);
}
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 8598b70..231adbd 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -15,11 +15,10 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
-import java.io.DataInputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -27,9 +26,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.util.IntSerDeUtils;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
@@ -50,7 +47,6 @@
private ArrayTupleBuilder builder;
private GrowableArray builderData;
private FrameTupleAppender appender;
- private ByteBuffer writeBuffer;
public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx,
RecordDescriptor inputRecDesc, RecordDescriptor outputRecDesc,
@@ -68,12 +64,10 @@
@Override
public void open() throws HyracksDataException {
- accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRecDesc);
- writeBuffer = ctx.allocateFrame();
+ accessor = new FrameTupleAccessor(inputRecDesc);
builder = new ArrayTupleBuilder(outputRecDesc.getFieldCount());
builderData = builder.getFieldData();
- appender = new FrameTupleAppender(ctx.getFrameSize());
- appender.reset(writeBuffer, true);
+ appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
writer.open();
}
@@ -157,19 +151,8 @@
}
- if (!appender.append(builder.getFieldEndOffsets(),
- builder.getByteArray(), 0, builder.getSize())) {
- FrameUtils.flushFrame(writeBuffer, writer);
- appender.reset(writeBuffer, true);
-
- if (!appender.append(builder.getFieldEndOffsets(),
- builder.getByteArray(), 0, builder.getSize())) {
- throw new HyracksDataException("Record size ("
- + builder.getSize()
- + ") larger than frame size ("
- + appender.getBuffer().capacity() + ")");
- }
- }
+ FrameUtils.appendToWriter(writer, appender, builder.getFieldEndOffsets(),
+ builder.getByteArray(), 0, builder.getSize());
}
@@ -179,9 +162,7 @@
@Override
public void close() throws HyracksDataException {
- if (appender.getTupleCount() > 0) {
- FrameUtils.flushFrame(writeBuffer, writer);
- }
+ appender.flush(writer, true);
writer.close();
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index ddaeac0..95ec645 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -15,8 +15,10 @@
package edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow;
+import edu.uci.ics.hyracks.api.comm.IFrame;
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.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexOperatorDescriptor;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
index 59690c3..3c6c4cd 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeFrameTupleAccessor.java
@@ -67,6 +67,11 @@
}
@Override
+ public int getTupleLength(int tupleIndex) {
+ return getTupleEndOffset(tupleIndex) - getTupleStartOffset(tupleIndex);
+ }
+
+ @Override
public int getFieldSlotsLength() {
return 0;
}
@@ -92,6 +97,11 @@
}
@Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
public void reset(ByteBuffer buffer) {
this.buffer = buffer;
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 1aded5f..a479815 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -549,7 +549,7 @@
private final int FRAME_SIZE = 32768;
@Override
- public int getFrameSize() {
+ public int getInitialFrameSize() {
return FRAME_SIZE;
}
@@ -563,8 +563,16 @@
return ByteBuffer.allocate(FRAME_SIZE);
}
+ @Override public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ return ByteBuffer.allocate(bytes);
+ }
+
+ @Override public ByteBuffer reallocateFrame(ByteBuffer bytes, int newSizeInBytes, boolean copyOldData) throws HyracksDataException {
+ throw new HyracksDataException("TODO");
+ }
+
@Override
- public void deallocateFrames(int frameCount) {
+ public void deallocateFrames(int bytes) {
// TODO Auto-generated method stub
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index f3b019f..5b68c7b 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -19,18 +19,18 @@
import java.nio.ByteBuffer;
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppenderAccessor;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
@@ -59,9 +59,8 @@
protected final MultiComparator invListCmp;
protected final ArrayTupleBuilder queryTokenBuilder = new ArrayTupleBuilder(QUERY_TOKEN_REC_DESC.getFieldCount());
- protected final ByteBuffer queryTokenFrame;
- protected final FrameTupleAppender queryTokenAppender;
- protected final FrameTupleAccessor queryTokenAccessor;
+ protected final IFrame queryTokenFrame;
+ protected final FrameTupleAppenderAccessor queryTokenAppender;
protected final FrameTupleReference searchKey = new FrameTupleReference();
protected int occurrenceThreshold;
@@ -78,10 +77,9 @@
this.invListCursorFactory = new InvertedListCursorFactory(invIndex);
this.invListCursorCache = new ObjectCache<IInvertedListCursor>(invListCursorFactory, OBJECT_CACHE_INIT_SIZE,
OBJECT_CACHE_EXPAND_SIZE);
- this.queryTokenFrame = ctx.allocateFrame();
- this.queryTokenAppender = new FrameTupleAppender(ctx.getFrameSize());
- this.queryTokenAccessor = new FrameTupleAccessor(ctx.getFrameSize(), QUERY_TOKEN_REC_DESC);
- this.queryTokenAccessor.reset(queryTokenFrame);
+ this.queryTokenFrame = new VSizeFrame(ctx);
+ this.queryTokenAppender = new FrameTupleAppenderAccessor(QUERY_TOKEN_REC_DESC);
+ this.queryTokenAppender.reset(queryTokenFrame, true);
}
public void reset() {
@@ -116,7 +114,7 @@
}
public IFrameTupleAccessor createResultFrameTupleAccessor() {
- return new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), searchResult.getTypeTraits());
+ return new FixedSizeFrameTupleAccessor(ctx.getInitialFrameSize(), searchResult.getTypeTraits());
}
public ITupleReference createResultFrameTupleReference() {
@@ -144,8 +142,10 @@
ByteBuffer testBuf = buffer.get(i);
resultFrameTupleAcc.reset(testBuf);
for (int j = 0; j < resultFrameTupleAcc.getTupleCount(); j++) {
- strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(), resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
- strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(), resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
+ strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(),
+ resultFrameTupleAcc.getFieldStartOffset(j, 0)) + ",");
+ strBuffer.append(IntegerPointable.getInteger(resultFrameTupleAcc.getBuffer().array(),
+ resultFrameTupleAcc.getFieldStartOffset(j, 1)) + " ");
}
}
System.out.println(strBuffer.toString());
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
index b7456e6..1e68099 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/PartitionedTOccurrenceSearcher.java
@@ -24,7 +24,6 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexOperationContext;
import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
import edu.uci.ics.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
@@ -64,7 +63,7 @@
lowerBoundTupleBuilder.addFieldEndOffset();
lowerBoundTuple.reset(lowerBoundTupleBuilder.getFieldEndOffsets(), lowerBoundTupleBuilder.getByteArray());
// Only needed for setting the number of fields in searchKey.
- searchKey.reset(queryTokenAccessor, 0);
+ searchKey.reset(queryTokenAppender, 0);
fullLowSearchKey.reset();
fullLowSearchKey.addTuple(searchKey);
fullLowSearchKey.addTuple(lowerBoundTuple);
@@ -75,7 +74,7 @@
upperBoundTupleBuilder.addFieldEndOffset();
upperBoundTuple.reset(upperBoundTupleBuilder.getFieldEndOffsets(), upperBoundTupleBuilder.getByteArray());
// Only needed for setting the number of fields in searchKey.
- searchKey.reset(queryTokenAccessor, 0);
+ searchKey.reset(queryTokenAppender, 0);
fullHighSearchKey.reset();
fullHighSearchKey.addTuple(searchKey);
fullHighSearchKey.addTuple(upperBoundTuple);
@@ -93,7 +92,7 @@
}
tokenizeQuery(searchPred);
- short numQueryTokens = (short) queryTokenAccessor.getTupleCount();
+ short numQueryTokens = (short) queryTokenAppender.getTupleCount();
IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
short numTokensLowerBound = searchModifier.getNumTokensLowerBound(numQueryTokens);
@@ -109,7 +108,7 @@
partitions.reset(numTokensLowerBound, numTokensUpperBound);
cursorsOrderedByTokens.clear();
for (int i = 0; i < numQueryTokens; i++) {
- searchKey.reset(queryTokenAccessor, i);
+ searchKey.reset(queryTokenAppender, i);
if (!partInvIndex.openInvertedListPartitionCursors(this, ictx, numTokensLowerBound, numTokensUpperBound,
partitions, cursorsOrderedByTokens)) {
maxCountPossible--;
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
index c4056c5..4af9546 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/SearchResult.java
@@ -54,8 +54,8 @@
// Integer for counting occurrences.
typeTraits[invListFields.length] = IntegerPointable.TYPE_TRAITS;
this.ctx = ctx;
- appender = new FixedSizeFrameTupleAppender(ctx.getFrameSize(), typeTraits);
- accessor = new FixedSizeFrameTupleAccessor(ctx.getFrameSize(), typeTraits);
+ appender = new FixedSizeFrameTupleAppender(ctx.getInitialFrameSize(), typeTraits);
+ accessor = new FixedSizeFrameTupleAccessor(ctx.getInitialFrameSize(), typeTraits);
tuple = new FixedSizeTupleReference(typeTraits);
buffers.add(ctx.allocateFrame());
}
diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
index 348ef75..c61a86e 100644
--- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
+++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/search/TOccurrenceSearcher.java
@@ -38,12 +38,12 @@
public void search(OnDiskInvertedIndexSearchCursor resultCursor, InvertedIndexSearchPredicate searchPred,
IIndexOperationContext ictx) throws HyracksDataException, IndexException {
tokenizeQuery(searchPred);
- int numQueryTokens = queryTokenAccessor.getTupleCount();
+ int numQueryTokens = queryTokenAppender.getTupleCount();
invListCursors.clear();
invListCursorCache.reset();
for (int i = 0; i < numQueryTokens; i++) {
- searchKey.reset(queryTokenAccessor, i);
+ searchKey.reset(queryTokenAppender, i);
IInvertedListCursor invListCursor = invListCursorCache.getNext();
invIndex.openInvertedListCursor(invListCursor, searchKey, ictx);
invListCursors.add(invListCursor);
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
index e1a196a..03ff58f 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestJobletContext.java
@@ -27,10 +27,12 @@
import edu.uci.ics.hyracks.api.resources.IDeallocatable;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.memory.FrameManager;
public class TestJobletContext implements IHyracksJobletContext {
private final int frameSize;
private final INCApplicationContext appContext;
+ private final FrameManager frameManger;
private JobId jobId;
private WorkspaceFileFactory fileFactory;
@@ -39,10 +41,23 @@
this.appContext = appContext;
this.jobId = jobId;
fileFactory = new WorkspaceFileFactory(this, (IOManager) getIOManager());
+ this.frameManger = new FrameManager(frameSize);
}
- public ByteBuffer allocateFrame() {
- return ByteBuffer.allocate(frameSize);
+ ByteBuffer allocateFrame() throws HyracksDataException {
+ return frameManger.allocateFrame();
+ }
+
+ public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ return frameManger.allocateFrame(bytes);
+ }
+
+ ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newFrameSizeInBytes, boolean copyOldData) throws HyracksDataException {
+ return frameManger.reallocateFrame(tobeDeallocate, newFrameSizeInBytes, copyOldData);
+ }
+
+ void deallocateFrames(int bytes) {
+ frameManger.deallocateFrames(bytes);
}
public int getFrameSize() {
@@ -106,4 +121,5 @@
public ClassLoader getClassLoader() {
return this.getClass().getClassLoader();
}
+
}
\ No newline at end of file
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index 2e4c812..9e395ab 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -43,16 +43,29 @@
}
@Override
- public ByteBuffer allocateFrame() {
+ public ByteBuffer allocateFrame() throws HyracksDataException {
return jobletContext.allocateFrame();
}
-
- @Override
- public void deallocateFrames(int frameCount) {
- }
@Override
- public int getFrameSize() {
+ public ByteBuffer allocateFrame(int bytes) throws HyracksDataException {
+ return jobletContext.allocateFrame(bytes);
+ }
+
+ @Override
+ public ByteBuffer reallocateFrame(ByteBuffer tobeDeallocate, int newSizeInBytes, boolean copyOldData)
+ throws HyracksDataException {
+ return jobletContext.reallocateFrame(tobeDeallocate,newSizeInBytes, copyOldData);
+
+ }
+
+ @Override
+ public void deallocateFrames(int bytes) {
+ jobletContext.deallocateFrames(bytes);
+ }
+
+ @Override
+ public int getInitialFrameSize() {
return jobletContext.getFrameSize();
}
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
index 2c06c20..72e410a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -15,13 +15,14 @@
package edu.uci.ics.hyracks.storage.am.btree;
import java.io.DataOutput;
-import java.nio.ByteBuffer;
import java.util.Random;
import java.util.logging.Level;
import org.junit.Test;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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.ISerializerDeserializer;
@@ -110,16 +111,16 @@
LOGGER.info("INSERTING INTO TREE");
}
- ByteBuffer frame = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ IFrame frame = new VSizeFrame(ctx);
+ FrameTupleAppender appender = new FrameTupleAppender();
ArrayTupleBuilder tb = new ArrayTupleBuilder(fieldCount);
DataOutput dos = tb.getDataOutput();
ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE };
RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(frame);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(recDesc);
+ accessor.reset(frame.getBuffer());
FrameTupleReference tuple = new FrameTupleReference();
ITreeIndexAccessor indexAccessor = btree.createAccessor(TestOperationCallback.INSTANCE,
diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
index a86238a..49b9a8a 100644
--- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
+++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/FieldPrefixNSMTest.java
@@ -16,14 +16,15 @@
package edu.uci.ics.hyracks.storage.am.btree;
import java.io.DataOutput;
-import java.nio.ByteBuffer;
import java.util.Random;
import java.util.logging.Level;
import org.junit.Assert;
import org.junit.Test;
+import edu.uci.ics.hyracks.api.comm.IFrame;
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.comm.VSizeFrame;
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;
@@ -69,8 +70,8 @@
}
}
- ByteBuffer buf = ctx.allocateFrame();
- FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize());
+ IFrame buf = new VSizeFrame(ctx);
+ FrameTupleAppender appender = new FrameTupleAppender(buf);
ArrayTupleBuilder tb = new ArrayTupleBuilder(3);
DataOutput dos = tb.getDataOutput();
@@ -78,8 +79,8 @@
ISerializerDeserializer[] recDescSers = { IntegerSerializerDeserializer.INSTANCE,
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
RecordDescriptor recDesc = new RecordDescriptor(recDescSers);
- IFrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), recDesc);
- accessor.reset(buf);
+ IFrameTupleAccessor accessor = new FrameTupleAccessor(recDesc);
+ accessor.reset(buf.getBuffer());
FrameTupleReference tuple = new FrameTupleReference();
tb.reset();