From bf0d7cf252f51fc3e79e2655d91de1781940bd5b Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Sun, 29 Jun 2014 19:08:31 +0200 Subject: [PATCH 1/2] Replaced DataInput and DataOutput with DataInputView and DataOutputView --- .../stratosphere/api/avro/AvroBaseValue.java | 8 +- .../record/datatypes/WritableWrapper.java | 8 +- .../mapred/wrapper/HadoopInputSplit.java | 8 +- .../mapreduce/wrapper/HadoopInputSplit.java | 8 +- .../common/accumulators/DoubleCounter.java | 9 +- .../api/common/accumulators/Histogram.java | 8 +- .../api/common/accumulators/IntCounter.java | 9 +- .../api/common/accumulators/LongCounter.java | 9 +- .../distributions/SimpleDistribution.java | 8 +- .../SimpleIntegerDistribution.java | 8 +- .../UniformDoubleDistribution.java | 8 +- .../UniformIntegerDistribution.java | 8 +- .../api/common/io/BinaryInputFormat.java | 11 +- .../api/common/io/BinaryOutputFormat.java | 9 +- .../stratosphere/api/common/io/BlockInfo.java | 8 +- .../api/common/io/SerializedInputFormat.java | 4 +- .../api/common/io/SerializedOutputFormat.java | 6 +- .../configuration/Configuration.java | 8 +- .../stratosphere/core/fs/FileInputSplit.java | 8 +- .../java/eu/stratosphere/core/fs/Path.java | 8 +- .../core/io/GenericInputSplit.java | 9 +- .../core/io/IOReadableWritable.java | 17 +-- .../core/io/LocatableInputSplit.java | 9 +- .../eu/stratosphere/core/io/StringRecord.java | 7 +- .../core/memory/DataInputView.java | 22 ++++ ...a => InputViewDataInputStreamWrapper.java} | 79 ++++++------ .../InputViewObjectInputStreamWrapper.java | 120 ++++++++++++++++++ ...=> OutputViewDataOutputStreamWrapper.java} | 80 ++++++------ .../OutputViewObjectOutputStreamWrapper.java | 111 ++++++++++++++++ .../eu/stratosphere/types/BooleanValue.java | 6 +- .../java/eu/stratosphere/types/ByteValue.java | 6 +- .../java/eu/stratosphere/types/CharValue.java | 6 +- .../eu/stratosphere/types/DoubleValue.java | 6 +- .../eu/stratosphere/types/FloatValue.java | 6 +- .../java/eu/stratosphere/types/IntValue.java | 6 +- .../java/eu/stratosphere/types/ListValue.java | 8 +- .../java/eu/stratosphere/types/LongValue.java | 6 +- .../java/eu/stratosphere/types/MapValue.java | 8 +- .../java/eu/stratosphere/types/NullValue.java | 6 +- .../main/java/eu/stratosphere/types/Pair.java | 8 +- .../java/eu/stratosphere/types/Record.java | 74 ++++++++++- .../eu/stratosphere/types/ShortValue.java | 6 +- .../eu/stratosphere/types/StringValue.java | 4 +- .../api/common/io/BinaryInputFormatTest.java | 4 +- .../api/common/io/SequentialFormatTest.java | 3 +- .../SimpleDataDistributionTest.java | 6 +- .../core/testutils/CommonTestUtils.java | 6 +- .../types/CollectionsDataTypeTest.java | 14 +- .../types/PrimitiveDataTypeTest.java | 50 ++++---- .../eu/stratosphere/types/RecordTest.java | 73 ++++++----- .../api/java/io/CollectionInputFormat.java | 13 +- .../record/io/ExternalProcessInputSplit.java | 8 +- .../eu/stratosphere/nephele/AbstractID.java | 8 +- .../nephele/client/AbstractJobResult.java | 8 +- .../nephele/client/JobProgressResult.java | 8 +- .../nephele/client/JobSubmissionResult.java | 9 +- .../ChannelDeploymentDescriptor.java | 8 +- .../deployment/GateDeploymentDescriptor.java | 8 +- .../deployment/TaskDeploymentDescriptor.java | 8 +- .../nephele/event/job/AbstractEvent.java | 8 +- .../event/job/ExecutionStateChangeEvent.java | 8 +- .../nephele/event/job/JobEvent.java | 8 +- .../nephele/event/job/RecentJobEvent.java | 8 +- .../event/job/VertexAssignmentEvent.java | 8 +- .../nephele/event/job/VertexEvent.java | 8 +- .../nephele/event/task/IntegerTaskEvent.java | 9 +- .../nephele/event/task/StringTaskEvent.java | 8 +- .../LibraryCacheProfileRequest.java | 8 +- .../LibraryCacheProfileResponse.java | 8 +- .../librarycache/LibraryCacheUpdate.java | 8 +- .../nephele/instance/HardwareDescription.java | 8 +- .../instance/InstanceConnectionInfo.java | 8 +- .../eu/stratosphere/nephele/ipc/Client.java | 3 +- .../nephele/ipc/ConnectionHeader.java | 8 +- .../java/eu/stratosphere/nephele/ipc/RPC.java | 8 +- .../eu/stratosphere/nephele/ipc/Server.java | 8 +- .../nephele/jobgraph/AbstractJobVertex.java | 8 +- .../nephele/jobgraph/JobGraph.java | 13 +- .../splitassigner/InputSplitWrapper.java | 8 +- .../managementgraph/ManagementGraph.java | 8 +- .../ManagementGroupVertex.java | 8 +- .../managementgraph/ManagementVertex.java | 8 +- .../InternalExecutionVertexProfilingData.java | 8 +- ...nalExecutionVertexThreadProfilingData.java | 8 +- .../types/InternalInputGateProfilingData.java | 8 +- .../types/InternalInstanceProfilingData.java | 8 +- .../InternalOutputGateProfilingData.java | 8 +- .../impl/types/ProfilingDataContainer.java | 8 +- .../types/InputGateProfilingEvent.java | 8 +- .../types/InstanceProfilingEvent.java | 8 +- .../types/OutputGateProfilingEvent.java | 8 +- .../profiling/types/ProfilingEvent.java | 8 +- .../types/SingleInstanceProfilingEvent.java | 8 +- .../profiling/types/ThreadProfilingEvent.java | 8 +- .../profiling/types/VertexProfilingEvent.java | 8 +- .../accumulators/AccumulatorEvent.java | 8 +- .../memorymanager/AbstractPagedInputView.java | 51 ++++++-- .../taskmanager/AbstractTaskResult.java | 8 +- .../taskmanager/TaskExecutionState.java | 8 +- .../RegisterTaskManagerResult.java | 8 +- .../nephele/topology/NetworkNode.java | 8 +- .../nephele/topology/NetworkTopology.java | 8 +- .../nephele/types/FileRecord.java | 8 +- .../nephele/types/IntegerRecord.java | 8 +- .../nephele/util/SerializableArrayList.java | 8 +- .../nephele/util/SerializableHashMap.java | 8 +- .../nephele/util/SerializableHashSet.java | 8 +- .../event/IterationEventWithAggregators.java | 14 +- .../iterative/event/TerminationEvent.java | 8 +- .../iterative/event/WorkerDoneEvent.java | 8 +- .../plugable/DeserializationDelegate.java | 17 +-- .../plugable/SerializationDelegate.java | 18 +-- .../pact/runtime/task/util/TaskConfig.java | 14 +- .../io/channels/ChannelCloseEvent.java | 8 +- .../io/channels/EndOfSuperstepEvent.java | 8 +- .../network/ConnectionInfoLookupResponse.java | 8 +- .../runtime/io/network/RemoteReceiver.java | 8 +- .../runtime/io/network/SenderHintEvent.java | 8 +- .../AdaptiveSpanningRecordDeserializer.java | 56 +++++++- .../serialization/DataInputDeserializer.java | 48 ++++++- .../serialization/DataOutputSerializer.java | 25 +++- .../services/iomanager/IOManagerITCase.java | 8 +- .../IOManagerPerformanceBenchmark.java | 6 +- .../nephele/util/CommonTestUtils.java | 6 +- .../nephele/util/ManagementTestUtils.java | 6 +- .../nephele/util/ServerTestUtils.java | 6 +- .../util/tasks/JobFileInputVertex.java | 8 +- .../util/tasks/JobFileOutputVertex.java | 8 +- .../event/EventWithAggregatorsTest.java | 6 +- .../runtime/task/util/OutputEmitterTest.java | 6 +- .../task/util/RecordOutputEmitterTest.java | 6 +- .../netty/InboundEnvelopeDecoderTest.java | 12 +- .../SpanningRecordSerializerTest.java | 8 +- .../serialization/types/AsciiStringType.java | 9 +- .../io/serialization/types/BooleanType.java | 9 +- .../io/serialization/types/ByteArrayType.java | 9 +- .../serialization/types/ByteSubArrayType.java | 9 +- .../io/serialization/types/ByteType.java | 9 +- .../io/serialization/types/CharType.java | 9 +- .../io/serialization/types/DoubleType.java | 9 +- .../io/serialization/types/FloatType.java | 9 +- .../io/serialization/types/IntType.java | 9 +- .../io/serialization/types/LongType.java | 9 +- .../io/serialization/types/ShortType.java | 9 +- .../serialization/types/UnsignedByteType.java | 9 +- .../types/UnsignedShortType.java | 9 +- .../test/accumulators/AccumulatorITCase.java | 8 +- .../danglingpagerank/BooleanValue.java | 8 +- .../danglingpagerank/LongArrayView.java | 8 +- .../danglingpagerank/PageRankStats.java | 8 +- .../GlobalSortingMixedOrderITCase.java | 8 +- .../graph/pageRankUtil/LongArrayView.java | 8 +- .../graph/pageRankUtil/PageRankStats.java | 8 +- .../recordJobs/kmeans/KMeansBroadcast.java | 8 +- .../recordJobs/kmeans/KMeansSingleStep.java | 8 +- .../recordJobs/kmeans/udfs/CoordVector.java | 8 +- .../sort/tsUtil/TeraDistribution.java | 9 +- .../test/recordJobs/sort/tsUtil/TeraKey.java | 8 +- .../recordJobs/sort/tsUtil/TeraValue.java | 8 +- .../test/recordJobs/util/Tuple.java | 8 +- .../wordcount/WordCountAccumulators.java | 8 +- .../test/runtime/NetworkStackThroughput.java | 8 +- .../testPrograms/util/tests/TupleTest.java | 14 +- 163 files changed, 1255 insertions(+), 772 deletions(-) rename stratosphere-core/src/main/java/eu/stratosphere/core/memory/{InputViewDataInputWrapper.java => InputViewDataInputStreamWrapper.java} (60%) create mode 100644 stratosphere-core/src/main/java/eu/stratosphere/core/memory/InputViewObjectInputStreamWrapper.java rename stratosphere-core/src/main/java/eu/stratosphere/core/memory/{OutputViewDataOutputWrapper.java => OutputViewDataOutputStreamWrapper.java} (61%) create mode 100644 stratosphere-core/src/main/java/eu/stratosphere/core/memory/OutputViewObjectOutputStreamWrapper.java 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 7ed7ab6684c72..8768e17795756 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 AvroBaseValue(T datum) { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // the null flag if (datum() == null) { out.writeBoolean(false); @@ -64,7 +64,7 @@ public void write(DataOutput out) throws IOException { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { // the null flag if (in.readBoolean()) { diff --git a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java index d74eb74d2abc3..23348781d3d62 100644 --- a/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java +++ b/stratosphere-addons/hadoop-compatibility/src/main/java/eu/stratosphere/hadoopcompatibility/mapred/record/datatypes/WritableWrapper.java @@ -13,10 +13,10 @@ package eu.stratosphere.hadoopcompatibility.mapred.record.datatypes; -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 eu.stratosphere.types.Value; @@ -42,13 +42,13 @@ public T 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 980d0e6db9905..dfa21be047779 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 HadoopInputSplit(org.apache.hadoop.mapred.InputSplit hInputSplit, JobConf } @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 985ad59c0f520..ebaeaf31d8bbe 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 HadoopInputSplit(org.apache.hadoop.mapreduce.InputSplit mapreduceInputSpl } @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 void write(DataOutput out) throws IOException { } @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 e2c1d469d608b..14f9db3712516 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 void resetLocal() { } @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 aa72cbb13d67a..121a1e2357ecd 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 String toString() { } @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 void write(DataOutput out) throws IOException { } @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 341b09e0a962e..37ac56884421d 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 void resetLocal() { } @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 eba78b1240d57..b4274eb6cfc5b 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 void resetLocal() { } @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 7c1ff8bd9d3bc..71752f82b4c19 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 Key[] getBucketBoundary(int bucketNum, int totalNumBuckets) { } @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 void write(DataOutput out) throws IOException { @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 e7606a8b05a7e..a1e1df9e1d93e 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 IntValue[] getBucketBoundary(int bucketNum, int totalNumBuckets) { } @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 void write(DataOutput out) throws IOException { } @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 706500bd3b87a..b630e5de3edb6 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 int getNumberOfFields() { } @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 c9bfddaccd40d..8ad0e76f400f2 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 int getNumberOfFields() { } @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 72ed645cab61f..b0efa3aefe0f8 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.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 @@ protected SequentialStatistics createStatistics(List files, FileBase 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 void open(FileInputSplit split) throws IOException { // 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 T nextRecord(T record) throws IOException { 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 b4922040ec067..7f4e7b3c37d6e 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 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 void open(int taskNumber, int numTasks) throws IOException { 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 @@ private void writeInfo() throws IOException { 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 709d704092a9b..9084c5de6b63b 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 void setFirstRecordStart(long firstRecordStart) { } @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 000bfd785f0a6..428988c78cc9b 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 d11083b1e1ee5..e08b307764edc 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 iterator() { * @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 void read(final DataInput in) throws IOException { * @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 ac9e0913aa8c4..fd105892498f7 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 String toString() { * @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 { this.value = in.readLong(); } @@ -97,7 +95,7 @@ public void read(final DataInput in) throws IOException { * @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.writeLong(this.value); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/MapValue.java b/stratosphere-core/src/main/java/eu/stratosphere/types/MapValue.java index 37485031da6db..6680b84a9377d 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/MapValue.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/MapValue.java @@ -13,14 +13,14 @@ package eu.stratosphere.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.Map; import java.util.Set; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.util.ReflectionUtil; /** @@ -73,7 +73,7 @@ public MapValue(Map map) { * @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 void read(final DataInput in) throws IOException { * @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 fc7e996bd8efc..fa29d8fc03e23 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 String toString() { // -------------------------------------------------------------------------------------------- @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 3906984edcb60..e2d5fea0af4bd 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 String toString() { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.first.read(in); this.second.read(in); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { this.first.write(out); this.second.write(out); } diff --git a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java index 73671fad0bd51..78e90f0065992 100644 --- a/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java +++ b/stratosphere-core/src/main/java/eu/stratosphere/types/Record.java @@ -1070,7 +1070,7 @@ private final void serializeHeader(final InternalDeSerializer serializer, final // -------------------------------------------------------------------------------------------- @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 void write(DataOutput out) throws IOException { } @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 @@ private final void initFields(final byte[] data, final int begin, final int len) } /** - * 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 @@ private static final int readVarLengthInt(DataInput in) throws IOException { 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 void read(DataInput in) throws IOException { /** * 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 int skipBytes(int n) throws IOException { 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 @@ private final void resize(int minCapacityAdd) throws IOException { 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 52d6fd24d69a8..d2429865ca50f 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 String toString() { // -------------------------------------------------------------------------------------------- @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 cc970e84852c6..1a133f62ebe49 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 Appendable append(StringValue csq, int start, int end) { // -------------------------------------------------------------------------------------------- @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 void read(final DataInput in) throws IOException { } @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 7d10a9fd134aa..e8200132ab21e 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 @@ private static final class MyBinaryInputFormat extends BinaryInputFormat 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 420fdf6a28138..164899ce62567 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.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 void calcRawDataSize() throws IOException { 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 c87d2e14acfe5..eb83f8c23f440 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.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 void testWriteRead() { 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 void testWriteRead() { 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 dcfdb805e8eee..fcdddcbb333ea 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 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 static T createCopy(final T original) thro 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 static T createCopy(final T original) thro 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 9e81d742b8e78..163473e545133 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.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 void testPair() { 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 void testPactMap() { // 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 void testPactList() { // 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 7e893ed1c5f1c..723ea18a3a913 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.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 void testIntValue() { 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 void testDoubleValue() { 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 void testStringValue() { // 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 void testPactNull() { 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 2f14257017dc5..89b3ce63b3149 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.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 void testEmptyRecordSerialization() 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 void testUpdateBinaryRepresentations() 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 void testDeSerialization() 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 void testClear() throws IOException 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 void blackBoxTests() } } - 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 @@ static final void blackboxTestRecordWithValues(Value[] values, Random rnd, DataI 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 82f275573f415..db75ecd28b5e6 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.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 @@ private void writeObject(ObjectOutputStream out) throws IOException { 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 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE 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 f07ddd5efc22b..e6677a47d6f5f 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 String getExternalProcessCommand() { @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 648c8dcde454b..fa9b740be62e6 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 int hashCode() { } @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 13b337d38187b..931f1a2aa13a3 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 AbstractJobResult() { @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 void read(final DataInput in) throws IOException { @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 5f4e34bf2d8b0..d6d05ba6cb666 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 JobProgressResult() { @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 void read(final DataInput in) throws IOException { @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 9913be529fc49..9443b0b9c1030 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 JobSubmissionResult() { @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 4068e5b3ade5a..daacd13c4f54b 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 ChannelDeploymentDescriptor() { @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 void write(final DataOutput out) throws IOException { @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 02d6578a67621..7ad26dd3ab35f 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 GateDeploymentDescriptor() { @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 void write(final DataOutput out) throws IOException { @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 06fabde00c69a..42826008fb6d4 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 TaskDeploymentDescriptor() { @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 void write(final DataOutput out) throws IOException { @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 80e65ffe3df4f..cffec9d37d302 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 AbstractEvent() { @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 void read(final DataInput in) throws IOException { @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 a2cac79440ca8..cf451ab6b04b1 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 ExecutionState getNewExecutionState() { @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 void read(final DataInput in) throws IOException { @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 420784abc16a1..b231978e0e547 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 JobEvent() { @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 void read(final DataInput in) throws IOException { @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 a477c7a81c4f4..914adc374ae31 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 long getSubmissionTimestamp() { @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 void read(final DataInput in) throws IOException { @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 c86c12bb758a4..4b980ed7f7660 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 String getInstanceName() { } @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 void read(final DataInput in) throws IOException { @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 940f997ff4f0d..03f15257a0883 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 VertexEvent() { @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 void read(final DataInput in) throws IOException { @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 04e7b18a3beae..43e42e972c951 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 int getInteger() { @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 70797674cd622..68e1ef599bcee 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 String getString() { @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 dc5001286d4ed..82e86868c9b16 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 void setRequiredLibraries(final String[] requiredLibraries) { @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 void read(final DataInput in) throws IOException { @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 3a2c92ce326f1..7c1ec2666adc2 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 boolean isCached(final int pos) { @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 void read(final DataInput in) throws IOException { @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 05473c72019ed..6f7c404983138 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 LibraryCacheUpdate() { @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 83d650ac4eda1..2ca6068bc132e 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 HardwareDescription() { } @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 void write(final DataOutput out) throws IOException { } @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 257421bc814e5..d04a5aa15a004 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; /** @@ -202,7 +202,7 @@ public String domainName() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { final int addr_length = in.readInt(); byte[] address = new byte[addr_length]; @@ -222,7 +222,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { out.writeInt(this.inetAddress.getAddress().length); out.write(this.inetAddress.getAddress()); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java index ea7bd4a117fad..3e915ccf664bf 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Client.java @@ -21,6 +21,7 @@ import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.StringRecord; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; import eu.stratosphere.nephele.net.NetUtils; import eu.stratosphere.nephele.util.IOUtils; import eu.stratosphere.runtime.io.serialization.DataOutputSerializer; @@ -514,7 +515,7 @@ private void receiveResponse() { } catch (IllegalAccessException e) { LOG.error(e); } - value.read(in); // read value + value.read(new InputViewDataInputStreamWrapper(in)); // read value } call.setValue(value); } else if (state == Status.ERROR.state) { diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/ConnectionHeader.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/ConnectionHeader.java index d97dceb9939d5..78cffd3b4daed 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/ConnectionHeader.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/ConnectionHeader.java @@ -19,12 +19,12 @@ package eu.stratosphere.nephele.ipc; -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; /** * The IPC connection header sent by the client to the server @@ -49,14 +49,14 @@ public ConnectionHeader(String protocol) { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { this.protocol = StringRecord.readString(in); } @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { StringRecord.writeString(out, this.protocol); } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/RPC.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/RPC.java index 8ff35484bb4bd..f887673ea2392 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/RPC.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/RPC.java @@ -19,8 +19,6 @@ package eu.stratosphere.nephele.ipc; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.lang.reflect.InvocationHandler; import java.lang.reflect.InvocationTargetException; @@ -34,6 +32,8 @@ import javax.net.SocketFactory; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -102,7 +102,7 @@ public IOReadableWritable[] getParameters() { // TODO: See if type safety can be improved here @SuppressWarnings("unchecked") - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.methodName = StringRecord.readString(in); this.parameters = new IOReadableWritable[in.readInt()]; @@ -140,7 +140,7 @@ public void read(DataInput in) throws IOException { } } - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { StringRecord.writeString(out, methodName); out.writeInt(parameterClasses.length); for (int i = 0; i < parameterClasses.length; i++) { diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Server.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Server.java index 7929e5ffdee90..6757c81224551 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Server.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/ipc/Server.java @@ -52,6 +52,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; +import eu.stratosphere.core.memory.InputViewDataInputStreamWrapper; +import eu.stratosphere.core.memory.OutputViewDataOutputStreamWrapper; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -875,7 +877,7 @@ public int readAndProcess() throws IOException, InterruptedException { // / Reads the connection header following version private void processProtocol() throws IOException { DataInputStream in = new DataInputStream(new ByteArrayInputStream(data.array())); - header.read(in); + header.read(new InputViewDataInputStreamWrapper(in)); try { String protocolClassName = header.getProtocol(); if (protocolClassName != null) { @@ -894,7 +896,7 @@ private void processData() throws IOException, InterruptedException { IOReadableWritable invocation = newInstance(invocationClass); // read param - invocation.read(dis); + invocation.read(new InputViewDataInputStreamWrapper(dis)); Call call = new Call(id, invocation, this); callQueue.put(call); // queue the call; maybe blocked here @@ -1048,7 +1050,7 @@ private void setupResponse(ByteArrayOutputStream response, Call call, Status sta } else { out.writeBoolean(true); StringRecord.writeString(out, rv.getClass().getName()); - rv.write(out); + rv.write(new OutputViewDataOutputStreamWrapper(out)); } } else { diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java index cc936d9ed68db..5ccb4281d9dc9 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/AbstractJobVertex.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.jobgraph; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import org.apache.commons.lang.Validate; import eu.stratosphere.configuration.Configuration; @@ -374,7 +374,7 @@ public JobVertexID getID() { @SuppressWarnings("unchecked") @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { if (jobGraph == null) { throw new IOException("jobGraph is null, cannot deserialize"); @@ -454,7 +454,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Number of subtasks out.writeInt(this.numberOfSubtasks); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java index 3d14d0a34d0b9..b88fad0bce2ce 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/JobGraph.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.jobgraph; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -34,6 +32,8 @@ import eu.stratosphere.core.fs.Path; 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.util.ClassUtils; @@ -465,7 +465,7 @@ public AbstractJobVertex areVertexDegreesCorrect() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read job id this.jobID.read(in); @@ -553,7 +553,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write job ID this.jobID.write(out); @@ -598,7 +598,8 @@ public void write(final DataOutput out) throws IOException { * @throws IOException * thrown if an error occurs while writing to the stream */ - private void writeRequiredJarFiles(final DataOutput out, final AbstractJobVertex[] jobVertices) throws IOException { + private void writeRequiredJarFiles(final DataOutputView out, final AbstractJobVertex[] jobVertices) throws + IOException { // Now check if all the collected jar files really exist final FileSystem fs = FileSystem.getLocalFileSystem(); @@ -643,7 +644,7 @@ private void writeRequiredJarFiles(final DataOutput out, final AbstractJobVertex * @throws IOException * thrown if an error occurs while reading the stream */ - private void readRequiredJarFiles(final DataInput in) throws IOException { + private void readRequiredJarFiles(final DataInputView in) throws IOException { // Do jar files follow; final int numJars = in.readInt(); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitWrapper.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitWrapper.java index 811b919f0c86d..ac18a84cb2a2b 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitWrapper.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobmanager/splitassigner/InputSplitWrapper.java @@ -13,13 +13,13 @@ package eu.stratosphere.nephele.jobmanager.splitassigner; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import eu.stratosphere.core.io.IOReadableWritable; import eu.stratosphere.core.io.InputSplit; 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.jobgraph.JobID; import eu.stratosphere.util.StringUtils; @@ -68,7 +68,7 @@ public InputSplitWrapper() { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write the job ID this.jobID.write(out); @@ -90,7 +90,7 @@ public void write(final DataOutput out) throws IOException { @SuppressWarnings("unchecked") @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read the job ID this.jobID.read(in); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java index fab720df23714..54710f3bbf3fc 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGraph.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.managementgraph; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -28,6 +26,8 @@ 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.runtime.io.channels.ChannelType; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.nephele.util.EnumUtils; @@ -406,7 +406,7 @@ public List getGroupVerticesInReverseTopologicalOrder() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read job ID this.jobID.read(in); @@ -485,7 +485,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write job ID this.jobID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java index b98a153646927..afe64ec243fe4 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementGroupVertex.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.managementgraph; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; @@ -23,6 +21,8 @@ import java.util.Map; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.runtime.io.channels.ChannelType; import eu.stratosphere.nephele.util.EnumUtils; @@ -342,7 +342,7 @@ public int getStageNumber() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { int numberOfForwardLinks = in.readInt(); for (int i = 0; i < numberOfForwardLinks; i++) { @@ -359,7 +359,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write the number of forward links out.writeInt(this.forwardEdges.size()); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java index a270700cd7077..c0a72b156e652 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/managementgraph/ManagementVertex.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.managementgraph; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayList; 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; import eu.stratosphere.nephele.execution.ExecutionState; import eu.stratosphere.nephele.util.EnumUtils; import eu.stratosphere.util.StringUtils; @@ -266,7 +266,7 @@ public String getOptMessage() { } @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { // Read the execution state this.executionState = EnumUtils.readEnum(in, ExecutionState.class); @@ -288,7 +288,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { // Write the execution state EnumUtils.writeEnum(out, this.executionState); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexProfilingData.java index a0288357a4e3f..df3e7db30fd14 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexProfilingData.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexProfilingData.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.impl.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.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -47,14 +47,14 @@ public JobID getJobID() { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.jobId.read(in); this.executionVertexID.read(in); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.jobId.write(out); this.executionVertexID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java index f1064f6101f1e..ac40ae96c9b73 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.impl.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.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -48,7 +48,7 @@ public InternalExecutionVertexThreadProfilingData() { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); @@ -60,7 +60,7 @@ public void read(DataInput in) throws IOException { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInputGateProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInputGateProfilingData.java index 376c7ce09df19..80f9b7aa3a2e9 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInputGateProfilingData.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInputGateProfilingData.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.impl.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.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -50,7 +50,7 @@ public InternalInputGateProfilingData(JobID jobID, ExecutionVertexID executionVe } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.jobId.read(in); this.executionVertexID.read(in); @@ -60,7 +60,7 @@ public void read(DataInput in) throws IOException { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.jobId.write(out); this.executionVertexID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInstanceProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInstanceProfilingData.java index a39f59715cfac..3256be2a1e4fd 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInstanceProfilingData.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalInstanceProfilingData.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.impl.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.nephele.instance.InstanceConnectionInfo; public class InternalInstanceProfilingData implements InternalProfilingData { @@ -145,7 +145,7 @@ public long getTransmittedBytes() { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.freeMemory = in.readLong(); this.ioWaitCPU = in.readInt(); @@ -166,7 +166,7 @@ public void read(DataInput in) throws IOException { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeLong(this.freeMemory); out.writeInt(this.ioWaitCPU); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalOutputGateProfilingData.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalOutputGateProfilingData.java index b6a19863967bb..df4bb58cd1e88 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalOutputGateProfilingData.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/InternalOutputGateProfilingData.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.impl.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.nephele.executiongraph.ExecutionVertexID; import eu.stratosphere.nephele.jobgraph.JobID; @@ -50,7 +50,7 @@ public InternalOutputGateProfilingData(JobID jobID, ExecutionVertexID executionV } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.jobId.read(in); this.executionVertexID.read(in); @@ -60,7 +60,7 @@ public void read(DataInput in) throws IOException { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.jobId.write(out); this.executionVertexID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/ProfilingDataContainer.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/ProfilingDataContainer.java index 1d7ad29cddb11..49112cd51a036 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/ProfilingDataContainer.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/impl/types/ProfilingDataContainer.java @@ -13,8 +13,6 @@ package eu.stratosphere.nephele.profiling.impl.types; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.ArrayDeque; import java.util.Iterator; @@ -22,6 +20,8 @@ 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; public class ProfilingDataContainer implements IOReadableWritable { @@ -43,7 +43,7 @@ public void clear() { @SuppressWarnings("unchecked") @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { final int numberOfRecords = in.readInt(); for (int i = 0; i < numberOfRecords; i++) { @@ -83,7 +83,7 @@ public Iterator getIterator() { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { // Write the number of records out.writeInt(this.queuedProfilingData.size()); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InputGateProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InputGateProfilingEvent.java index 9c9b6b1b6a226..c5a8d53b8993d 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InputGateProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InputGateProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.jobgraph.JobID; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; @@ -96,7 +96,7 @@ public int getNoRecordsAvailableCounter() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); this.gateIndex = in.readInt(); @@ -105,7 +105,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); out.writeInt(this.gateIndex); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InstanceProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InstanceProfilingEvent.java index fb4b460254e26..ffd21ee6ca4ae 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InstanceProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/InstanceProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.jobgraph.JobID; /** @@ -300,7 +300,7 @@ public final long getTransmittedBytes() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); this.profilingInterval = in.readInt(); @@ -324,7 +324,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); out.writeInt(this.profilingInterval); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/OutputGateProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/OutputGateProfilingEvent.java index b134c29d98ac3..a8cd49f579587 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/OutputGateProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/OutputGateProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.jobgraph.JobID; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; @@ -94,7 +94,7 @@ public int getChannelCapacityExhausted() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); this.gateIndex = in.readInt(); @@ -103,7 +103,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); out.writeInt(this.gateIndex); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ProfilingEvent.java index 1d2681b18a13a..bc2454b5d9da3 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.event.job.AbstractEvent; import eu.stratosphere.nephele.event.job.ManagementEvent; import eu.stratosphere.nephele.jobgraph.JobID; @@ -86,7 +86,7 @@ public long getProfilingTimestamp() { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); this.jobID = new JobID(); @@ -97,7 +97,7 @@ public void read(final DataInput in) throws IOException { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); this.jobID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/SingleInstanceProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/SingleInstanceProfilingEvent.java index 09e0b50ffb7fc..901d5ea1843bb 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/SingleInstanceProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/SingleInstanceProfilingEvent.java @@ -13,11 +13,11 @@ package eu.stratosphere.nephele.profiling.types; -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; /** @@ -98,7 +98,7 @@ public String getInstanceName() { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); this.instanceName = StringRecord.readString(in); @@ -106,7 +106,7 @@ public void read(DataInput in) throws IOException { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); StringRecord.writeString(out, this.instanceName); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ThreadProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ThreadProfilingEvent.java index e33f6fdb6766b..57cbf49cee126 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ThreadProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/ThreadProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.jobgraph.JobID; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; @@ -91,7 +91,7 @@ public int getWaitedTime() { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); this.userTime = in.readInt(); @@ -102,7 +102,7 @@ public void read(DataInput in) throws IOException { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); out.writeInt(this.userTime); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/VertexProfilingEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/VertexProfilingEvent.java index 2c0fb35a0a702..e6e9ee9e17b03 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/VertexProfilingEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/profiling/types/VertexProfilingEvent.java @@ -13,10 +13,10 @@ package eu.stratosphere.nephele.profiling.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.nephele.jobgraph.JobID; import eu.stratosphere.nephele.managementgraph.ManagementVertexID; @@ -65,7 +65,7 @@ public int getProfilingInterval() { @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { super.read(in); this.vertexID = new ManagementVertexID(); @@ -76,7 +76,7 @@ public void read(DataInput in) throws IOException { @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { super.write(out); this.vertexID.write(out); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/accumulators/AccumulatorEvent.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/accumulators/AccumulatorEvent.java index 08f18a20f9512..be5429b175f7b 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/accumulators/AccumulatorEvent.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/accumulators/AccumulatorEvent.java @@ -13,14 +13,14 @@ package eu.stratosphere.nephele.services.accumulators; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.util.HashMap; import java.util.Map; import eu.stratosphere.api.common.accumulators.Accumulator; import eu.stratosphere.core.io.IOReadableWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.nephele.execution.librarycache.LibraryCacheManager; import eu.stratosphere.nephele.jobgraph.JobID; import eu.stratosphere.util.StringUtils; @@ -60,7 +60,7 @@ public JobID getJobID() { } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { out.writeBoolean(this.useUserClassLoader); jobID.write(out); out.writeInt(accumulators.size()); @@ -74,7 +74,7 @@ public void write(DataOutput out) throws IOException { @SuppressWarnings("unchecked") @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.useUserClassLoader = in.readBoolean(); jobID = new JobID(); jobID.read(in); diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java index 435f84f84a27a..c6ca7b9a21c2c 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java @@ -183,44 +183,69 @@ protected void clear() { // -------------------------------------------------------------------------------------------- // Data Input Specific methods // -------------------------------------------------------------------------------------------- - + @Override - public void readFully(byte[] b) throws IOException { - readFully(b, 0, b.length); + public int read(byte[] b) throws IOException{ + return read(b,0,b.length); } @Override - public void readFully(byte[] b, int off, int len) throws IOException { + public int read(byte[] b, int off, int len) throws IOException{ if (off < 0 || len < 0 || off + len > 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 ffb2bb20451b1..093f9b2ed4773 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 ReturnCode getReturnCode() { @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 void read(DataInput in) throws IOException { @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 9f372939176d0..edd442efb1713 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 JobID getJobID() { @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 void read(final DataInput in) throws IOException { @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 b396edd557483..dfd2aff803ba6 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 RegisterTaskManagerResult(ReturnCode returnCode){ @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 9f6542b62c19d..a0f7f5b34d922 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 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 int getDistance(final String nodeName) { @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 void read(final DataInput in) throws IOException { @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 f54a520c64c21..cba792d6bc8ce 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.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 NetworkNode getRootNode() { } @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 55b2ec6eab197..2114407833023 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 @@ private void setCapacity(final int len, final boolean keepData) { @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 void read(final DataInput in) throws IOException { @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 09ca0b4d626fb..814138469d674 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 void setValue(final int value) { } @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 4c1dc2bd5c996..2d6426938fb5a 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 SerializableArrayList(final int initialCapacity) { @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 void write(final DataOutput out) throws IOException { @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 fd1200ca7ce94..1a87ac119c3dc 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 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 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 T getInstance() { } @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 7e940cfdef085..8eac86dc16acb 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 T getInstance() { } @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 b44a4893be627..def7591600533 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 @@ 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.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 void setOutputDataDistribution(DataDistribution distribution, int outputN 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 DataDistribution getOutputDataDistribution(int outputNum, final ClassLoad 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 Set keySet() { // -------------------------------------------------------------------------------------------- @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 dcdcbb2571d8e..9a3ca8c1f4a84 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 e5f9589b730e1..c68ca4e463067 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 aeb50255f8279..0de4da46da7b6 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 ChannelID getLocalTarget() { } @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 void read(DataInput in) throws IOException { } @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 ab65b4ca9da24..c1883300fe2b1 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 boolean equals(final Object obj) { @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 void write(final DataOutput out) throws IOException { @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 65287f76e1475..4e926efa6d14c 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 RemoteReceiver getRemoteReceiver() { @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 void write(final DataOutput out) throws IOException { @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 696915fde86f0..7f61cc0fc4ddc 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 boolean hasUnfinishedData() { // ----------------------------------------------------------------------------------------------------------------- - private static final class NonSpanningWrapper implements DataInput { + private static final class NonSpanningWrapper implements DataInputView { private MemorySegment segment; @@ -339,11 +339,46 @@ public final int skipBytes(int n) throws IOException { 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 String readLine() throws IOException { 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 a8a53fe394638..7e3312060dd76 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 @@ /** * 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 int skipBytes(int n) throws IOException { 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 ce088f0abb409..ea4f6941f0b29 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 @@ /** * 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 @@ private final void resize(int minCapacityAdd) throws IOException { 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 02190ca898501..5299e7875c51c 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 Value(String val) { } @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 7936a95c5a9c0..bcb0335fa5c2b 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.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 @@ private final void speedTestStream(int bufferSize) throws IOException { 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 @@ private final void speedTestStream(int bufferSize) throws IOException { 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 8e4a68cbc42f8..679cc0e2035d3 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 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 static T createCopy(final T original) thro 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 static T createCopy(final T original) thro 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 7d60e8f914832..ebc739bea9447 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.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 static IOReadableWritable createCopy(IOReadableWritable original) { 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 static IOReadableWritable createCopy(IOReadableWritable original) { 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 59de8cc33f646..e1e6925934019 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.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 static T createCopy(final T original) thro 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 static T createCopy(final T original) thro 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 fb0da9180d341..6dc199f181501 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.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 Path getFilePath() { } @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 void read(final DataInput in) throws IOException { } @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 593b520aee415..c1e30f259290d 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 Path getFilePath() { } @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 void read(final DataInput in) throws IOException { } @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 8204eedcd053c..1a8edddee482b 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.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 @@ private IterationEventWithAggregators pipeThroughSerialization(IterationEventWit 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 @@ private IterationEventWithAggregators pipeThroughSerialization(IterationEventWit 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 13fbfbc08af3d..757175979f299 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.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 void testWrongKeyClass() { 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 2bd5d9849169b..724c800117aa2 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.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 void testWrongKeyClass() { 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 1c6270a7acd90..bf0c30baef6c4 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.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 TestEvent1(long id) { } @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 TestEvent2(long id) { } @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 637b7d58709c1..1517b7bfc7c66 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 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 int length() { } @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 7aadc7cfcfed9..8ecacd3c4b90d 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 int length() { } @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 32b2ba3aaf669..f4cef3e08a9bf 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 int length() { } @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 cb29a5c0a7c0a..82c9ee1313ac1 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 int length() { } @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 2b683d2135b26..e538460714011 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 int length() { } @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 52abdcb16987d..2a3a4ae7987ac 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 int length() { } @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 25df737c313d2..5e7e418540bca 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 int length() { } @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 1046e75c3a3ee..91db7036b25b3 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 int length() { } @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 4a50873de5809..0cee3040fc13b 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 int length() { } @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 50a3546fffd26..be84e077492b8 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 int length() { } @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 1402fb50f37df..aa109c22c93b7 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 int length() { } @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 7711e88a9f697..e834b2d07cf02 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 int length() { } @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 9a1f1fbeb41a9..afa7de2390294 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 int length() { } @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 ac80ef79b0c39..4bc787d2b6918 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 int length() { } @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 18bc3e9c67a7a..da7292ddff44d 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.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 void merge(Accumulator> other) { } @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 f35382c8bfc6c..1ab872158a9a3 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 void set(boolean 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 38904d92fcd5c..87d5e83355ec1 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 @@ private void ensureCapacity() { } } - 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 cbde78c58cf88..d73c81eeccbbe 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 double finalDiff() { } @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 void write(DataOutput out) throws IOException { } @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 a9bda2bc1abad..31f3222fde370 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.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 TripleIntDistribution(Order orderI1, Order orderI2, Order orderI3) { 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 317d5fceb7be2..71fca15da85ac 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 @@ private void ensureCapacity() { } } - 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 20558ed175b5f..8cf9491be2502 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 long edges() { } @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 void write(DataOutput out) throws IOException { } @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 31966e7e61236..e6be052725cde 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.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 void clear() { } @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 9600856c53f46..23d1c49dd5a19 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.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 void clear() { } @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 e2b0420eb2796..e2baf7854f37d 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 double computeEuclidianDistance(CoordVector cv) { @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 void read(DataInput in) throws IOException { @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 14f0611d96b47..f08dce5684215 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 int getNumberOfFields() { } @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 f541a07d64280..c86563489a8d9 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 void setValue(final byte[] data, int offset) { @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 714019c5c578b..8d35456c587f6 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 void setValue(final byte[] data, int offset) { } @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 d9720e8ebed8b..921ace0677147 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 void setContents(byte[] bytes, int offset, int len, char delimiter) // ------------------------------------------------------------------------ @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 void read(DataInput in) throws IOException { } @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 811c26629d2bf..f7bc696318860 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.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 void merge(Accumulator> other) { } @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 9b202c625f80e..9b3e1be23225c 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 SpeedTestRecord() { } @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 e43ce90d16d6f..26976f5ac19c3 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.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 void testSerialization() { 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 void testSerialization() { 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 void testSerialization() { 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 void testSerialization() { 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 void testSerialization() { t = new Tuple(); try { - t.read(dis); + t.read(new InputViewDataInputStreamWrapper(dis)); Assert.assertTrue(t.getNumberOfColumns() == 5); Assert.assertTrue(t.getStringValueAt(0).equals("attr1")); From e4c3d7c76bdd126198712864808e85ec6af4115d Mon Sep 17 00:00:00 2001 From: Till Rohrmann Date: Mon, 7 Jul 2014 16:20:07 +0200 Subject: [PATCH 2/2] Added test cases for AbstractPagedInputView, fixed problem with hbase. --- .../addons/hbase/TableInputSplit.java | 8 +- .../addons/hbase/common/HBaseKey.java | 8 +- .../addons/hbase/common/HBaseResult.java | 8 +- .../memorymanager/AbstractPagedInputView.java | 5 +- .../io/serialization/PagedViewsTest.java | 259 +++++++++++++++++- 5 files changed, 273 insertions(+), 15 deletions(-) diff --git a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/TableInputSplit.java b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/TableInputSplit.java index 382befad9f8fe..bcf5d007641c6 100644 --- a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/TableInputSplit.java +++ b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/TableInputSplit.java @@ -13,10 +13,10 @@ package eu.stratosphere.addons.hbase; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; +import eu.stratosphere.core.memory.DataOutputView; +import eu.stratosphere.core.memory.DataInputView; import eu.stratosphere.core.io.LocatableInputSplit; /** @@ -103,7 +103,7 @@ public byte[] getEndRow() { @Override - public void write(final DataOutput out) throws IOException { + public void write(final DataOutputView out) throws IOException { super.write(out); @@ -134,7 +134,7 @@ public void write(final DataOutput out) throws IOException { @Override - public void read(final DataInput in) throws IOException { + public void read(final DataInputView in) throws IOException { super.read(in); diff --git a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseKey.java b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseKey.java index 700db054f31ae..eba09cda95023 100644 --- a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseKey.java +++ b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseKey.java @@ -13,12 +13,12 @@ package eu.stratosphere.addons.hbase.common; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Key; /** @@ -52,12 +52,12 @@ public void setWritable(ImmutableBytesWritable writable) { // -------------------------------------------------------------------------------------------- @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.writable.write(out); } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.writable.readFields(in); } diff --git a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseResult.java b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseResult.java index a8a2df958e8f4..a827551cb03f0 100644 --- a/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseResult.java +++ b/stratosphere-addons/hbase/src/main/java/eu/stratosphere/addons/hbase/common/HBaseResult.java @@ -13,12 +13,12 @@ package eu.stratosphere.addons.hbase.common; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import org.apache.hadoop.hbase.client.Result; +import eu.stratosphere.core.memory.DataInputView; +import eu.stratosphere.core.memory.DataOutputView; import eu.stratosphere.types.Value; public class HBaseResult implements Value { @@ -53,12 +53,12 @@ public String getStringData() { } @Override - public void read(DataInput in) throws IOException { + public void read(DataInputView in) throws IOException { this.result.readFields(in); } @Override - public void write(DataOutput out) throws IOException { + public void write(DataOutputView out) throws IOException { this.result.write(out); } } diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java index c6ca7b9a21c2c..35badeff65c0b 100644 --- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java +++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/services/memorymanager/AbstractPagedInputView.java @@ -222,6 +222,7 @@ public int read(byte[] b, int off, int len) throws IOException{ try { advance(); }catch(EOFException eof){ + this.positionInSegment += toRead; return bytesRead; } remaining = this.limitInSegment - this.positionInSegment; @@ -244,8 +245,8 @@ public void readFully(byte[] b) throws IOException { 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."); + if(bytesRead < len){ + throw new EOFException("There is no enough data left in the DataInputView."); } } diff --git a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java index 817c0e6438c20..e128344672042 100644 --- a/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java +++ b/stratosphere-runtime/src/test/java/eu/stratosphere/runtime/io/serialization/PagedViewsTest.java @@ -21,10 +21,13 @@ import eu.stratosphere.runtime.io.serialization.types.Util; import org.junit.Test; +import java.io.EOFException; import java.io.IOException; import java.util.*; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class PagedViewsTest { @@ -69,6 +72,260 @@ public void testRandomTypes() { } } + @Test + public void testReadFully() { + int bufferSize = 100; + byte[] expected = new byte[bufferSize]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(bufferSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[bufferSize]; + + try { + inputView.readFully(buffer); + } catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(inputView.getCurrentPositionInSegment(), bufferSize); + assertArrayEquals(expected, buffer); + } + + @Test + public void testReadFullyAcrossSegments() { + int bufferSize = 100; + int segmentSize = 30; + byte[] expected = new byte[bufferSize]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(segmentSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[bufferSize]; + + try { + inputView.readFully(buffer); + } catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(inputView.getCurrentPositionInSegment(), bufferSize % segmentSize); + assertArrayEquals(expected, buffer); + } + + @Test + public void testReadAcrossSegments() { + int bufferSize = 100; + int bytes2Write = 75; + int segmentSize = 30; + byte[] expected = new byte[bytes2Write]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(segmentSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[bufferSize]; + int bytesRead = 0; + + try { + bytesRead = inputView.read(buffer); + } catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(bytes2Write, bytesRead); + assertEquals(inputView.getCurrentPositionInSegment(), bytes2Write % segmentSize); + + byte[] tempBuffer = new byte[bytesRead]; + System.arraycopy(buffer,0,tempBuffer,0,bytesRead); + assertArrayEquals(expected, tempBuffer); + } + + @Test + public void testEmptyingInputView() { + int bufferSize = 100; + int bytes2Write = 75; + int segmentSize = 30; + byte[] expected = new byte[bytes2Write]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(segmentSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[bufferSize]; + int bytesRead = 0; + + try { + bytesRead = inputView.read(buffer); + } catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(bytes2Write, bytesRead); + + byte[] tempBuffer = new byte[bytesRead]; + System.arraycopy(buffer,0,tempBuffer,0,bytesRead); + assertArrayEquals(expected, tempBuffer); + + try{ + bytesRead = inputView.read(buffer); + }catch(IOException e){ + e.printStackTrace(); + fail("Unexpected exception: Input view should be empty and thus return -1."); + } + + assertEquals(-1, bytesRead); + assertEquals(inputView.getCurrentPositionInSegment(), bytes2Write % segmentSize); + } + + @Test + public void testReadFullyWithNotEnoughData() { + int bufferSize = 100; + int bytes2Write = 99; + int segmentSize = 30; + byte[] expected = new byte[bytes2Write]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(segmentSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[bufferSize]; + boolean eofException = false; + + try { + inputView.readFully(buffer); + }catch(EOFException e){ + //Expected exception + eofException = true; + } + catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertTrue("EOFException should have occurred.", eofException); + + int bytesRead = 0; + + try{ + bytesRead =inputView.read(buffer); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(-1, bytesRead); + } + + @Test + public void testReadFullyWithOffset(){ + int bufferSize = 100; + int segmentSize = 30; + byte[] expected = new byte[bufferSize]; + new Random().nextBytes(expected); + + TestOutputView outputView = new TestOutputView(segmentSize); + + try { + outputView.write(expected); + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not write to TestOutputView."); + } + + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[2*bufferSize]; + + try { + inputView.readFully(buffer, bufferSize, bufferSize); + } catch (IOException e) { + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertEquals(inputView.getCurrentPositionInSegment(), bufferSize % segmentSize); + byte[] tempBuffer = new byte[bufferSize]; + System.arraycopy(buffer, bufferSize, tempBuffer,0, bufferSize); + assertArrayEquals(expected, tempBuffer); + } + + @Test + public void testReadFullyEmptyView(){ + int segmentSize = 30; + TestOutputView outputView = new TestOutputView(segmentSize); + outputView.close(); + + TestInputView inputView = new TestInputView(outputView.segments); + byte[] buffer = new byte[segmentSize]; + boolean eofException = false; + + try{ + inputView.readFully(buffer); + }catch(EOFException e){ + //expected Exception + eofException = true; + }catch(Exception e){ + e.printStackTrace(); + fail("Unexpected exception: Could not read TestInputView."); + } + + assertTrue("EOFException expected.", eofException); + } + + private static void testSequenceOfTypes(Iterable sequence, int segmentSize) throws Exception { List elements = new ArrayList(512); @@ -148,7 +405,7 @@ protected MemorySegment nextSegment(MemorySegment current) throws IOException { if (num < segments.size()) { return segments.get(num).segment; } else { - return null; + throw new EOFException(); } }