diff --git a/stratosphere-addons/avro/src/main/java/eu/stratosphere/api/avro/AvroBaseValue.java b/stratosphere-addons/avro/src/main/java/eu/stratosphere/api/avro/AvroBaseValue.java index 7ed7ab6684c727c8ee3493efc0e7ee5ba95f06b2..8768e17795756718917d883ccec6ec7df75841d5 100644 --- a/stratosphere-addons/avro/src/main/java/eu/stratosphere/api/avro/AvroBaseValue.java +++ b/stratosphere-addons/avro/src/main/java/eu/stratosphere/api/avro/AvroBaseValue.java @@ -14,10 +14,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.avro; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.avro.mapred.AvroValue; import org.apache.avro.reflect.ReflectDatumReader; import org.apache.avro.reflect.ReflectDatumWriter; @@ -50,7 +50,7 @@ public abstract class AvroBaseValue extends AvroValue implements Key extends AvroValue implements Key implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeUTF(wrappedType); wrapped.write(out); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { if(cl == null) { cl = Thread.currentThread().getContextClassLoader(); } diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java index 980d0e6db99057442cc963fa817105079ee86052..dfa21be0477791197df32b4663a46c8cdd09dde9 100644 --- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java +++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java @@ -13,10 +13,10 @@ package eu.stratosphere.hadoopcompatibility.mapred.wrapper; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.hadoop.io.WritableFactories; import org.apache.hadoop.mapred.JobConf; @@ -48,14 +48,14 @@ public class HadoopInputSplit implements InputSplit { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(splitNumber); out.writeUTF(hadoopInputSplitTypeName); hadoopInputSplit.write(out); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.splitNumber=in.readInt(); this.hadoopInputSplitTypeName = in.readUTF(); if(hadoopInputSplit == null) { diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java index 985ad59c0f520136ff1a6f5adfc209406876daeb..ebaeaf31d8bbecc186b7bb968064cf69f1db5f47 100644 --- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java +++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapreduce/wrapper/HadoopInputSplit.java @@ -13,10 +13,10 @@ package eu.stratosphere.hadoopcompatibility.mapreduce.wrapper; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableFactories; import org.apache.hadoop.mapreduce.JobContext; @@ -50,7 +50,7 @@ public class HadoopInputSplit implements InputSplit { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.splitNumber); out.writeUTF(this.mapreduceInputSplit.getClass().getName()); Writable w = (Writable) this.mapreduceInputSplit; @@ -58,7 +58,7 @@ public class HadoopInputSplit implements InputSplit { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.splitNumber=in.readInt(); String className = in.readUTF(); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java index e2c1d469d608b2286cd4cabc1360bb64042e7217..14f9db3712516682e95f0e4849faff06024fc4de 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/DoubleCounter.java @@ -13,8 +13,9 @@ package eu.stratosphere.api.common.accumulators; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; public class DoubleCounter implements SimpleAccumulator { @@ -44,12 +45,12 @@ public class DoubleCounter implements SimpleAccumulator { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(localValue); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.localValue = in.readDouble(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java index aa72cbb13d67a15ba0bfdc95c075b1ba781397f4..121a1e2357ecd9beb6adf82f5139a644ef1b0f9b 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/Histogram.java @@ -13,12 +13,12 @@ package eu.stratosphere.api.common.accumulators; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Map; import com.google.common.collect.Maps; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * Histogram for discrete-data. Let's you populate a histogram distributedly. @@ -73,7 +73,7 @@ public class Histogram implements Accumulator> { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(hashMap.size()); for (Map.Entry entry : hashMap.entrySet()) { out.writeInt(entry.getKey()); @@ -82,7 +82,7 @@ public class Histogram implements Accumulator> { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { int size = in.readInt(); for (int i = 0; i < size; ++i) { hashMap.put(in.readInt(), in.readInt()); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java index 341b09e0a962e2e7f01df8d2d55d42a4d3c13ba4..37ac56884421d0d0f49f9db086f160bb5caa204e 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/IntCounter.java @@ -13,8 +13,9 @@ package eu.stratosphere.api.common.accumulators; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; public class IntCounter implements SimpleAccumulator { @@ -44,12 +45,12 @@ public class IntCounter implements SimpleAccumulator { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(localValue); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { localValue = in.readInt(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java index eba78b1240d57df01d573480505836e47d24be1a..b4274eb6cfc5b1cc5f4b89c67aba507419ef41c2 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/accumulators/LongCounter.java @@ -13,8 +13,9 @@ package eu.stratosphere.api.common.accumulators; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; @@ -46,12 +47,12 @@ public class LongCounter implements SimpleAccumulator { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(this.localValue); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.localValue = in.readLong(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleDistribution.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleDistribution.java index 7c1ff8bd9d3bc12f076f9509b03fbae2b4e64c6b..71752f82b4c194f6d5feec21382ba842b5e58e9e 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleDistribution.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleDistribution.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.distributions; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Key; import eu.stratosphere.util.InstantiationUtil; @@ -123,7 +123,7 @@ public class SimpleDistribution implements DataDistribution { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.dim); out.writeInt(boundaries.length); @@ -141,7 +141,7 @@ public class SimpleDistribution implements DataDistribution { @SuppressWarnings("unchecked") @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.dim = in.readInt(); final int len = in.readInt(); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleIntegerDistribution.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleIntegerDistribution.java index e7606a8b05a7efe97e5e494fc060f26b3e379568..a1e1df9e1d93ede860fc20d8596e8216b01486c0 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleIntegerDistribution.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/SimpleIntegerDistribution.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.distributions; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.IntValue; public class SimpleIntegerDistribution extends SimpleDistribution { @@ -113,7 +113,7 @@ public class SimpleIntegerDistribution extends SimpleDistribution { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.dim); out.writeInt(boundaries.length); @@ -125,7 +125,7 @@ public class SimpleIntegerDistribution extends SimpleDistribution { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.dim = in.readInt(); final int len = in.readInt(); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformDoubleDistribution.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformDoubleDistribution.java index 706500bd3b87a1ef5b91b1a797f4f6731440e519..b630e5de3edb6a731a3b5f3289aafea6108d4a7f 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformDoubleDistribution.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformDoubleDistribution.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.distributions; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.DoubleValue; @@ -45,13 +45,13 @@ public class UniformDoubleDistribution implements DataDistribution { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(min); out.writeDouble(max); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { min = in.readDouble(); max = in.readDouble(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformIntegerDistribution.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformIntegerDistribution.java index c9bfddaccd40d7d3a9a8722f46420c9a477f74cf..8ad0e76f400f2bb2bedc273039c010f6f2df6975 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformIntegerDistribution.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/distributions/UniformIntegerDistribution.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.distributions; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.IntValue; @@ -46,13 +46,13 @@ public class UniformIntegerDistribution implements DataDistribution { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(min); out.writeInt(max); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { min = in.readInt(); max = in.readInt(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryInputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryInputFormat.java index 72ed645cab61fc684361eaeb92662b82e112e9a3..b0efa3aefe0f8aa54f5e27c0969b3551375fcf14 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryInputFormat.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryInputFormat.java @@ -12,7 +12,6 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.io; -import java.io.DataInput; import java.io.DataInputStream; import java.io.FilterInputStream; import java.io.IOException; @@ -20,6 +19,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -213,7 +214,7 @@ public abstract class BinaryInputFormat extends Fi fdis.seek(file.getLen() - blockInfo.getInfoSize()); DataInputStream input = new DataInputStream(fdis); - blockInfo.read(input); + blockInfo.read(new InputViewDataInputStreamWrapper(input)); totalCount += blockInfo.getAccumulatedRecordCount(); } @@ -249,7 +250,7 @@ public abstract class BinaryInputFormat extends Fi // TODO: seek not supported by compressed streams. Will throw exception this.stream.seek(this.splitStart + this.splitLength - this.blockInfo.getInfoSize()); DataInputStream infoStream = new DataInputStream(this.stream); - this.blockInfo.read(infoStream); + this.blockInfo.read(new InputViewDataInputStreamWrapper(infoStream)); } this.stream.seek(this.splitStart + this.blockInfo.getFirstRecordStart()); @@ -269,12 +270,12 @@ public abstract class BinaryInputFormat extends Fi return null; } - record = this.deserialize(record, this.dataInputStream); + record = this.deserialize(record, new InputViewDataInputStreamWrapper(this.dataInputStream)); this.readRecords++; return record; } - protected abstract T deserialize(T reuse, DataInput dataInput) throws IOException; + protected abstract T deserialize(T reuse, DataInputView dataInput) throws IOException; /** * Writes a block info at the end of the blocks.
diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryOutputFormat.java index b4922040ec06728af718439095f9c36a71afcfdd..7f4e7b3c37d6efa3dffb1f7cd12b83725d7227bf 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryOutputFormat.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BinaryOutputFormat.java @@ -12,7 +12,6 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.io; -import java.io.DataOutput; import java.io.DataOutputStream; import java.io.FilterOutputStream; import java.io.IOException; @@ -20,6 +19,8 @@ import java.io.OutputStream; import eu.stratosphere.configuration.Configuration; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; public abstract class BinaryOutputFormat extends FileOutputFormat { @@ -79,12 +80,12 @@ public abstract class BinaryOutputFormat extends F this.dataOutputStream = new DataOutputStream(this.blockBasedInput); } - protected abstract void serialize(T record, DataOutput dataOutput) throws IOException; + protected abstract void serialize(T record, DataOutputView dataOutput) throws IOException; @Override public void writeRecord(T record) throws IOException { this.blockBasedInput.startRecord(); - this.serialize(record, this.dataOutputStream); + this.serialize(record, new OutputViewDataOutputStreamWrapper(this.dataOutputStream)); } /** @@ -165,7 +166,7 @@ public abstract class BinaryOutputFormat extends F this.blockInfo.setAccumulatedRecordCount(this.totalCount); this.blockInfo.setFirstRecordStart(this.firstRecordStartPos == NO_RECORD ? 0 : this.firstRecordStartPos); BinaryOutputFormat.this.complementBlockInfo(this.blockInfo); - this.blockInfo.write(this.headerStream); + this.blockInfo.write(new OutputViewDataOutputStreamWrapper(this.headerStream)); this.blockPos = 0; this.blockCount = 0; this.firstRecordStartPos = NO_RECORD; diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BlockInfo.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BlockInfo.java index 709d704092a9b4cb9b7e9555d118a522222e5ed2..9084c5de6b63bd449da8349bdea1b22551babb02 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BlockInfo.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/BlockInfo.java @@ -12,11 +12,11 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.io; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; public class BlockInfo implements IOReadableWritable { @@ -50,14 +50,14 @@ public class BlockInfo implements IOReadableWritable { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(this.recordCount); out.writeLong(this.accumulatedRecordCount); out.writeLong(this.firstRecordStart); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.recordCount = in.readLong(); this.accumulatedRecordCount = in.readLong(); this.firstRecordStart = in.readLong(); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedInputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedInputFormat.java index 000bfd785f0a674c3dc06a37b67bf3bd3d69f04d..428988c78cc9b7f3c3850aeac9474912a71788f2 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedInputFormat.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedInputFormat.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.io; -import java.io.DataInput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; /** * Reads elements by deserializing them with their regular serialization/deserialization functionality. @@ -27,7 +27,7 @@ public class SerializedInputFormat extends BinaryI private static final long serialVersionUID = 1L; @Override - protected T deserialize(T reuse, DataInput dataInput) throws IOException { + protected T deserialize(T reuse, DataInputView dataInput) throws IOException { reuse.read(dataInput); return reuse; } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedOutputFormat.java b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedOutputFormat.java index d11083b1e1ee5a65647c1547c6d07d93fe22d3de..e08b307764edc89a290efd385bfda1f8d0cffa70 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedOutputFormat.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/api/common/io/SerializedOutputFormat.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.api.common.io; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataOutputView; /** * Stores elements by serializing them with their regular serialization/deserialization functionality. @@ -27,7 +27,7 @@ public class SerializedOutputFormat extends BinaryOutputFormat, ResettableVa // -------------------------------------------------------------------------------------------- @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readBoolean(); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/ByteValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/ByteValue.java index ab74fa8eb8314a5d0375741456730a276595c5cf..45c057050d0d11115f66f5a6f6cf00ce5cdd94f2 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/ByteValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/ByteValue.java @@ -13,8 +13,6 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.memory.DataInputView; @@ -75,12 +73,12 @@ public class ByteValue implements NormalizableKey, ResettableValue, ResettableValue, ResettableValue, ResettableValue, // -------------------------------------------------------------------------------------------- @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readFloat(); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeFloat(this.value); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/IntValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/IntValue.java index 183f06b4b96e3487ac5feac523aa9ecedd91627d..d608ebebe654089b44043029b93d245b63a3d5c4 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/IntValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/IntValue.java @@ -13,8 +13,6 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.memory.DataInputView; @@ -80,12 +78,12 @@ public class IntValue implements NormalizableKey, ResettableValue implements Value, List { * @see eu.stratosphere.nephele.io.IOReadableWritable#read(java.io.DataInput) */ @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { int size = in.readInt(); this.list.clear(); @@ -106,7 +106,7 @@ public abstract class ListValue implements Value, List { * @see eu.stratosphere.nephele.io.IOReadableWritable#write(java.io.DataOutput) */ @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(this.list.size()); for (final V value : this.list) { value.write(out); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/LongValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/LongValue.java index ac9e0913aa8c49f87e4aeac63acb847eb2924bc7..fd105892498f7322e06d0d06476a74cdf8dc00e2 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/LongValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/LongValue.java @@ -13,8 +13,6 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.memory.DataInputView; @@ -88,7 +86,7 @@ public class LongValue implements NormalizableKey, ResettableValue, ResettableValue implements Valu * @see eu.stratosphere.nephele.io.IOReadableWritable#read(java.io.DataInput) */ @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { int size = in.readInt(); this.map.clear(); @@ -97,7 +97,7 @@ public abstract class MapValue implements Valu * @see eu.stratosphere.nephele.io.IOReadableWritable#write(java.io.DataOutput) */ @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(this.map.size()); for (final Entry entry : this.map.entrySet()) { entry.getKey().write(out); diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/NullValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/NullValue.java index fc7e996bd8efc2bd811aec3dc9ef35553a65dfd5..fa29d8fc03e239f5a29603b2af4685320952189e 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/NullValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/NullValue.java @@ -13,8 +13,6 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.memory.DataInputView; @@ -60,12 +58,12 @@ public final class NullValue implements NormalizableKey, CopyableValu // -------------------------------------------------------------------------------------------- @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { in.readBoolean(); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(false); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Pair.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Pair.java index 3906984edcb60468994a8aacfd1f18ea59daa391..e2d5fea0af4bd9d69c700ca0da6fed36693b8a0e 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/Pair.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Pair.java @@ -13,10 +13,10 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.ReflectionUtil; /** @@ -123,13 +123,13 @@ public abstract class Pair, V extends Key> implements Key { // -------------------------------------------------------------------------------------------- @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // make sure everything is in a valid binary representation updateBinaryRepresenation(); @@ -1080,7 +1080,7 @@ public final class Record implements Value, CopyableValue { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { final int len = readVarLengthInt(in); this.binaryLen = len; @@ -1187,7 +1187,7 @@ public final class Record implements Value, CopyableValue { } /** - * Writes this record to the given output view. This method is similar to {@link #write(DataOutput)}, but + * Writes this record to the given output view. This method is similar to {@link eu.stratosphere.core.io.IOReadableWritable#write(eu.stratosphere.core.memory.DataOutputView)}, but * it returns the number of bytes written. * * @param target The view to write the record to. @@ -1262,12 +1262,12 @@ public final class Record implements Value, CopyableValue { private static final long serialVersionUID = 1L; @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { throw new UnsupportedOperationException(); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { throw new UnsupportedOperationException(); } }; @@ -1275,7 +1275,7 @@ public final class Record implements Value, CopyableValue { /** * Internal interface class to provide serialization for the data types. */ - private static final class InternalDeSerializer implements DataInput, DataOutput, Serializable { + private static final class InternalDeSerializer implements DataInputView, DataOutputView, Serializable { private static final long serialVersionUID = 1L; @@ -1510,6 +1510,49 @@ public final class Record implements Value, CopyableValue { return n; } } + + @Override + public void skipBytesToRead(int numBytes) throws IOException { + if(this.end - this.position < numBytes) { + throw new EOFException("Could not skip " + numBytes + "."); + } + skipBytes(numBytes); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if(b == null){ + throw new NullPointerException("Byte array b cannot be null."); + } + + if(off < 0){ + throw new IndexOutOfBoundsException("Offset cannot be negative."); + } + + if(len < 0){ + throw new IndexOutOfBoundsException("Length cannot be negative."); + } + + if(b.length - off < len){ + throw new IndexOutOfBoundsException("Byte array does not provide enough space to store requested data" + + "."); + } + + if(this.position >= this.end){ + return -1; + }else{ + int toRead = Math.min(this.end-this.position, len); + System.arraycopy(this.memory,this.position,b,off,toRead); + this.position += toRead; + + return toRead; + } + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } // ---------------------------------------------------------------------------------------- // Data Output @@ -1736,5 +1779,24 @@ public final class Record implements Value, CopyableValue { private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN); + + @Override + public void skipBytesToWrite(int numBytes) throws IOException { + int skippedBytes = skipBytes(numBytes); + + if(skippedBytes != numBytes){ + throw new EOFException("Could not skip " + numBytes + " bytes."); + } + } + + @Override + public void write(DataInputView source, int numBytes) throws IOException { + if(numBytes > this.end - this.position){ + throw new IOException("Could not write " + numBytes + " bytes since the buffer is full."); + } + + source.read(this.memory,this.position, numBytes); + this.position += numBytes; + } } } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/ShortValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/ShortValue.java index 52d6fd24d69a841f724e2cb6b85c8df530185427..d2429865ca50f7b4c182ab50c2ced500702cee01 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/ShortValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/ShortValue.java @@ -13,8 +13,6 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.memory.DataInputView; @@ -80,12 +78,12 @@ public class ShortValue implements NormalizableKey, ResettableValue< // -------------------------------------------------------------------------------------------- @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readShort(); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeShort(this.value); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/StringValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/StringValue.java index cc970e84852c62dc88475173d587e06990dd756c..1a133f62ebe492d7f6e37966e69024417ce99256 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/StringValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/StringValue.java @@ -470,7 +470,7 @@ public class StringValue implements NormalizableKey, CharSequence, // -------------------------------------------------------------------------------------------- @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { int len = in.readUnsignedByte(); if (len >= HIGH_BIT) { @@ -508,7 +508,7 @@ public class StringValue implements NormalizableKey, CharSequence, } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { int len = this.len; // write the length, variable-length encoded diff --git a/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/BinaryInputFormatTest.java b/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/BinaryInputFormatTest.java index 7d10a9fd134aaef6af9259c4e95fec6d4eef6744..e8200132ab21e7f496d2d2035567b51f4abbc0e2 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/BinaryInputFormatTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/BinaryInputFormatTest.java @@ -13,11 +13,11 @@ package eu.stratosphere.api.common.io; -import java.io.DataInput; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; import org.apache.log4j.Level; import org.junit.Assert; import org.junit.BeforeClass; @@ -35,7 +35,7 @@ public class BinaryInputFormatTest { private static final long serialVersionUID = 1L; @Override - protected Record deserialize(Record record, DataInput dataInput) { + protected Record deserialize(Record record, DataInputView dataInput) { return record; } } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/SequentialFormatTest.java b/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/SequentialFormatTest.java index 420fdf6a28138955b993cfddd8dd5c22ebc2f9be..164899ce625672e8747bf2836a79a764a53e7436 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/SequentialFormatTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/api/common/io/SequentialFormatTest.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.Comparator; import java.util.List; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import org.apache.log4j.Level; import org.junit.After; import org.junit.Assert; @@ -92,7 +93,7 @@ public class SequentialFormatTest { ByteCounter byteCounter = new ByteCounter(); DataOutputStream out = new DataOutputStream(byteCounter); for (int fileCount = 0; fileCount < this.getNumberOfTuplesPerFile(fileIndex); fileCount++, recordIndex++) { - this.getRecord(recordIndex).write(out); + this.getRecord(recordIndex).write(new OutputViewDataOutputStreamWrapper(out)); } this.rawDataSizes[fileIndex] = byteCounter.getLength(); } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/api/distributions/SimpleDataDistributionTest.java b/stratosphere-core/src/test/java/eu/stratosphere/api/distributions/SimpleDataDistributionTest.java index c87d2e14acfe5f28e3ad53d98c2298f5c0ab61c1..eb83f8c23f44085d2404772c1f820396888bb35d 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/api/distributions/SimpleDataDistributionTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/api/distributions/SimpleDataDistributionTest.java @@ -19,6 +19,8 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import org.junit.Test; @@ -107,7 +109,7 @@ public class SimpleDataDistributionTest { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream dos = new DataOutputStream(baos); try { - ddWrite.write(dos); + ddWrite.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException e) { Assert.fail("Error serializing the DataDistribution: " + e.getMessage()); } @@ -120,7 +122,7 @@ public class SimpleDataDistributionTest { SimpleDistribution ddRead = new SimpleDistribution(); try { - ddRead.read(in); + ddRead.read(new InputViewDataInputStreamWrapper(in)); } catch (Exception ex) { Assert.fail("The deserialization of the encoded data distribution caused an error"); } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/core/testutils/CommonTestUtils.java b/stratosphere-core/src/test/java/eu/stratosphere/core/testutils/CommonTestUtils.java index dcfdb805e8eee601343c2751bdd7a591acc7e99d..fcdddcbb333eaa3d5422d1be649d02926c8a3a39 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/core/testutils/CommonTestUtils.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/core/testutils/CommonTestUtils.java @@ -25,6 +25,8 @@ import java.io.IOException; import eu.stratosphere.configuration.ConfigConstants; import eu.stratosphere.configuration.GlobalConfiguration; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; /** * This class contains auxiliary methods for unit tests in the Nephele common module. @@ -95,7 +97,7 @@ public class CommonTestUtils { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream dos = new DataOutputStream(baos); - original.write(dos); + original.write(new OutputViewDataOutputStreamWrapper(dos)); final String className = original.getClass().getName(); if (className == null) { @@ -130,7 +132,7 @@ public class CommonTestUtils { final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); final DataInputStream dis = new DataInputStream(bais); - copy.read(dis); + copy.read(new InputViewDataInputStreamWrapper(dis)); return copy; } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/types/CollectionsDataTypeTest.java b/stratosphere-core/src/test/java/eu/stratosphere/types/CollectionsDataTypeTest.java index 9e81d742b8e78f69df4763905e41bc1edb2fc0b1..163473e5451335be9b1cfbd5580f957ef005051c 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/types/CollectionsDataTypeTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/types/CollectionsDataTypeTest.java @@ -22,6 +22,8 @@ import java.util.Arrays; import java.util.HashSet; import java.util.Map.Entry; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import org.junit.Before; @@ -58,8 +60,8 @@ public class CollectionsDataTypeTest { try { NfIntStringPair mPairActual = new NfIntStringPair(); - pair1.write(out); - mPairActual.read(in); + pair1.write(new OutputViewDataOutputStreamWrapper(out)); + mPairActual.read(new InputViewDataInputStreamWrapper(in)); Assert.assertEquals(pair1, mPairActual); } catch (IOException e) { @@ -182,8 +184,8 @@ public class CollectionsDataTypeTest { // now test data transfer NfIntStringMap nMap = new NfIntStringMap(); try { - map0.write(out); - nMap.read(in); + map0.write(new OutputViewDataOutputStreamWrapper(out)); + nMap.read(new InputViewDataInputStreamWrapper(in)); } catch (Exception e) { Assert.assertTrue(false); } @@ -210,8 +212,8 @@ public class CollectionsDataTypeTest { // test data transfer NfStringList mList2 = new NfStringList(); try { - list.write(out); - mList2.read(in); + list.write(new OutputViewDataOutputStreamWrapper(out)); + mList2.read(new InputViewDataInputStreamWrapper(in)); } catch (Exception e) { Assert.assertTrue(false); } diff --git a/stratosphere-core/src/test/java/eu/stratosphere/types/PrimitiveDataTypeTest.java b/stratosphere-core/src/test/java/eu/stratosphere/types/PrimitiveDataTypeTest.java index 7e893ed1c5f1cc49decc607b2ed83e3c300da1d5..723ea18a3a91364954e4e2feb59c708bb1a88a2f 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/types/PrimitiveDataTypeTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/types/PrimitiveDataTypeTest.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.io.PipedInputStream; import java.io.PipedOutputStream; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import org.junit.Before; @@ -56,15 +58,15 @@ public class PrimitiveDataTypeTest { Assert.assertEquals(int0.compareTo(int3), -1); // test stream output and retrieval try { - int0.write(mOut); - int2.write(mOut); - int3.write(mOut); + int0.write(new OutputViewDataOutputStreamWrapper(mOut)); + int2.write(new OutputViewDataOutputStreamWrapper(mOut)); + int3.write(new OutputViewDataOutputStreamWrapper(mOut)); IntValue int1n = new IntValue(); IntValue int2n = new IntValue(); IntValue int3n = new IntValue(); - int1n.read(mIn); - int2n.read(mIn); - int3n.read(mIn); + int1n.read(new InputViewDataInputStreamWrapper(mIn)); + int2n.read(new InputViewDataInputStreamWrapper(mIn)); + int3n.read(new InputViewDataInputStreamWrapper(mIn)); Assert.assertEquals(int0.compareTo(int1n), 0); Assert.assertEquals(int0.getValue(), int1n.getValue()); Assert.assertEquals(int2.compareTo(int2n), 0); @@ -92,15 +94,15 @@ public class PrimitiveDataTypeTest { Assert.assertEquals(double0.compareTo(double3), -1); // test stream output and retrieval try { - double0.write(mOut); - double2.write(mOut); - double3.write(mOut); + double0.write(new OutputViewDataOutputStreamWrapper(mOut)); + double2.write(new OutputViewDataOutputStreamWrapper(mOut)); + double3.write(new OutputViewDataOutputStreamWrapper(mOut)); DoubleValue double1n = new DoubleValue(); DoubleValue double2n = new DoubleValue(); DoubleValue double3n = new DoubleValue(); - double1n.read(mIn); - double2n.read(mIn); - double3n.read(mIn); + double1n.read(new InputViewDataInputStreamWrapper(mIn)); + double2n.read(new InputViewDataInputStreamWrapper(mIn)); + double3n.read(new InputViewDataInputStreamWrapper(mIn)); Assert.assertEquals(double0.compareTo(double1n), 0); Assert.assertEquals(double0.getValue(), double1n.getValue()); Assert.assertEquals(double2.compareTo(double2n), 0); @@ -156,21 +158,21 @@ public class PrimitiveDataTypeTest { // test stream out/input try { - string0.write(mOut); - string4.write(mOut); - string2.write(mOut); - string3.write(mOut); - string7.write(mOut); + string0.write(new OutputViewDataOutputStreamWrapper(mOut)); + string4.write(new OutputViewDataOutputStreamWrapper(mOut)); + string2.write(new OutputViewDataOutputStreamWrapper(mOut)); + string3.write(new OutputViewDataOutputStreamWrapper(mOut)); + string7.write(new OutputViewDataOutputStreamWrapper(mOut)); StringValue string1n = new StringValue(); StringValue string2n = new StringValue(); StringValue string3n = new StringValue(); StringValue string4n = new StringValue(); StringValue string7n = new StringValue(); - string1n.read(mIn); - string4n.read(mIn); - string2n.read(mIn); - string3n.read(mIn); - string7n.read(mIn); + string1n.read(new InputViewDataInputStreamWrapper(mIn)); + string4n.read(new InputViewDataInputStreamWrapper(mIn)); + string2n.read(new InputViewDataInputStreamWrapper(mIn)); + string3n.read(new InputViewDataInputStreamWrapper(mIn)); + string7n.read(new InputViewDataInputStreamWrapper(mIn)); Assert.assertEquals(string0.compareTo(string1n), 0); Assert.assertEquals(string0.toString(), string1n.toString()); Assert.assertEquals(string4.compareTo(string4n), 0); @@ -209,12 +211,12 @@ public class PrimitiveDataTypeTest { try { // write it multiple times for (int i = 0; i < numWrites; i++) { - pn.write(mOut); + pn.write(new OutputViewDataOutputStreamWrapper(mOut)); } // read it multiple times for (int i = 0; i < numWrites; i++) { - pn.read(mIn); + pn.read(new InputViewDataInputStreamWrapper(mIn)); } Assert.assertEquals("Reading PactNull does not consume the same data as was written.", mIn.available(), 0); diff --git a/stratosphere-core/src/test/java/eu/stratosphere/types/RecordTest.java b/stratosphere-core/src/test/java/eu/stratosphere/types/RecordTest.java index 2f14257017dc5922672307edd0f07bf5a636f2f1..89b3ce63b3149a2356e99c5dd56fe637ee62a143 100644 --- a/stratosphere-core/src/test/java/eu/stratosphere/types/RecordTest.java +++ b/stratosphere-core/src/test/java/eu/stratosphere/types/RecordTest.java @@ -26,6 +26,8 @@ import java.io.PipedOutputStream; import java.util.Arrays; import java.util.Random; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -59,15 +61,15 @@ public class RecordTest { try { // test deserialize into self Record empty = new Record(); - empty.write(this.out); - empty.read(this.in); + empty.write(new OutputViewDataOutputStreamWrapper(this.out)); + empty.read(new InputViewDataInputStreamWrapper(this.in)); Assert.assertTrue("Deserialized Empty record is not another empty record.", empty.getNumFields() == 0); // test deserialize into new empty = new Record(); - empty.write(this.out); + empty.write(new OutputViewDataOutputStreamWrapper(this.out)); empty = new Record(); - empty.read(this.in); + empty.read(new InputViewDataInputStreamWrapper(this.in)); Assert.assertTrue("Deserialized Empty record is not another empty record.", empty.getNumFields() == 0); } catch (Throwable t) { @@ -382,18 +384,18 @@ public class RecordTest { try { // serialize and deserialize to remove all buffered info - r.write(out); + r.write(new OutputViewDataOutputStreamWrapper(out)); r = new Record(); - r.read(in); + r.read(new InputViewDataInputStreamWrapper(in)); r.setField(1, new IntValue(10)); r.setField(4, new StringValue("Some long value")); r.setField(5, new StringValue("An even longer value")); r.setField(10, new IntValue(10)); - r.write(out); + r.write(new OutputViewDataOutputStreamWrapper(out)); r = new Record(); - r.read(in); + r.read(new InputViewDataInputStreamWrapper(in)); assertTrue(r.getField(0, IntValue.class).getValue() == 0); assertTrue(r.getField(1, IntValue.class).getValue() == 10); @@ -427,8 +429,8 @@ public class RecordTest { Record record2 = new Record(); try { // De/Serialize the record - record1.write(this.out); - record2.read(this.in); + record1.write(new OutputViewDataOutputStreamWrapper(this.out)); + record2.read(new InputViewDataInputStreamWrapper(this.in)); assertTrue(record1.getNumFields() == record2.getNumFields()); @@ -456,20 +458,20 @@ public class RecordTest { try { Record record = new Record(new IntValue(42)); - record.write(out); + record.write(new OutputViewDataOutputStreamWrapper(out)); Assert.assertEquals(42, record.getField(0, IntValue.class).getValue()); record.setField(0, new IntValue(23)); - record.write(out); + record.write(new OutputViewDataOutputStreamWrapper(out)); Assert.assertEquals(23, record.getField(0, IntValue.class).getValue()); record.clear(); Assert.assertEquals(0, record.getNumFields()); Record record2 = new Record(new IntValue(42)); - record2.read(in); + record2.read(new InputViewDataInputStreamWrapper(in)); Assert.assertEquals(42, record2.getField(0, IntValue.class).getValue()); - record2.read(in); + record2.read(new InputViewDataInputStreamWrapper(in)); Assert.assertEquals(23, record2.getField(0, IntValue.class).getValue()); } catch (Throwable t) { Assert.fail("Test failed due to an exception: " + t.getMessage()); @@ -541,7 +543,8 @@ public class RecordTest { } } - static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataInput reader, DataOutput writer) + static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataInputStream reader, + DataOutputStream writer) throws Exception { final int[] permutation1 = createPermutation(rnd, values.length); @@ -586,9 +589,9 @@ public class RecordTest { final int pos = permutation1[i]; rec.setField(pos, values[pos]); } - rec.write(writer); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); rec = new Record(); - rec.read(reader); + rec.read(new InputViewDataInputStreamWrapper(reader)); testAllRetrievalMethods(rec, permutation2, values); // test adding and retrieving with full stream serialization and deserialization into the same record @@ -597,8 +600,8 @@ public class RecordTest { final int pos = permutation1[i]; rec.setField(pos, values[pos]); } - rec.write(writer); - rec.read(reader); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); + rec.read(new InputViewDataInputStreamWrapper(reader)); testAllRetrievalMethods(rec, permutation2, values); // test adding and retrieving with partial stream serialization and deserialization into a new record @@ -606,18 +609,18 @@ public class RecordTest { updatePos = rnd.nextInt(values.length + 1); for (int i = 0; i < values.length; i++) { if (i == updatePos) { - rec.write(writer); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); rec = new Record(); - rec.read(reader); + rec.read(new InputViewDataInputStreamWrapper(reader)); } final int pos = permutation1[i]; rec.setField(pos, values[pos]); } if (updatePos == values.length) { - rec.write(writer); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); rec = new Record(); - rec.read(reader); + rec.read(new InputViewDataInputStreamWrapper(reader)); } testAllRetrievalMethods(rec, permutation2, values); @@ -626,16 +629,16 @@ public class RecordTest { updatePos = rnd.nextInt(values.length + 1); for (int i = 0; i < values.length; i++) { if (i == updatePos) { - rec.write(writer); - rec.read(reader); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); + rec.read(new InputViewDataInputStreamWrapper(reader)); } final int pos = permutation1[i]; rec.setField(pos, values[pos]); } if (updatePos == values.length) { - rec.write(writer); - rec.read(reader); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); + rec.read(new InputViewDataInputStreamWrapper(reader)); } testAllRetrievalMethods(rec, permutation2, values); @@ -644,17 +647,17 @@ public class RecordTest { updatePos = rnd.nextInt(values.length + 1); for (int i = 0; i < values.length; i++) { if (i == updatePos) { - rec.write(writer); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); rec = new Record(); - rec.read(reader); + rec.read(new InputViewDataInputStreamWrapper(reader)); } final int pos = permutation1[i]; rec.setField(pos, values[pos]); } - rec.write(writer); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); rec = new Record(); - rec.read(reader); + rec.read(new InputViewDataInputStreamWrapper(reader)); testAllRetrievalMethods(rec, permutation2, values); // test adding and retrieving with partial stream serialization and deserialization into the same record @@ -662,15 +665,15 @@ public class RecordTest { updatePos = rnd.nextInt(values.length + 1); for (int i = 0; i < values.length; i++) { if (i == updatePos) { - rec.write(writer); - rec.read(reader); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); + rec.read(new InputViewDataInputStreamWrapper(reader)); } final int pos = permutation1[i]; rec.setField(pos, values[pos]); } - rec.write(writer); - rec.read(reader); + rec.write(new OutputViewDataOutputStreamWrapper(writer)); + rec.read(new InputViewDataInputStreamWrapper(reader)); testAllRetrievalMethods(rec, permutation2, values); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java index 82f275573f4154f944880d6e73b5bb031aa28b08..db75ecd28b5e64e63077bef53714944e07dcb931 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/io/CollectionInputFormat.java @@ -27,8 +27,8 @@ import eu.stratosphere.api.common.io.GenericInputFormat; import eu.stratosphere.api.common.io.NonParallelInput; import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.core.io.GenericInputSplit; -import eu.stratosphere.core.memory.InputViewDataInputWrapper; -import eu.stratosphere.core.memory.OutputViewDataOutputWrapper; +import eu.stratosphere.core.memory.InputViewObjectInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewObjectOutputStreamWrapper; /** * An input format that returns objects from a collection. @@ -78,11 +78,8 @@ public class CollectionInputFormat extends GenericInputFormat implements N out.defaultWriteObject(); out.writeInt(dataSet.size()); - OutputViewDataOutputWrapper outWrapper = new OutputViewDataOutputWrapper(); - outWrapper.setDelegate(out); - for (T element : dataSet){ - serializer.serialize(element, outWrapper); + serializer.serialize(element, new OutputViewObjectOutputStreamWrapper(out)); } } @@ -92,12 +89,10 @@ public class CollectionInputFormat extends GenericInputFormat implements N int collectionLength = in.readInt(); List list = new ArrayList(collectionLength); - InputViewDataInputWrapper inWrapper = new InputViewDataInputWrapper(); - inWrapper.setDelegate(in); for (int i = 0; i < collectionLength; i++){ T element = serializer.createInstance(); - element = serializer.deserialize(element, inWrapper); + element = serializer.deserialize(element, new InputViewObjectInputStreamWrapper(in)); list.add(element); } diff --git a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/io/ExternalProcessInputSplit.java b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/io/ExternalProcessInputSplit.java index f07ddd5efc22b871cd5b69535d7225ac2c1150ab..e6677a47d6f5fea83ab4b7d99af120c78a3bb345 100644 --- a/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/io/ExternalProcessInputSplit.java +++ b/stratosphere-java/src/main/java/eu/stratosphere/api/java/record/io/ExternalProcessInputSplit.java @@ -13,12 +13,12 @@ package eu.stratosphere.api.java.record.io; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.GenericInputSplit; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * The ExternalProcessInputSplit contains all informations for {@link InputFormat} that read their data from external processes. @@ -58,13 +58,13 @@ public class ExternalProcessInputSplit extends GenericInputSplit { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); this.extProcessCommand = StringRecord.readString(in); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); StringRecord.writeString(out, this.extProcessCommand); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java index 648c8dcde454bce43f94264600dfdf03c5bcfbad..fa9b740be62e63d04398096b15dec3f1dc2eb679 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/AbstractID.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.StringUtils; import io.netty.buffer.ByteBuf; @@ -152,13 +152,13 @@ public class AbstractID implements IOReadableWritable { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.lowerPart = in.readLong(); this.upperPart = in.readLong(); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(this.lowerPart); out.writeLong(this.upperPart); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/AbstractJobResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/AbstractJobResult.java index 13b337d38187b79a6ae57cab301b1aa874aaf44e..931f1a2aa13a3705a2be3a69eb0d023db248a4e6 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/AbstractJobResult.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/AbstractJobResult.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele.client; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.util.EnumUtils; /** @@ -78,7 +78,7 @@ public abstract class AbstractJobResult implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read the return code this.returnCode = EnumUtils.readEnum(in, ReturnCode.class); @@ -89,7 +89,7 @@ public abstract class AbstractJobResult implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write the return code EnumUtils.writeEnum(out, this.returnCode); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobProgressResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobProgressResult.java index 5f4e34bf2d8b09e792d39209a4339d12b139bbe7..d6d05ba6cb66645917f45718a78fde3c1766c9bf 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobProgressResult.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobProgressResult.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.client; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Iterator; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.event.job.AbstractEvent; import eu.stratosphere.nephele.util.SerializableArrayList; @@ -62,7 +62,7 @@ public class JobProgressResult extends AbstractJobResult { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); this.events.read(in); @@ -70,7 +70,7 @@ public class JobProgressResult extends AbstractJobResult { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); this.events.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobSubmissionResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobSubmissionResult.java index 9913be529fc49cef509adbefc58afe7b420335b8..9443b0b9c1030b10a35cdbbf75d2a3b66bb3b6e6 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobSubmissionResult.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/client/JobSubmissionResult.java @@ -13,8 +13,9 @@ package eu.stratosphere.nephele.client; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; /** @@ -49,13 +50,13 @@ public class JobSubmissionResult extends AbstractJobResult { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java index 4068e5b3ade5abcbfab3c0e9ce5c694df0e33327..daacd13c4f54b355b7ba5e0cc096e27b76154494 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/ChannelDeploymentDescriptor.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.deployment; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.runtime.io.channels.ChannelID; /** @@ -72,7 +72,7 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.outputChannelID.write(out); this.inputChannelID.write(out); @@ -80,7 +80,7 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.outputChannelID.read(in); this.inputChannelID.read(in); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java index 02d6578a67621faae9ba630dfdceba073585348a..7ad26dd3ab35f138e59d8cde30e78c884f9b4239 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/GateDeploymentDescriptor.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.deployment; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.runtime.io.gates.GateID; import eu.stratosphere.runtime.io.channels.ChannelType; import eu.stratosphere.nephele.util.EnumUtils; @@ -93,7 +93,7 @@ public final class GateDeploymentDescriptor implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.gateID.write(out); EnumUtils.writeEnum(out, channelType); @@ -106,7 +106,7 @@ public final class GateDeploymentDescriptor implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.gateID.read(in); this.channelType = EnumUtils.readEnum(in, ChannelType.class); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptor.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptor.java index 06fabde00c69a9a37f24ed9ae53b42f0df26287b..42826008fb6d42ad1abe2900daf8493c8304b0ae 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptor.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/deployment/TaskDeploymentDescriptor.java @@ -13,13 +13,13 @@ package eu.stratosphere.nephele.deployment; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.configuration.Configuration; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -189,7 +189,7 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.jobID.write(out); this.vertexID.write(out); @@ -222,7 +222,7 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable { @SuppressWarnings("unchecked") @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.jobID.read(in); this.vertexID.read(in); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/AbstractEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/AbstractEvent.java index 80e65ffe3df4fa88a84a6c1d00b02113c2a4ba87..cffec9d37d30217dee9c9eb4e7538b2d3afe5c00 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/AbstractEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/AbstractEvent.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.text.SimpleDateFormat; import java.util.Date; import java.util.concurrent.atomic.AtomicLong; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * An abstract event is transmitted from the job manager to the @@ -74,7 +74,7 @@ public abstract class AbstractEvent implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read the timestamp this.timestamp = in.readLong(); @@ -83,7 +83,7 @@ public abstract class AbstractEvent implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write the timestamp out.writeLong(this.timestamp); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/ExecutionStateChangeEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/ExecutionStateChangeEvent.java index a2cac79440ca8a17014ddfac2d194f468e67bf98..cf451ab6b04b1cc343910f90f9397fb86e5c1060 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/ExecutionStateChangeEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/ExecutionStateChangeEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; import eu.stratosphere.nephele.util.EnumUtils; @@ -86,7 +86,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); @@ -96,7 +96,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/JobEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/JobEvent.java index 420784abc16a157a6e5e5d4652685402bf7cbacf..b231978e0e547cf8ea16ba8b40af7758050b8a6e 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/JobEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/JobEvent.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.jobgraph.JobStatus; import eu.stratosphere.nephele.util.EnumUtils; @@ -68,7 +68,7 @@ public class JobEvent extends AbstractEvent { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); // Read job status @@ -80,7 +80,7 @@ public class JobEvent extends AbstractEvent { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); // Write job status diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/RecentJobEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/RecentJobEvent.java index a477c7a81c4f45754d4950a5fb790fde6d2e917c..914adc374ae311d49bc0047935bf625997588eb9 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/RecentJobEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/RecentJobEvent.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.jobgraph.JobStatus; import eu.stratosphere.nephele.util.EnumUtils; @@ -139,7 +139,7 @@ public final class RecentJobEvent extends AbstractEvent implements ManagementEve @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); // Read the job ID @@ -161,7 +161,7 @@ public final class RecentJobEvent extends AbstractEvent implements ManagementEve @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); // Write the job ID diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java index c86c12bb758a4254ecda314c8f8145933e5ec144..4b980ed7f7660fe018d97cc7003f898c2456a80d 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexAssignmentEvent.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; /** @@ -83,7 +83,7 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); @@ -93,7 +93,7 @@ public final class VertexAssignmentEvent extends AbstractEvent implements Manage @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexEvent.java index 940f997ff4f0df48544eddfbfc157860ea8a97bc..03f15257a0883d4ed1a79faa4589f41b2093778d 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/job/VertexEvent.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.event.job; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.jobgraph.JobVertexID; import eu.stratosphere.nephele.util.EnumUtils; @@ -108,7 +108,7 @@ public class VertexEvent extends AbstractEvent { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); @@ -122,7 +122,7 @@ public class VertexEvent extends AbstractEvent { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/IntegerTaskEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/IntegerTaskEvent.java index 04e7b18a3beaec3b256d1d2b5136b52045610e51..43e42e972c9513197d6902529b97c118e10a5c81 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/IntegerTaskEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/IntegerTaskEvent.java @@ -30,8 +30,9 @@ package eu.stratosphere.nephele.event.task; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; /** @@ -74,13 +75,13 @@ public class IntegerTaskEvent extends AbstractTaskEvent { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(this.value); } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.value = in.readInt(); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/StringTaskEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/StringTaskEvent.java index 70797674cd622cb09413efae7b61bfeda61c48c2..68e1ef599bcee8bad96a085ba15934b605986507 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/StringTaskEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/event/task/StringTaskEvent.java @@ -29,11 +29,11 @@ */ package eu.stratosphere.nephele.event.task; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * This class provides a simple implementation of an event that holds a string value. @@ -73,14 +73,14 @@ public class StringTaskEvent extends AbstractTaskEvent { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { StringRecord.writeString(out, this.message); } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.message = StringRecord.readString(in); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileRequest.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileRequest.java index dc5001286d4ed2508043d1e81b2f089b733b84f3..82e86868c9b169f601d1f45eaa1359cad47a468c 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileRequest.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileRequest.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele.execution.librarycache; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * A library cache profile request includes a set of library names and issues a task manager to report which of these @@ -54,7 +54,7 @@ public class LibraryCacheProfileRequest implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read required jar files this.requiredLibraries = new String[in.readInt()]; @@ -66,7 +66,7 @@ public class LibraryCacheProfileRequest implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { if (this.requiredLibraries == null) { throw new IOException("requiredLibraries is null"); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileResponse.java index 3a2c92ce326f1e69d7b94ff25ca52d1d17a95fab..7c1ec2666adc262e9ac47e58cbc143648703d04f 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileResponse.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheProfileResponse.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele.execution.librarycache; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * A library cache profile response is the response to a library cache profile request. It contains the set of @@ -92,7 +92,7 @@ public class LibraryCacheProfileResponse implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read the names of the required jar files this.requiredLibraries = new String[in.readInt()]; @@ -110,7 +110,7 @@ public class LibraryCacheProfileResponse implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { if (this.requiredLibraries == null) { throw new IOException("requiredLibraries is null"); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheUpdate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheUpdate.java index 05473c72019ed0ba1f4c005de7017e14a8ecdc0d..6f7c404983138771fb4a0fe05c4adb39413fd903 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheUpdate.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/execution/librarycache/LibraryCacheUpdate.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.execution.librarycache; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * This class is used to encapsulate the transmission of a library file in a Nephele RPC call. @@ -48,14 +48,14 @@ public class LibraryCacheUpdate implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { LibraryCacheManager.readLibraryFromStream(in); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { if (this.libraryFileName == null) { throw new IOException("libraryFileName is null"); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescription.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescription.java index 83d650ac4eda1a4b88894d984bf0d25cfd70f5dc..2ca6068bc132e1dfb1c5291490a4cf2b68be5ef5 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescription.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/HardwareDescription.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.instance; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * A hardware description reflects the hardware environment which is actually present on the task manager's compute @@ -68,7 +68,7 @@ public final class HardwareDescription implements IOReadableWritable { } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(this.numberOfCPUCores); out.writeLong(this.sizeOfPhysicalMemory); @@ -76,7 +76,7 @@ public final class HardwareDescription implements IOReadableWritable { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.numberOfCPUCores = in.readInt(); this.sizeOfPhysicalMemory = in.readLong(); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java index 98c38105f1f767d4dac6d3e874b7350219c06826..b47f649f5285589995e7ea83081fb8c1de977c77 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/instance/InstanceConnectionInfo.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.instance; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.net.InetAddress; import java.net.UnknownHostException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.StringUtils; /** @@ -206,7 +206,7 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable b.length) { throw new IndexOutOfBoundsException(); } - + int remaining = this.limitInSegment - this.positionInSegment; if (remaining >= len) { this.currentSegment.get(this.positionInSegment, b, off, len); this.positionInSegment += len; + return len; } else { if (remaining == 0) { - advance(); + try { + advance(); + }catch(EOFException eof){ + return -1; + } remaining = this.limitInSegment - this.positionInSegment; } - + + int bytesRead = 0; while (true) { - int toRead = Math.min(remaining, len); + int toRead = Math.min(remaining, len-bytesRead); this.currentSegment.get(this.positionInSegment, b, off, toRead); off += toRead; - len -= toRead; - - if (len > 0) { - advance(); - remaining = this.limitInSegment - this.positionInSegment; + bytesRead += toRead; + + if (len > bytesRead) { + try { + advance(); + }catch(EOFException eof){ + return bytesRead; + } + remaining = this.limitInSegment - this.positionInSegment; } else { this.positionInSegment += toRead; break; } } + return len; + } + } + + @Override + public void readFully(byte[] b) throws IOException { + readFully(b, 0, b.length); + } + + @Override + public void readFully(byte[] b, int off, int len) throws IOException { + int bytesRead = read(b,off,len); + + if(bytesRead == -1){ + throw new EOFException("There is no more data left in the DataInputView."); } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java index ffb2bb20451b1448f8c2e1cf84483a443a14cd09..093f9b2ed47733069066f4b6e57fab13861dc035 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/AbstractTaskResult.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele.taskmanager; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.util.EnumUtils; @@ -100,7 +100,7 @@ public abstract class AbstractTaskResult implements IOReadableWritable { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { // Read the jobID boolean isNotNull = in.readBoolean(); @@ -118,7 +118,7 @@ public abstract class AbstractTaskResult implements IOReadableWritable { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // Write jobID if (this.vertexID == null) { diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskExecutionState.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskExecutionState.java index 9f372939176d056503b7cf10dec68483c2a8cadc..edd442efb1713cd2788e9844ec2185fc4843b5ce 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskExecutionState.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/TaskExecutionState.java @@ -13,12 +13,12 @@ package eu.stratosphere.nephele.taskmanager; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -103,7 +103,7 @@ public class TaskExecutionState implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { boolean isNotNull = in.readBoolean(); @@ -133,7 +133,7 @@ public class TaskExecutionState implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { if (this.jobID == null) { out.writeBoolean(false); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java index b396edd557483882c9d0a509386320ad1dfdda9c..dfd2aff803ba6a6377438d758b45d64ebd2f36c4 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/taskmanager/transferenvelope/RegisterTaskManagerResult.java @@ -14,10 +14,10 @@ package eu.stratosphere.nephele.taskmanager.transferenvelope; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.util.EnumUtils; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; public class RegisterTaskManagerResult implements IOReadableWritable { @@ -39,12 +39,12 @@ public class RegisterTaskManagerResult implements IOReadableWritable { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { EnumUtils.writeEnum(out, this.returnCode); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.returnCode = EnumUtils.readEnum(in, ReturnCode.class); } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java index 9f6542b62c19ded156ef4d4ca2f58763159e4600..a0f7f5b34d922c7f9229693f799f513d52e1e24e 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkNode.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.topology; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; @@ -22,6 +20,8 @@ import java.util.List; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; public class NetworkNode implements IOReadableWritable { @@ -209,7 +209,7 @@ public class NetworkNode implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.name = StringRecord.readString(in); @@ -228,7 +228,7 @@ public class NetworkNode implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { StringRecord.writeString(out, this.name); out.writeInt(this.childNodes.size()); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkTopology.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkTopology.java index f54a520c64c21dfe6bfafcc3cc9977d72a1d92b9..cba792d6bc8ce6449fda17a3ce50f3b750238fab 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkTopology.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/topology/NetworkTopology.java @@ -14,9 +14,7 @@ package eu.stratosphere.nephele.topology; import java.io.BufferedReader; -import java.io.DataInput; import java.io.DataInputStream; -import java.io.DataOutput; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -28,6 +26,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; public class NetworkTopology implements IOReadableWritable { @@ -120,14 +120,14 @@ public class NetworkTopology implements IOReadableWritable { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.rootNode.read(in); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.rootNode.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/FileRecord.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/FileRecord.java index 55b2ec6eab1976d76557c49b3cfca99cc7caa5df..211440783302314f57e12290a4514b63e6f2fe2f 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/FileRecord.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/FileRecord.java @@ -13,13 +13,13 @@ package eu.stratosphere.nephele.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; public class FileRecord implements IOReadableWritable { @@ -80,7 +80,7 @@ public class FileRecord implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.fileName = StringRecord.readString(in); final int newLength = in.readInt(); @@ -90,7 +90,7 @@ public class FileRecord implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { StringRecord.writeString(out, fileName); out.writeInt(this.bytes.length); out.write(this.bytes, 0, this.bytes.length); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/IntegerRecord.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/IntegerRecord.java index 09ca0b4d626fbbd9531e323c271af8b82ec9693c..814138469d6743d1acf4bf0e084c897b424a89bf 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/IntegerRecord.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/types/IntegerRecord.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * This class represents record for integer values. @@ -65,13 +65,13 @@ public class IntegerRecord implements IOReadableWritable { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Simply read the value from the stream this.value = in.readInt(); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Simply write the value to the stream out.writeInt(this.value); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableArrayList.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableArrayList.java index 4c1dc2bd5c9964e3d45883c1c0078b63718e8d53..2d6426938fb5a868fcecda945aa0d1564dcf71ab 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableArrayList.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableArrayList.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.util; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.StringUtils; /** @@ -59,7 +59,7 @@ public class SerializableArrayList extends ArrayLi @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(size()); final Iterator it = iterator(); @@ -76,7 +76,7 @@ public class SerializableArrayList extends ArrayLi @SuppressWarnings("unchecked") @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Make sure the list is empty clear(); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableHashMap.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableHashMap.java index fd1200ca7ce94f8947e9adff3586b403862f3f8b..1a87ac119c3dcf981d5a8cbf327ff0ed7faea196 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableHashMap.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/util/SerializableHashMap.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.util; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.HashMap; import java.util.Iterator; @@ -22,6 +20,8 @@ import java.util.Map; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.StringUtils; /** @@ -46,7 +46,7 @@ public class SerializableHashMap extends HashSet extends HashSet implements IOReadableWritable { @@ -28,12 +27,9 @@ public class DeserializationDelegate implements IOReadableWritable { private final TypeSerializer serializer; - private final InputViewDataInputWrapper wrapper; - - + public DeserializationDelegate(TypeSerializer serializer) { this.serializer = serializer; - this.wrapper = new InputViewDataInputWrapper(); } public void setInstance(T instance) { @@ -45,13 +41,12 @@ public class DeserializationDelegate implements IOReadableWritable { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { throw new IllegalStateException("Serialization method called on DeserializationDelegate."); } @Override - public void read(DataInput in) throws IOException { - this.wrapper.setDelegate(in); - this.instance = this.serializer.deserialize(this.instance, this.wrapper); + public void read(DataInputView in) throws IOException { + this.instance = this.serializer.deserialize(this.instance, in); } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/plugable/SerializationDelegate.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/plugable/SerializationDelegate.java index 7e940cfdef0859708b62a118c66aec7cb3b4f8f3..8eac86dc16acb36ce3094b1ac1661ac4b28d9e04 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/plugable/SerializationDelegate.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/plugable/SerializationDelegate.java @@ -13,13 +13,12 @@ package eu.stratosphere.pact.runtime.plugable; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.api.common.typeutils.TypeSerializer; import eu.stratosphere.core.io.IOReadableWritable; -import eu.stratosphere.core.memory.OutputViewDataOutputWrapper; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; public class SerializationDelegate implements IOReadableWritable { @@ -28,12 +27,9 @@ public class SerializationDelegate implements IOReadableWritable { private final TypeSerializer serializer; - private final OutputViewDataOutputWrapper wrapper; - - + public SerializationDelegate(TypeSerializer serializer) { this.serializer = serializer; - this.wrapper = new OutputViewDataOutputWrapper(); } public void setInstance(T instance) { @@ -45,14 +41,12 @@ public class SerializationDelegate implements IOReadableWritable { } @Override - public void write(DataOutput out) throws IOException { - this.wrapper.setDelegate(out); - this.serializer.serialize(this.instance, this.wrapper); + public void write(DataOutputView out) throws IOException { + this.serializer.serialize(this.instance, out); } - @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { throw new IllegalStateException("Deserialization method called on SerializationDelegate."); } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java index b44a4893be627b312b5a27216bafcfd1de62bb8b..def7591600533b056891323bef5e80add0f0a721 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/pact/runtime/task/util/TaskConfig.java @@ -15,9 +15,7 @@ package eu.stratosphere.pact.runtime.task.util; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.DataInput; import java.io.DataInputStream; -import java.io.DataOutput; import java.io.DataOutputStream; import java.io.IOException; import java.util.ArrayList; @@ -37,6 +35,10 @@ import eu.stratosphere.api.common.typeutils.TypeComparatorFactory; import eu.stratosphere.api.common.typeutils.TypePairComparatorFactory; import eu.stratosphere.api.common.typeutils.TypeSerializerFactory; import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import eu.stratosphere.pact.runtime.shipping.ShipStrategyType; import eu.stratosphere.pact.runtime.task.DriverStrategy; import eu.stratosphere.pact.runtime.task.PactDriver; @@ -535,7 +537,7 @@ public class TaskConfig { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream dos = new DataOutputStream(baos); try { - distribution.write(dos); + distribution.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException e) { throw new RuntimeException("Error serializing the DataDistribution: " + e.getMessage(), e); } @@ -569,7 +571,7 @@ public class TaskConfig { final DataInputStream in = new DataInputStream(bais); try { - distribution.read(in); + distribution.read(new InputViewDataInputStreamWrapper(in)); return distribution; } catch (Exception ex) { throw new RuntimeException("The deserialization of the encoded data distribution state caused an error" @@ -1261,13 +1263,13 @@ public class TaskConfig { // -------------------------------------------------------------------------------------------- @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.prefix = in.readUTF(); this.backingConfig.read(in); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeUTF(this.prefix); this.backingConfig.write(out); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java index dcdcbb2571d8e75d3772d833f120f3de544afcfd..9a3ca8c1f4a8422835d69fc2691a96ec9c86d849 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/ChannelCloseEvent.java @@ -13,21 +13,21 @@ package eu.stratosphere.runtime.io.channels; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.event.task.AbstractEvent; public final class ChannelCloseEvent extends AbstractEvent { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { // Nothing to do here } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // Nothing to do here } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java index e5f9589b730e1fe38ddd942c83c0ff7560a1ce12..c68ca4e463067833001b298c033659553a380ec1 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/channels/EndOfSuperstepEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.runtime.io.channels; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.event.task.AbstractEvent; /** @@ -27,8 +27,8 @@ public class EndOfSuperstepEvent extends AbstractEvent { public static final EndOfSuperstepEvent INSTANCE = new EndOfSuperstepEvent(); @Override - public void write(DataOutput out) throws IOException {} + public void write(DataOutputView out) throws IOException {} @Override - public void read(DataInput in) throws IOException {} + public void read(DataInputView in) throws IOException {} } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java index aeb50255f8279cac96766c2e7bd16e3f85c18144..0de4da46da7b6f44825abf234295ff03e67848e8 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/ConnectionInfoLookupResponse.java @@ -16,11 +16,11 @@ package eu.stratosphere.runtime.io.network; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.runtime.io.channels.ChannelID; public class ConnectionInfoLookupResponse implements IOReadableWritable { @@ -66,7 +66,7 @@ public class ConnectionInfoLookupResponse implements IOReadableWritable { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.returnCode = ReturnCode.values()[in.readInt()]; if (in.readBoolean()) { @@ -80,7 +80,7 @@ public class ConnectionInfoLookupResponse implements IOReadableWritable { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.returnCode.ordinal()); if (this.remoteTarget != null) { diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java index ab65b4ca9da244d61dcbe50898f460cbff7da001..c1883300fe2b12dd4a8ebc208a4054790d192f61 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/RemoteReceiver.java @@ -13,13 +13,13 @@ package eu.stratosphere.runtime.io.network; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; /** * Objects of this class uniquely identify a connection to a remote {@link TaskManager}. @@ -111,7 +111,7 @@ public final class RemoteReceiver implements IOReadableWritable { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { final InetAddress ia = this.connectionAddress.getAddress(); out.writeInt(ia.getAddress().length); @@ -123,7 +123,7 @@ public final class RemoteReceiver implements IOReadableWritable { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { final int addr_length = in.readInt(); final byte[] address = new byte[addr_length]; in.readFully(address); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java index 65287f76e1475bb0f7e0d58a360620ff7e8b482f..4e926efa6d14c0bc03d2387081c7c62d8d9cb496 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/network/SenderHintEvent.java @@ -13,12 +13,12 @@ package eu.stratosphere.runtime.io.network; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Arrays; import java.util.List; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.runtime.io.channels.ChannelID; @@ -65,7 +65,7 @@ public final class SenderHintEvent extends AbstractEvent { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.source.write(out); this.remoteReceiver.write(out); @@ -73,7 +73,7 @@ public final class SenderHintEvent extends AbstractEvent { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.source.read(in); this.remoteReceiver.read(in); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java index 696915fde86f0b436085bebcbffe7e5a0da10f1b..7f61cc0fc4ddce0d878ff98211dc24da3d9304d5 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/AdaptiveSpanningRecordDeserializer.java @@ -14,9 +14,9 @@ package eu.stratosphere.runtime.io.serialization; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; import eu.stratosphere.core.memory.MemorySegment; -import java.io.DataInput; import java.io.EOFException; import java.io.IOException; import java.io.UTFDataFormatException; @@ -116,7 +116,7 @@ public class AdaptiveSpanningRecordDeserializer im // ----------------------------------------------------------------------------------------------------------------- - private static final class NonSpanningWrapper implements DataInput { + private static final class NonSpanningWrapper implements DataInputView { private MemorySegment segment; @@ -339,11 +339,46 @@ public class AdaptiveSpanningRecordDeserializer im this.position += toSkip; return toSkip; } + + @Override + public void skipBytesToRead(int numBytes) throws IOException { + int skippedBytes = skipBytes(numBytes); + + if(skippedBytes < numBytes){ + throw new EOFException("Could not skip " + numBytes + " bytes."); + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if(b == null){ + throw new NullPointerException("Byte array b cannot be null."); + } + + if(off < 0){ + throw new IllegalArgumentException("The offset off cannot be negative."); + } + + if(len < 0){ + throw new IllegalArgumentException("The length len cannot be negative."); + } + + int toRead = Math.min(len, remaining()); + this.segment.get(this.position,b,off, toRead); + this.position += toRead; + + return toRead; + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } } // ----------------------------------------------------------------------------------------------------------------- - private static final class SpanningWrapper implements DataInput { + private static final class SpanningWrapper implements DataInputView { private final DataOutputSerializer serializationBuffer; @@ -525,5 +560,20 @@ public class AdaptiveSpanningRecordDeserializer im public String readUTF() throws IOException { return this.serializationReadBuffer.readUTF(); } + + @Override + public void skipBytesToRead(int numBytes) throws IOException { + this.serializationReadBuffer.skipBytesToRead(numBytes); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return this.serializationReadBuffer.read(b, off, len); + } + + @Override + public int read(byte[] b) throws IOException { + return this.serializationReadBuffer.read(b); + } } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java index a8a53fe39463894366fd03e7ed4ed25b5735485b..7e3312060dd7690740015384d07afab17991928e 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataInputDeserializer.java @@ -15,9 +15,9 @@ package eu.stratosphere.runtime.io.serialization; +import eu.stratosphere.core.memory.DataInputView; import eu.stratosphere.core.memory.MemoryUtils; -import java.io.DataInput; import java.io.EOFException; import java.io.IOException; import java.io.UTFDataFormatException; @@ -27,7 +27,7 @@ import java.nio.ByteOrder; /** * A simple and efficient deserializer for the {@link java.io.DataInput} interface. */ -public class DataInputDeserializer implements DataInput { +public class DataInputDeserializer implements DataInputView { private byte[] buffer; @@ -311,4 +311,48 @@ public class DataInputDeserializer implements DataInput { private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN); + + @Override + public void skipBytesToRead(int numBytes) throws IOException { + int skippedBytes = skipBytes(numBytes); + + if(skippedBytes < numBytes){ + throw new EOFException("Could not skip " + numBytes +" bytes."); + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if(b == null){ + throw new NullPointerException("Byte array b cannot be null."); + } + + if(off < 0){ + throw new IndexOutOfBoundsException("Offset cannot be negative."); + } + + if(len < 0){ + throw new IndexOutOfBoundsException("Length cannot be negative."); + } + + if(b.length - off < len){ + throw new IndexOutOfBoundsException("Byte array does not provide enough space to store requested data" + + "."); + } + + if(this.position >= this.end){ + return -1; + }else{ + int toRead = Math.min(this.end-this.position, len); + System.arraycopy(this.buffer,this.position,b,off,toRead); + this.position += toRead; + + return toRead; + } + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java index ce088f0abb409e54785362f0d5d59d1919f5be4e..ea4f6941f0b298d51da83418dcd5eaa0bddc9389 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/runtime/io/serialization/DataOutputSerializer.java @@ -14,9 +14,11 @@ **********************************************************************************************************************/ package eu.stratosphere.runtime.io.serialization; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.core.memory.MemoryUtils; -import java.io.DataOutput; +import java.io.EOFException; import java.io.IOException; import java.io.UTFDataFormatException; import java.nio.ByteBuffer; @@ -25,7 +27,7 @@ import java.nio.ByteOrder; /** * A simple and efficient serializer for the {@link java.io.DataOutput} interface. */ -public class DataOutputSerializer implements DataOutput { +public class DataOutputSerializer implements DataOutputView { private byte[] buffer; @@ -257,4 +259,23 @@ public class DataOutputSerializer implements DataOutput { private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class); private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN); + + @Override + public void skipBytesToWrite(int numBytes) throws IOException { + if(buffer.length - this.position < numBytes){ + throw new EOFException("Could not skip " + numBytes + " bytes."); + } + + this.position += numBytes; + } + + @Override + public void write(DataInputView source, int numBytes) throws IOException { + if(buffer.length - this.position < numBytes){ + throw new EOFException("Could not write " + numBytes + " bytes. Buffer overflow."); + } + + source.read(this.buffer, this.position, numBytes); + this.position += numBytes; + } } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java index 02190ca8985014b9be954ce22c1110343ca9c0a8..5299e7875c51c0e9ed515300bb4766cce6a30c7b 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerITCase.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.services.iomanager; -import java.io.DataInput; -import java.io.DataOutput; import java.io.EOFException; import java.io.File; import java.io.IOException; import java.util.List; import java.util.Random; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import junit.framework.Assert; import org.apache.commons.logging.Log; @@ -265,12 +265,12 @@ public class IOManagerITCase { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { value = in.readUTF(); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeUTF(this.value); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java index 7936a95c5a9c06bd1245012023bbb001d007bbfd..bcb0335fa5c2bfd469074e74d078d50b0c40679c 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/services/iomanager/IOManagerPerformanceBenchmark.java @@ -26,6 +26,8 @@ import java.nio.ByteBuffer; import java.nio.channels.FileChannel; import java.util.List; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import org.apache.commons.logging.Log; @@ -251,7 +253,7 @@ public class IOManagerPerformanceBenchmark int valsLeft = NUM_INTS_WRITTEN; while (valsLeft-- > 0) { rec.setValue(valsLeft); - rec.write(daos); + rec.write(new OutputViewDataOutputStreamWrapper(daos)); } daos.close(); daos = null; @@ -267,7 +269,7 @@ public class IOManagerPerformanceBenchmark valsLeft = NUM_INTS_WRITTEN; while (valsLeft-- > 0) { - rec.read(dais); + rec.read(new InputViewDataInputStreamWrapper(dais)); } dais.close(); dais = null; diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/CommonTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/CommonTestUtils.java index 8e4a68cbc42f8b6b57a644305aaae32b202ecf7b..679cc0e2035d30752983587ef684bebfd8709281 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/CommonTestUtils.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/CommonTestUtils.java @@ -25,6 +25,8 @@ import java.io.IOException; import eu.stratosphere.configuration.ConfigConstants; import eu.stratosphere.configuration.GlobalConfiguration; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; /** * This class contains auxiliary methods for unit tests in the Nephele common module. @@ -95,7 +97,7 @@ public class CommonTestUtils { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream dos = new DataOutputStream(baos); - original.write(dos); + original.write(new OutputViewDataOutputStreamWrapper(dos)); final String className = original.getClass().getName(); if (className == null) { @@ -130,7 +132,7 @@ public class CommonTestUtils { final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); final DataInputStream dis = new DataInputStream(bais); - copy.read(dis); + copy.read(new InputViewDataInputStreamWrapper(dis)); return copy; } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ManagementTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ManagementTestUtils.java index 7d60e8f914832a1aaa2c3cbacf410a1b598d7ae4..ebc739bea944708937bc741b4d633bbf586222e1 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ManagementTestUtils.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ManagementTestUtils.java @@ -23,6 +23,8 @@ import java.io.File; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; /** * This class contains auxiliary methods for unit tests in the Nephele management module. @@ -61,7 +63,7 @@ public final class ManagementTestUtils { final DataOutputStream dos = new DataOutputStream(baos); try { - original.write(dos); + original.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException ioe) { fail(ioe.getMessage()); } @@ -100,7 +102,7 @@ public final class ManagementTestUtils { final DataInputStream dis = new DataInputStream(bais); try { - copy.read(dis); + copy.read(new InputViewDataInputStreamWrapper(dis)); } catch (IOException e) { fail(e.getMessage()); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java index 59de8cc33f646f7871b0934f8d00079d3e3435db..e1e692593401949924b96f09779729be081c2cc4 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/ServerTestUtils.java @@ -29,6 +29,8 @@ import java.util.jar.JarOutputStream; import java.util.jar.Manifest; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import eu.stratosphere.nephele.jobmanager.JobManagerITCase; import eu.stratosphere.nephele.protocols.ExtendedManagementProtocol; @@ -197,7 +199,7 @@ public final class ServerTestUtils { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); final DataOutputStream dos = new DataOutputStream(baos); - original.write(dos); + original.write(new OutputViewDataOutputStreamWrapper(dos)); final String className = original.getClass().getName(); if (className == null) { @@ -232,7 +234,7 @@ public final class ServerTestUtils { final ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); final DataInputStream dis = new DataInputStream(bais); - copy.read(dis); + copy.read(new InputViewDataInputStreamWrapper(dis)); return copy; } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java index fb0da9180d341784f4841fb60c4f2a48feefa8d3..6dc199f1815017b0e2f49c44f56d1e8e2c4e0fa8 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileInputVertex.java @@ -16,8 +16,6 @@ package eu.stratosphere.nephele.util.tasks; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -29,6 +27,8 @@ import eu.stratosphere.core.fs.FileStatus; import eu.stratosphere.core.fs.FileSystem; import eu.stratosphere.core.fs.Path; import eu.stratosphere.core.io.InputSplit; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.jobgraph.AbstractJobInputVertex; import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobVertexID; @@ -94,7 +94,7 @@ public final class JobFileInputVertex extends AbstractJobInputVertex { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); // Read path of the input file @@ -106,7 +106,7 @@ public final class JobFileInputVertex extends AbstractJobInputVertex { } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); // Write out the path of the input file diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java index 593b520aee4153c897f15b7157fc12a2b0f83b53..c1e30f259290d9455bbf16770fa3471fa7ee66b2 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/nephele/util/tasks/JobFileOutputVertex.java @@ -15,11 +15,11 @@ package eu.stratosphere.nephele.util.tasks; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.fs.Path; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.jobgraph.AbstractJobOutputVertex; import eu.stratosphere.nephele.jobgraph.JobGraph; import eu.stratosphere.nephele.jobgraph.JobVertexID; @@ -83,7 +83,7 @@ public class JobFileOutputVertex extends AbstractJobOutputVertex { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); // Read path of the input file @@ -95,7 +95,7 @@ public class JobFileOutputVertex extends AbstractJobOutputVertex { } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); // Write out the path of the input file diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java index 8204eedcd053ccc508b037e8469ce348558e32cb..1a8edddee482b491ab8f111c04bb9d1f239d1a84 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/iterative/event/EventWithAggregatorsTest.java @@ -22,6 +22,8 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import org.junit.Assert; import org.junit.Test; @@ -94,7 +96,7 @@ public class EventWithAggregatorsTest { try { ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream out = new DataOutputStream(baos); - event.write(out); + event.write(new OutputViewDataOutputStreamWrapper(out)); out.flush(); byte[] data = baos.toByteArray(); @@ -103,7 +105,7 @@ public class EventWithAggregatorsTest { DataInputStream in = new DataInputStream(new ByteArrayInputStream(data)); IterationEventWithAggregators newEvent = event.getClass().newInstance(); - newEvent.read(in); + newEvent.read(new InputViewDataInputStreamWrapper(in)); in.close(); return newEvent; diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java index 13fbfbc08af3d38ee29c34a530d84d538e21c629..757175979f2993d155aafc6046eb4a0d1f337b22 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/OutputEmitterTest.java @@ -19,6 +19,8 @@ import java.io.IOException; import java.io.PipedInputStream; import java.io.PipedOutputStream; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import junit.framework.TestCase; @@ -356,9 +358,9 @@ public class OutputEmitterTest extends TestCase { rec = new Record(1); rec.setField(0, new IntValue()); - rec.write(out); + rec.write(new OutputViewDataOutputStreamWrapper(out)); rec = new Record(); - rec.read(in); + rec.read(new InputViewDataInputStreamWrapper(in)); } catch (IOException e) { fail("Test erroneous"); diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java index 2bd5d9849169b2ddeda6e45e56211b2706d71568..724c800117aa2f7d6260423d84e7babbc7a58346 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/pact/runtime/task/util/RecordOutputEmitterTest.java @@ -20,6 +20,8 @@ import java.io.PipedInputStream; import java.io.PipedOutputStream; import java.util.Random; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import junit.framework.Assert; import junit.framework.TestCase; @@ -298,9 +300,9 @@ public class RecordOutputEmitterTest extends TestCase { rec = new Record(1); rec.setField(0, new IntValue()); - rec.write(out); + rec.write(new OutputViewDataOutputStreamWrapper(out)); rec = new Record(); - rec.read(in); + rec.read(new InputViewDataInputStreamWrapper(in)); } catch (IOException e) { fail("Test erroneous"); diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java index 1c6270a7acd9048874cb55b822e8a6bd6c4c3407..bf0c30baef6c4f3cab4c7ad4015cf3cd464a4317 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/network/netty/InboundEnvelopeDecoderTest.java @@ -13,6 +13,8 @@ package eu.stratosphere.runtime.io.network.netty; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.core.memory.MemorySegment; import eu.stratosphere.nephele.event.task.AbstractEvent; import eu.stratosphere.nephele.jobgraph.JobID; @@ -37,8 +39,6 @@ import org.mockito.MockitoAnnotations; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -792,12 +792,12 @@ public class InboundEnvelopeDecoderTest { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(id); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { id = in.readLong(); } @@ -830,12 +830,12 @@ public class InboundEnvelopeDecoderTest { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(id); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { id = in.readLong(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java index 637b7d58709c1e0a0b75c21b9c70eff593cd1272..1517b7bfc7c66b8b0d78f8d7c1945f5386d435a2 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/SpanningRecordSerializerTest.java @@ -13,6 +13,8 @@ package eu.stratosphere.runtime.io.serialization; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.core.memory.MemorySegment; import eu.stratosphere.runtime.io.Buffer; import eu.stratosphere.runtime.io.serialization.RecordSerializer.SerializationResult; @@ -22,8 +24,6 @@ import eu.stratosphere.runtime.io.serialization.types.Util; import junit.framework.Assert; import org.junit.Test; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Random; @@ -88,11 +88,11 @@ public class SpanningRecordSerializerTest { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { } @Override diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java index 7aadc7cfcfed9ebac09a6d52544fe024af6edc33..8ecacd3c4b90dba7d65ebf55e799528fc0180d5f 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/AsciiStringType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -51,12 +52,12 @@ public class AsciiStringType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeUTF(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readUTF(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java index 32b2ba3aaf6691a472e01ca243610a310a9b2c99..f4cef3e08a9bf60f3614b8d0c4de2da77b4d5e38 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/BooleanType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class BooleanType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readBoolean(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java index cb29a5c0a7c0a9a73218b5473fce36e51e03f594..82c9ee1313ac1d836c3b813ccab0e8e8441d7f95 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteArrayType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Arrays; import java.util.Random; @@ -47,13 +48,13 @@ public class ByteArrayType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.data.length); out.write(this.data); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { final int len = in.readInt(); this.data = new byte[len]; in.readFully(this.data); diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java index 2b683d2135b261c4e70ee8f0c853170fbd9f70fd..e5384607140115a1497ae2e3d9a4840f690202b8 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteSubArrayType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Arrays; import java.util.Random; @@ -52,13 +53,13 @@ public class ByteSubArrayType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.len); out.write(this.data, 0, this.len); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.len = in.readInt(); in.readFully(this.data, 0, this.len); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java index 52abdcb16987d5e3a0b76776164089f91f364571..2a3a4ae7987ac464a3571ddab2d4b4ba033de7ae 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ByteType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class ByteType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeByte(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readByte(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java index 25df737c313d293cbd642114d2762a56483e7a96..5e7e418540bca294900c193eaf248ed6778cabaf 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/CharType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class CharType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeChar(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readChar(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java index 1046e75c3a3ee6f8f61c0612beb3f177e3727865..91db7036b25b32a5d4050f3bca5f1b9d5336e8fd 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/DoubleType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class DoubleType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readDouble(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java index 4a50873de58090f9c95c9ea125431f7fb4edc67a..0cee3040fc13b6f616ce4d8cf1d0be66533c3a00 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/FloatType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class FloatType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeFloat(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readFloat(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java index 50a3546fffd268265c04fa12b79cc10f66652931..be84e077492b862583385a6d81afa5f081889427 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/IntType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class IntType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readInt(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java index 1402fb50f37df03a09b249694477087719e91cfe..aa109c22c93b744c486272baf09b9a02d64550bc 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/LongType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class LongType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readLong(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java index 7711e88a9f6976d57ca7db78cc400c496c1c9bd4..e834b2d07cf02ea7f731f34146714527a675eff2 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/ShortType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class ShortType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeShort(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readShort(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java index 9a1f1fbeb41a956130d5e89168cf1beb0e25793b..afa7de23902945d63c186c02f817474c72c01d24 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedByteType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class UnsignedByteType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeByte(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readUnsignedByte(); } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java index ac80ef79b0c39a3d367a9614401244405dc95408..4bc787d2b69184d40aa162b465468d1bce5b70e4 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/types/UnsignedShortType.java @@ -13,8 +13,9 @@ package eu.stratosphere.runtime.io.serialization.types; -import java.io.DataInput; -import java.io.DataOutput; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; + import java.io.IOException; import java.util.Random; @@ -41,12 +42,12 @@ public class UnsignedShortType implements SerializationTestType { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeShort(this.value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.value = in.readUnsignedShort(); } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java index 18bc3e9c67a7ad9312b496d807ee1c0446616b30..da7292ddff44d02929a4ad70b3ae286975202cce 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/accumulators/AccumulatorITCase.java @@ -13,8 +13,6 @@ package eu.stratosphere.test.accumulators; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.io.Serializable; import java.util.Collection; @@ -22,6 +20,8 @@ import java.util.Iterator; import java.util.Map; import java.util.Set; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.test.util.RecordAPITestBase; import org.junit.Assert; import org.junit.runner.RunWith; @@ -308,12 +308,12 @@ public class AccumulatorITCase extends RecordAPITestBase { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.set.write(out); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.set.read(in); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/BooleanValue.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/BooleanValue.java index f35382c8bfc6ce05cb640daea0eb0e7707a2d93a..1ab872158a9a310cd3af53a64052dfe551181866 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/BooleanValue.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/BooleanValue.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.iterative.nephele.danglingpagerank; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; public class BooleanValue implements Value { @@ -40,12 +40,12 @@ public class BooleanValue implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(value); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { value = in.readBoolean(); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/LongArrayView.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/LongArrayView.java index 38904d92fcd5cdd3f8a09908eff534c2e8f9d45d..87d5e83355ec1bd0e41367e6c1bb4f90604d76c1 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/LongArrayView.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/LongArrayView.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.iterative.nephele.danglingpagerank; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; public class LongArrayView implements Value { @@ -65,14 +65,14 @@ public class LongArrayView implements Value { } } - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(numEntries); for (int n = 0; n < numEntries; n++) { out.writeLong(entries[n]); } } - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { numEntries = in.readInt(); ensureCapacity(); for (int n = 0; n < numEntries; n++) { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/PageRankStats.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/PageRankStats.java index cbde78c58cf88a31c9375e4cdedc168b1ace6c6e..d73c81eeccbbedc7631d104f67a260e8163691d1 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/PageRankStats.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/iterative/nephele/danglingpagerank/PageRankStats.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.iterative.nephele.danglingpagerank; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; public class PageRankStats implements Value { @@ -86,7 +86,7 @@ public class PageRankStats implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(diff); out.writeDouble(rank); out.writeDouble(danglingRank); @@ -98,7 +98,7 @@ public class PageRankStats implements Value { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { diff = in.readDouble(); rank = in.readDouble(); danglingRank = in.readDouble(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java index a9bda2bc1abad01bbca2494fb8bd768fd7ac0cc7..31f3222fde370eea55716b6f7b6d96471209b867 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobTests/GlobalSortingMixedOrderITCase.java @@ -13,8 +13,6 @@ package eu.stratosphere.test.recordJobTests; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -29,6 +27,8 @@ import eu.stratosphere.api.common.operators.Order; import eu.stratosphere.api.common.operators.Ordering; import eu.stratosphere.api.java.record.io.CsvInputFormat; import eu.stratosphere.api.java.record.io.CsvOutputFormat; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.test.util.RecordAPITestBase; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Key; @@ -110,14 +110,14 @@ public class GlobalSortingMixedOrderITCase extends RecordAPITestBase { public TripleIntDistribution() {} @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(this.ascendingI1); out.writeBoolean(this.ascendingI2); out.writeBoolean(this.ascendingI3); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.ascendingI1 = in.readBoolean(); this.ascendingI2 = in.readBoolean(); this.ascendingI3 = in.readBoolean(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java index 317d5fceb7be22e1e7f28ed7d007193ffaf045f5..71fca15da85ac01af5cb7accd30f2c5165a491fc 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/LongArrayView.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.recordJobs.graph.pageRankUtil; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; public class LongArrayView implements Value { @@ -66,14 +66,14 @@ public class LongArrayView implements Value { } } - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(numEntries); for (int n = 0; n < numEntries; n++) { out.writeLong(entries[n]); } } - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { numEntries = in.readInt(); ensureCapacity(); for (int n = 0; n < numEntries; n++) { diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java index 20558ed175b5f2f19260d2c4216726abb1111f0f..8cf9491be250287e5cfd6894b96644fe40daaafe 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/graph/pageRankUtil/PageRankStats.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.recordJobs.graph.pageRankUtil; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; public class PageRankStats implements Value { @@ -71,7 +71,7 @@ public class PageRankStats implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(diff); out.writeDouble(rank); out.writeDouble(danglingRank); @@ -81,7 +81,7 @@ public class PageRankStats implements Value { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { diff = in.readDouble(); rank = in.readDouble(); danglingRank = in.readDouble(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java index 31966e7e61236336c31ed3126fb7a4a0ab990327..e6be052725cde174d0893eeed1714b7e1a18bc23 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansBroadcast.java @@ -14,8 +14,6 @@ package eu.stratosphere.test.recordJobs.kmeans; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -37,6 +35,8 @@ import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; import eu.stratosphere.client.LocalExecutor; import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; @@ -147,14 +147,14 @@ public class KMeansBroadcast implements Program, ProgramDescription { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(x); out.writeDouble(y); out.writeDouble(z); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { x = in.readDouble(); y = in.readDouble(); z = in.readDouble(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java index 9600856c53f4654c2a93f00b934fe86d988cf86e..23d1c49dd5a1952d5edb1b501915926dd811f3b3 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/KMeansSingleStep.java @@ -14,8 +14,6 @@ package eu.stratosphere.test.recordJobs.kmeans; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -35,6 +33,8 @@ import eu.stratosphere.api.java.record.operators.MapOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.DoubleValue; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; @@ -128,14 +128,14 @@ public class KMeansSingleStep implements Program, ProgramDescription { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeDouble(x); out.writeDouble(y); out.writeDouble(z); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { x = in.readDouble(); y = in.readDouble(); z = in.readDouble(); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java index e2b0420eb2796ab8804f77fc103cbe95bd47fe64..e2baf7854f37d02fff2c32a157a91045a1c78f09 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/kmeans/udfs/CoordVector.java @@ -12,10 +12,10 @@ **********************************************************************************************************************/ package eu.stratosphere.test.recordJobs.kmeans.udfs; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Key; /** @@ -99,7 +99,7 @@ public final class CoordVector implements Key { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { int length = in.readInt(); this.coordinates = new double[length]; for (int i = 0; i < length; i++) { @@ -109,7 +109,7 @@ public final class CoordVector implements Key { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeInt(this.coordinates.length); for (int i = 0; i < this.coordinates.length; i++) { out.writeDouble(this.coordinates[i]); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java index 14f0611d96b4719064843d5293b9df56deb9519e..f08dce5684215ce5a9e730311952e9b0471c9280 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraDistribution.java @@ -13,10 +13,9 @@ package eu.stratosphere.test.recordJobs.sort.tsUtil; -import java.io.DataInput; -import java.io.DataOutput; - import eu.stratosphere.api.common.distributions.DataDistribution; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Key; /** @@ -49,8 +48,8 @@ public class TeraDistribution implements DataDistribution { } @Override - public void write(DataOutput out) {} + public void write(DataOutputView out) {} @Override - public void read(DataInput in) {} + public void read(DataInputView in) {} } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java index f541a07d6428070874ed20fd72aed12aa5cd0561..c86563489a8d9cd3490b96ccf062d2ca8f67e3d8 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraKey.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.recordJobs.sort.tsUtil; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Key; /** @@ -78,14 +78,14 @@ public final class TeraKey implements Key { @Override - public void write(DataOutput out) throws IOException + public void write(DataOutputView out) throws IOException { out.write(this.key, this.offset, KEY_SIZE); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { in.readFully(this.key, 0, KEY_SIZE); this.offset = 0; } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java index 714019c5c578b4ac7f0130de0e3d6eadab41fee4..8d35456c587f68a7d225595127b090c89895db70 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/sort/tsUtil/TeraValue.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.recordJobs.sort.tsUtil; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; /** @@ -76,12 +76,12 @@ public final class TeraValue implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.write(this.value, this.offset, VALUE_SIZE); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { in.readFully(this.value, 0, VALUE_SIZE); this.offset = 0; } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java index d9720e8ebed8b4c679e69d8916b542a0c6ed03f0..921ace06771470a62a4de3229389493ed8eda6f6 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/util/Tuple.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.recordJobs.util; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; public class Tuple implements Value { @@ -555,7 +555,7 @@ public class Tuple implements Value { // ------------------------------------------------------------------------ @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { // read the bytes int numBytes = in.readInt(); if (numBytes > 0) { @@ -576,7 +576,7 @@ public class Tuple implements Value { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // write the bytes int numBytes = (numCols > 0 ? offsets[numCols] : 0); out.writeInt(numBytes); diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java index 811c26629d2bf8765f785fcfc9ec3ca60ee74c88..f7bc696318860e51f9faae4d94ff594bdaaea400 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/recordJobs/wordcount/WordCountAccumulators.java @@ -13,8 +13,6 @@ package eu.stratosphere.test.recordJobs.wordcount; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.io.Serializable; import java.util.Iterator; @@ -40,6 +38,8 @@ import eu.stratosphere.api.java.record.operators.ReduceOperator; import eu.stratosphere.api.java.record.operators.ReduceOperator.Combinable; import eu.stratosphere.client.LocalExecutor; import eu.stratosphere.configuration.Configuration; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.util.SerializableHashSet; import eu.stratosphere.types.IntValue; import eu.stratosphere.types.Record; @@ -215,12 +215,12 @@ public class WordCountAccumulators implements Program, ProgramDescription { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.set.write(out); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.set.read(in); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java index 9b202c625f80e3c288edd4c5abc38f03e16fe003..9b3e1be23225c1c070e677ece33a379fe9af90ff 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/runtime/NetworkStackThroughput.java @@ -13,10 +13,10 @@ package eu.stratosphere.test.runtime; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.junit.After; @@ -248,12 +248,12 @@ public class NetworkStackThroughput { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.write(this.buf); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { in.readFully(this.buf); } } diff --git a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java index e43ce90d16d6ff8e760c855aa77a0e470e1dbba6..26976f5ac19c3e27ded574740dad3a41bae1d441 100644 --- a/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java +++ b/stratosphere-tests/src/test/java/eu/stratosphere/test/testPrograms/util/tests/TupleTest.java @@ -19,6 +19,8 @@ import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import org.junit.Assert; import org.junit.Test; @@ -636,14 +638,14 @@ public class TupleTest { Tuple t = new Tuple(); t.addAttribute("Hello world!"); try { - t.write(dos); + t.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException e1) { e1.printStackTrace(); } t.addAttribute("2ndAttribute"); try { - t.write(dos); + t.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException e) { e.printStackTrace(); } @@ -653,7 +655,7 @@ public class TupleTest { t = new Tuple(ba, of2, 5); try { - t.write(dos); + t.write(new OutputViewDataOutputStreamWrapper(dos)); } catch (IOException e) { e.printStackTrace(); } @@ -678,7 +680,7 @@ public class TupleTest { t = new Tuple(); try { - t.read(dis); + t.read(new InputViewDataInputStreamWrapper(dis)); Assert.assertTrue(t.getNumberOfColumns() == 1); Assert.assertTrue(t.getStringValueAt(0).equals("Hello world!")); @@ -689,7 +691,7 @@ public class TupleTest { t = new Tuple(); try { - t.read(dis); + t.read(new InputViewDataInputStreamWrapper(dis)); Assert.assertTrue(t.getNumberOfColumns() == 2); Assert.assertTrue(t.getStringValueAt(0).equals("Hello world!")); @@ -701,7 +703,7 @@ public class TupleTest { t = new Tuple(); try { - t.read(dis); + t.read(new InputViewDataInputStreamWrapper(dis)); Assert.assertTrue(t.getNumberOfColumns() == 5); Assert.assertTrue(t.getStringValueAt(0).equals("attr1"));