diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/BooleanValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/BooleanValue.java index 925d421c0..55f2e3f3b 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/BooleanValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/BooleanValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class BooleanValue implements Value { @@ -58,12 +58,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value = in.readBoolean(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeBoolean(this.value); } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/DoubleValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/DoubleValue.java index 0d0abe721..df7d35584 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/DoubleValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/DoubleValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class DoubleValue implements Value { @@ -55,12 +55,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value = in.readDouble(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeDouble(this.value); } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/FloatValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/FloatValue.java index 87faab208..0311d2086 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/FloatValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/FloatValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class FloatValue implements Value { @@ -55,12 +55,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value = in.readFloat(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeFloat(this.value); } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValue.java index b0701a3db..f91377d83 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValue.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.Arrays; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.util.BytesUtil; import com.baidu.hugegraph.util.Bytes; import com.baidu.hugegraph.util.E; @@ -60,7 +60,7 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { int len = in.readInt(); this.bytes = BytesUtil.ensureCapacityWithoutCopy(this.bytes, len); in.readFully(this.bytes, 0, len); @@ -68,20 +68,16 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.length); out.write(this.bytes, 0, this.length); } - public void writeId(GraphOutput out) throws IOException { - out.write(this.bytes, 0, this.length); - } - @Override - public int compareTo(IdValue obj) { - E.checkArgumentNotNull(obj, "The compare argument can't be null"); + public int compareTo(IdValue other) { + E.checkArgumentNotNull(other, "The compare argument can't be null"); return BytesUtil.compare(this.bytes, this.length, - obj.bytes, obj.length); + other.bytes, other.length); } @Override diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueList.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueList.java index 106e4bd32..99b9a26db 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueList.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueList.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; public class IdValueList extends ListValue { @@ -36,12 +36,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.read(in, false); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { this.write(out, false); } } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueListList.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueListList.java index bfdfbfccb..20ecf9630 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueListList.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueListList.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; public class IdValueListList extends ListValue { @@ -36,12 +36,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.read(in, false); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { this.write(out, false); } } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IntValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IntValue.java index 336c0e4ec..0a54cffdf 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IntValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IntValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class IntValue implements Value { @@ -55,12 +55,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value = in.readInt(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.value); } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java index 5d9db3a78..92405b7d5 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/ListValue.java @@ -29,8 +29,8 @@ import com.baidu.hugegraph.computer.core.common.ComputerContext; import com.baidu.hugegraph.computer.core.common.SerialEnum; import com.baidu.hugegraph.computer.core.graph.GraphFactory; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class ListValue> implements Value> { @@ -110,11 +110,11 @@ public ValueType elemType() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.read(in, true); } - protected void read(GraphInput in, boolean readElemType) + protected void read(RandomAccessInput in, boolean readElemType) throws IOException { int size = in.readInt(); if (readElemType) { @@ -131,11 +131,11 @@ protected void read(GraphInput in, boolean readElemType) } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { this.write(out, true); } - protected void write(GraphOutput out, boolean writeElemType) + protected void write(RandomAccessOutput out, boolean writeElemType) throws IOException { out.writeInt(this.values.size()); if (writeElemType) { diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/LongValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/LongValue.java index 936a236ce..de0f5e38b 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/LongValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/LongValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class LongValue implements Value { @@ -55,12 +55,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value = in.readLong(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeLong(this.value); } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/NullValue.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/NullValue.java index 4052a4de6..d7137f2fb 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/NullValue.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/graph/value/NullValue.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.util.E; public class NullValue implements Value { @@ -45,12 +45,12 @@ public ValueType type() { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { // Do nothing } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { // Do nothing } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphInput.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphInput.java index 1c18909d0..0f9803db4 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphInput.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphInput.java @@ -20,7 +20,6 @@ package com.baidu.hugegraph.computer.core.io; import java.io.Closeable; -import java.io.DataInput; import java.io.IOException; import com.baidu.hugegraph.computer.core.graph.edge.Edge; @@ -30,7 +29,7 @@ import com.baidu.hugegraph.computer.core.graph.value.Value; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; -public interface GraphInput extends DataInput, Closeable { +public interface GraphInput extends Closeable { Vertex readVertex() throws IOException; diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutput.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutput.java index a2ecc39b6..2617b71c1 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutput.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutput.java @@ -20,7 +20,6 @@ package com.baidu.hugegraph.computer.core.io; import java.io.Closeable; -import java.io.DataOutput; import java.io.IOException; import com.baidu.hugegraph.computer.core.graph.edge.Edge; @@ -30,7 +29,7 @@ import com.baidu.hugegraph.computer.core.graph.value.Value; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; -public interface GraphOutput extends DataOutput, Closeable { +public interface GraphOutput extends Closeable { void writeVertex(Vertex vertex) throws IOException; diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java similarity index 86% rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java rename to computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java index 3a794eb2e..4dba74020 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessInput.java @@ -51,6 +51,12 @@ public interface RandomAccessInput extends DataInput, Closeable { */ long available() throws IOException; + /** + * Creates a new input that shares content like buffer or file, + * but use independent position + */ + RandomAccessInput duplicate() throws IOException; + /** * Compare two inputs in the specified range. */ @@ -67,4 +73,12 @@ default byte[] readBytes(int size) throws IOException { this.readFully(bytes); return bytes; } + + default int readIntLength() throws IOException { + return this.readInt(); + } + + default long readLongLength() throws IOException { + return this.readLong(); + } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java similarity index 76% rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java rename to computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java index cbf0eef5e..b4f5415a0 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/RandomAccessOutput.java @@ -39,4 +39,20 @@ public interface RandomAccessOutput extends DataOutput, Closeable { void write(RandomAccessInput input, long offset, long length) throws IOException; + + default long writeIntLength(int v) throws IOException { + long position = this.position(); + this.writeInt(v); + return position; + } + + default void writeIntLength(long position, int v) throws IOException { + this.writeInt(position, v); + } + + default long writeLongLength(long v) throws IOException { + long position = this.position(); + this.writeLong(v); + return position; + } } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Readable.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Readable.java index bb75bb8bc..211985a8a 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Readable.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Readable.java @@ -23,5 +23,5 @@ public interface Readable { - void read(GraphInput in) throws IOException; + void read(RandomAccessInput in) throws IOException; } diff --git a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Writable.java b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Writable.java index ac2098b46..dc9c29daa 100644 --- a/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Writable.java +++ b/computer-api/src/main/java/com/baidu/hugegraph/computer/core/io/Writable.java @@ -23,5 +23,5 @@ public interface Writable { - void write(GraphOutput out) throws IOException; + void write(RandomAccessOutput out) throws IOException; } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/Constants.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/Constants.java index 902598653..3533b99fa 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/Constants.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/Constants.java @@ -33,7 +33,6 @@ public final class Constants { public static final int CHAR_LEN = Character.BYTES; public static final int FLOAT_LEN = Float.BYTES; public static final int DOUBLE_LEN = Double.BYTES; - public static final int BLOB_LEN = 4; public static final int UINT8_MAX = 0xff; public static final int UINT16_MAX = 0xffff; diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/ContainerInfo.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/ContainerInfo.java index 6e1becd60..a93eff750 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/ContainerInfo.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/ContainerInfo.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.io.Readable; import com.baidu.hugegraph.computer.core.io.Writable; import com.baidu.hugegraph.computer.core.util.JsonUtil; @@ -74,7 +74,7 @@ public int dataPort() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.id = in.readInt(); this.hostname = in.readUTF(); this.rpcPort = in.readInt(); @@ -82,7 +82,7 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.id); out.writeUTF(this.hostname); out.writeInt(this.rpcPort); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/ComputeException.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/ComputeException.java deleted file mode 100644 index b3bef43eb..000000000 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/ComputeException.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.computer.core.common.exception; - -public class ComputeException extends ComputerException { - - private static final long serialVersionUID = 185790114205374242L; - - public ComputeException(String message) { - super(message); - } - - public ComputeException(String message, Throwable cause) { - super(message, cause); - } - - public ComputeException(String message, Object... args) { - super(message, args); - } - - public ComputeException(String message, Throwable cause, Object... args) { - super(message, cause, args); - } -} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java index a9623c27d..06c9e840b 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/common/exception/TransportException.java @@ -61,7 +61,7 @@ public TransportException(String message, Throwable cause, Object... args) { } public Throwable rootCause() { - return ComputeException.rootCause(this); + return ComputerException.rootCause(this); } public int errorCode() { diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/SuperstepStat.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/SuperstepStat.java index f25b11a57..7208fd037 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/SuperstepStat.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/SuperstepStat.java @@ -23,12 +23,12 @@ import java.util.List; import com.baidu.hugegraph.computer.core.graph.partition.PartitionStat; -import com.baidu.hugegraph.computer.core.util.JsonUtil; -import com.baidu.hugegraph.computer.core.worker.WorkerStat; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.io.Readable; import com.baidu.hugegraph.computer.core.io.Writable; +import com.baidu.hugegraph.computer.core.util.JsonUtil; +import com.baidu.hugegraph.computer.core.worker.WorkerStat; import com.baidu.hugegraph.util.E; /** @@ -93,7 +93,7 @@ public boolean active() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.vertexCount = in.readLong(); this.edgeCount = in.readLong(); this.finishedVertexCount = in.readLong(); @@ -103,7 +103,7 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeLong(this.vertexCount); out.writeLong(this.edgeCount); out.writeLong(this.finishedVertexCount); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongId.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongId.java index dcdb32a51..7be40bb60 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongId.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongId.java @@ -22,8 +22,8 @@ import java.io.IOException; import com.baidu.hugegraph.computer.core.graph.value.IdValue; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.util.IdValueUtil; import com.baidu.hugegraph.util.NumericUtil; @@ -66,12 +66,12 @@ public byte[] asBytes() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.id = in.readLong(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeLong(this.id); } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/Utf8Id.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/Utf8Id.java index 5771b37a1..c97569c7c 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/Utf8Id.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/Utf8Id.java @@ -24,8 +24,8 @@ import com.baidu.hugegraph.computer.core.common.Constants; import com.baidu.hugegraph.computer.core.graph.value.IdValue; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.util.BytesUtil; import com.baidu.hugegraph.computer.core.util.CoderUtil; import com.baidu.hugegraph.computer.core.util.IdValueUtil; @@ -86,7 +86,7 @@ public byte[] asBytes() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { int len = in.readInt(); this.bytes = BytesUtil.ensureCapacityWithoutCopy(this.bytes, len); in.readFully(this.bytes, 0, len); @@ -94,7 +94,7 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.length); out.write(this.bytes, 0, this.length); } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/UuidId.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/UuidId.java index 559d50c2f..d23951554 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/UuidId.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/id/UuidId.java @@ -25,8 +25,8 @@ import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.graph.value.IdValue; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.util.IdValueUtil; public class UuidId implements Id { @@ -74,13 +74,13 @@ public byte[] asBytes() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.high = in.readLong(); this.low = in.readLong(); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeLong(this.high); out.writeLong(this.low); } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/partition/PartitionStat.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/partition/PartitionStat.java index 13ba7e35c..df7338cf2 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/partition/PartitionStat.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/graph/partition/PartitionStat.java @@ -21,8 +21,8 @@ import java.io.IOException; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.io.Readable; import com.baidu.hugegraph.computer.core.io.Writable; import com.baidu.hugegraph.computer.core.util.JsonUtil; @@ -83,7 +83,7 @@ public long messageBytes() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.partitionId = in.readInt(); this.vertexCount = in.readLong(); this.edgeCount = in.readLong(); @@ -93,7 +93,7 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.partitionId); out.writeLong(this.vertexCount); out.writeLong(this.edgeCount); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileInput.java index a64142132..689405938 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileInput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileInput.java @@ -26,9 +26,10 @@ import com.baidu.hugegraph.computer.core.common.Constants; import com.baidu.hugegraph.computer.core.util.BytesUtil; +import com.baidu.hugegraph.testutil.Whitebox; import com.baidu.hugegraph.util.E; -public class BufferedFileInput extends UnsafeByteArrayInput { +public class BufferedFileInput extends UnsafeBytesInput { private final int bufferCapacity; private final RandomAccessFile file; @@ -156,6 +157,14 @@ private void fillBuffer() throws IOException { this.limit(this.limit() + readLen); } + @Override + public BufferedFileInput duplicate() throws IOException { + String path = Whitebox.getInternalState(this.file, "path"); + BufferedFileInput input = new BufferedFileInput(new File(path)); + input.seek(this.position()); + return input; + } + @Override public int compare(long offset, long length, RandomAccessInput other, long otherOffset, long otherLength) throws IOException { diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileOutput.java index 56ae04906..503661c7e 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileOutput.java @@ -33,7 +33,7 @@ * buffer is full. The second is unsafe data output. * This class is not thread safe. */ -public class BufferedFileOutput extends UnsafeByteArrayOutput { +public class BufferedFileOutput extends UnsafeBytesOutput { private final int bufferCapacity; private final RandomAccessFile file; diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamInput.java index a693d4dd9..455761c03 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamInput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamInput.java @@ -25,7 +25,7 @@ import com.baidu.hugegraph.computer.core.common.Constants; import com.baidu.hugegraph.util.E; -public class BufferedStreamInput extends UnsafeByteArrayInput { +public class BufferedStreamInput extends UnsafeBytesInput { private final int bufferCapacity; private final InputStream input; diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamOutput.java index db53812ab..b2ebdcab6 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedStreamOutput.java @@ -29,7 +29,7 @@ * This is used to buffer and output the buffer to output stream when buffer * is full. */ -public class BufferedStreamOutput extends UnsafeByteArrayOutput { +public class BufferedStreamOutput extends UnsafeBytesOutput { private final int bufferCapacity; private final OutputStream output; diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutput.java index a4f53ce2d..2fac22a9b 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutput.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.computer.core.io; -import java.io.DataOutput; import java.io.IOException; import java.util.Map; @@ -32,13 +31,13 @@ public class CsvStructGraphOutput extends StructGraphOutput { - public CsvStructGraphOutput(ComputerContext context, DataOutput out) { + public CsvStructGraphOutput(ComputerContext context, + StructRandomAccessOutput out) { super(context, out); } @Override public void writeVertex(Vertex vertex) throws IOException { - this.writeLineStart(); this.writeId(vertex.id()); @@ -102,22 +101,22 @@ public void writeProperties(Properties properties) throws IOException { @Override public void writeObjectStart() throws IOException { - this.writeRawString("{"); + this.out.writeRawString("{"); } @Override public void writeObjectEnd() throws IOException { - this.writeRawString("}"); + this.out.writeRawString("}"); } @Override public void writeArrayStart() throws IOException { - this.writeRawString("["); + this.out.writeRawString("["); } @Override public void writeArrayEnd() throws IOException { - this.writeRawString("]"); + this.out.writeRawString("]"); } @Override @@ -132,6 +131,6 @@ public void writeJoiner() throws IOException { @Override public void writeSplitter() throws IOException { - this.writeRawString(","); + this.out.writeRawString(","); } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutputFactory.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutputFactory.java index 484870e64..4d93b2981 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutputFactory.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/GraphOutputFactory.java @@ -29,11 +29,14 @@ public static GraphOutput create(ComputerContext context, RandomAccessOutput out) { switch (format) { case BIN: - return new OptimizedStreamGraphOutput(context, out); + return new StreamGraphOutput(context, out); case CSV: - return new CsvStructGraphOutput(context, out); + StructRandomAccessOutput srao; + srao = new StructRandomAccessOutput(out); + return new CsvStructGraphOutput(context, srao); case JSON: - return new JsonStructGraphOutput(context, out); + srao = new StructRandomAccessOutput(out); + return new JsonStructGraphOutput(context, srao); default: throw new ComputerException("Can't create GraphOutput for %s", format); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutput.java index ee9f9044a..e21f4c248 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutput.java @@ -19,7 +19,6 @@ package com.baidu.hugegraph.computer.core.io; -import java.io.DataOutput; import java.io.IOException; import java.util.Map; @@ -32,7 +31,8 @@ public class JsonStructGraphOutput extends StructGraphOutput { - public JsonStructGraphOutput(ComputerContext context, DataOutput out) { + public JsonStructGraphOutput(ComputerContext context, + StructRandomAccessOutput out) { super(context, out); } @@ -122,36 +122,36 @@ public void writeProperties(Properties properties) throws IOException { @Override public void writeObjectStart() throws IOException { - this.writeRawString("{"); + this.out.writeRawString("{"); } @Override public void writeObjectEnd() throws IOException { - this.writeRawString("}"); + this.out.writeRawString("}"); } @Override public void writeArrayStart() throws IOException { - this.writeRawString("["); + this.out.writeRawString("["); } @Override public void writeArrayEnd() throws IOException { - this.writeRawString("]"); + this.out.writeRawString("]"); } @Override public void writeKey(String key) throws IOException { - this.writeString(key); + this.out.writeString(key); } @Override public void writeJoiner() throws IOException { - this.writeRawString(":"); + this.out.writeRawString(":"); } @Override public void writeSplitter() throws IOException { - this.writeRawString(","); + this.out.writeRawString(","); } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphInput.java deleted file mode 100644 index 5bdfe4f18..000000000 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphInput.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.computer.core.io; - -import java.io.IOException; - -import com.baidu.hugegraph.computer.core.common.ComputerContext; - -public class OptimizedStreamGraphInput extends StreamGraphInput { - - public OptimizedStreamGraphInput(ComputerContext context, - RandomAccessInput in) { - super(context, in); - } - - @Override - public int readInt() throws IOException { - return this.readVInt(); - } - - @Override - public long readLong() throws IOException { - return this.readVLong(); - } -} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphOutput.java deleted file mode 100644 index 4b4c27c3e..000000000 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedStreamGraphOutput.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.computer.core.io; - -import java.io.IOException; - -import com.baidu.hugegraph.computer.core.common.ComputerContext; - -public class OptimizedStreamGraphOutput extends StreamGraphOutput { - - public OptimizedStreamGraphOutput(ComputerContext context, - RandomAccessOutput out) { - super(context, out); - } - - @Override - public void writeInt(int v) throws IOException { - this.writeVInt(v); - } - - @Override - public void writeLong(long v) throws IOException { - this.writeVLong(v); - } -} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesInput.java new file mode 100644 index 000000000..8644d1189 --- /dev/null +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesInput.java @@ -0,0 +1,141 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import java.io.IOException; + +import com.baidu.hugegraph.computer.core.util.CoderUtil; +import com.baidu.hugegraph.util.Bytes; +import com.baidu.hugegraph.util.E; + +public class OptimizedUnsafeBytesInput extends UnsafeBytesInput { + + public OptimizedUnsafeBytesInput(byte[] buffer) { + this(buffer, buffer.length); + } + + public OptimizedUnsafeBytesInput(byte[] buffer, int limit) { + this(buffer, 0, limit); + } + + public OptimizedUnsafeBytesInput(byte[] buffer, long limit) { + this(buffer, 0, (int) limit); + } + + public OptimizedUnsafeBytesInput(byte[] buffer, int position, int limit) { + super(buffer, position, limit); + } + + @Override + public OptimizedUnsafeBytesInput duplicate() throws IOException { + return new OptimizedUnsafeBytesInput(this.buffer(), + (int) this.position(), + this.limit()); + } + + @Override + public int readInt() throws IOException { + return this.readVInt(); + } + + @Override + public long readLong() throws IOException { + return this.readVLong(); + } + + @Override + public String readUTF() throws IOException { + return this.readString(); + } + + private int readVInt() throws IOException { + byte leading = this.readByte(); + E.checkArgument(leading != 0x80, + "Unexpected varint with leading byte '0x%s'", + Bytes.toHex(leading)); + int value = leading & 0x7f; + if (leading >= 0) { + assert (leading & 0x80) == 0; + return value; + } + + int i = 1; + for (; i < 5; i++) { + byte b = this.readByte(); + if (b >= 0) { + value = b | (value << 7); + break; + } else { + value = (b & 0x7f) | (value << 7); + } + } + + E.checkArgument(i < 5, + "Unexpected varint %s with too many bytes(%s)", + value, i + 1); + E.checkArgument(i < 4 || (leading & 0x70) == 0, + "Unexpected varint %s with leading byte '0x%s'", + value, Bytes.toHex(leading)); + return value; + } + + private long readVLong() throws IOException { + byte leading = this.readByte(); + E.checkArgument(leading != 0x80, + "Unexpected varlong with leading byte '0x%s'", + Bytes.toHex(leading)); + long value = leading & 0x7fL; + if (leading >= 0) { + assert (leading & 0x80) == 0; + return value; + } + + int i = 1; + for (; i < 10; i++) { + byte b = this.readByte(); + if (b >= 0) { + value = b | (value << 7); + break; + } else { + value = (b & 0x7f) | (value << 7); + } + } + + E.checkArgument(i < 10, + "Unexpected varlong %s with too many bytes(%s)", + value, i + 1); + E.checkArgument(i < 9 || (leading & 0x7e) == 0, + "Unexpected varlong %s with leading byte '0x%s'", + value, Bytes.toHex(leading)); + return value; + } + + private String readString() throws IOException { + return CoderUtil.decode(this.readBytes()); + } + + private byte[] readBytes() throws IOException { + int length = this.readVInt(); + assert length >= 0; + byte[] bytes = new byte[length]; + this.readFully(bytes, 0, length); + return bytes; + } +} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesOutput.java new file mode 100644 index 000000000..25ac8b37c --- /dev/null +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesOutput.java @@ -0,0 +1,113 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import static com.baidu.hugegraph.computer.core.common.Constants.UINT16_MAX; + +import java.io.IOException; + +import com.baidu.hugegraph.computer.core.util.CoderUtil; +import com.baidu.hugegraph.util.E; + +public class OptimizedUnsafeBytesOutput extends UnsafeBytesOutput { + + public OptimizedUnsafeBytesOutput() { + super(); + } + + public OptimizedUnsafeBytesOutput(int size) { + super(size); + } + + @Override + public void writeInt(int v) throws IOException { + this.writeVInt(v); + } + + @Override + public void writeLong(long v) throws IOException { + this.writeVLong(v); + } + + @Override + public void writeUTF(String s) throws IOException { + this.writeString(s); + } + + private void writeVInt(int value) throws IOException { + // NOTE: negative numbers are not compressed + if (value > 0x0fffffff || value < 0) { + this.writeByte(0x80 | ((value >>> 28) & 0x7f)); + } + if (value > 0x1fffff || value < 0) { + this.writeByte(0x80 | ((value >>> 21) & 0x7f)); + } + if (value > 0x3fff || value < 0) { + this.writeByte(0x80 | ((value >>> 14) & 0x7f)); + } + if (value > 0x7f || value < 0) { + this.writeByte(0x80 | ((value >>> 7) & 0x7f)); + } + this.writeByte(value & 0x7f); + } + + private void writeVLong(long value) throws IOException { + if (value < 0) { + this.writeByte((byte) 0x81); + } + if (value > 0xffffffffffffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 56) & 0x7f)); + } + if (value > 0x1ffffffffffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 49) & 0x7f)); + } + if (value > 0x3ffffffffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 42) & 0x7f)); + } + if (value > 0x7ffffffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 35) & 0x7f)); + } + if (value > 0xfffffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 28) & 0x7f)); + } + if (value > 0x1fffffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 21) & 0x7f)); + } + if (value > 0x3fffL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 14) & 0x7f)); + } + if (value > 0x7fL || value < 0L) { + this.writeByte(0x80 | ((int) (value >>> 7) & 0x7f)); + } + this.write((int) value & 0x7f); + } + + private void writeString(String val) throws IOException { + this.writeBytes(CoderUtil.encode(val)); + } + + private void writeBytes(byte[] bytes) throws IOException { + E.checkArgument(bytes.length <= UINT16_MAX, + "The max length of bytes is %s, but got %s", + UINT16_MAX, bytes.length); + this.writeVInt(bytes.length); + this.write(bytes); + } +} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java index 51c82a653..dfb6330f1 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphInput.java @@ -22,6 +22,7 @@ import java.io.IOException; import com.baidu.hugegraph.computer.core.common.ComputerContext; +import com.baidu.hugegraph.computer.core.common.SerialEnum; import com.baidu.hugegraph.computer.core.config.Config; import com.baidu.hugegraph.computer.core.graph.GraphFactory; import com.baidu.hugegraph.computer.core.graph.edge.Edge; @@ -33,9 +34,6 @@ import com.baidu.hugegraph.computer.core.graph.value.ValueFactory; import com.baidu.hugegraph.computer.core.graph.value.ValueType; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; -import com.baidu.hugegraph.computer.core.util.CoderUtil; -import com.baidu.hugegraph.util.Bytes; -import com.baidu.hugegraph.util.E; public class StreamGraphInput implements GraphInput { @@ -53,7 +51,6 @@ public StreamGraphInput(ComputerContext context, RandomAccessInput in) { @Override public Vertex readVertex() throws IOException { - Id id = this.readId(); Value value = this.readValue(); /* @@ -82,12 +79,12 @@ public Edges readEdges() throws IOException { * the current vertex may read the super vertex's edges. */ - int numEdges = this.readInt(); + int numEdges = this.in.readInt(); if (numEdges == 0) { return this.graphFactory.createEdges(0); } @SuppressWarnings("unused") - int bytes = this.readFullInt(); + int bytes = this.in.readIntLength(); Edges edges = this.graphFactory.createEdges(numEdges); // TODO: lazy deserialization for (int i = 0; i < numEdges; ++i) { @@ -114,10 +111,13 @@ public Edge readEdge() throws IOException { @Override public Properties readProperties() throws IOException { Properties properties = this.graphFactory.createProperties(); - int size = this.readInt(); + int size = this.in.readInt(); for (int i = 0; i < size; i++) { - String key = this.readString(); - Value value = this.readValue(); + String key = this.in.readUTF(); + ValueType valueType = SerialEnum.fromCode(ValueType.class, + this.in.readByte()); + Value value = this.valueFactory.createValue(valueType); + value.read(this.in); properties.put(key, value); } return properties; @@ -125,9 +125,9 @@ public Properties readProperties() throws IOException { @Override public Id readId() throws IOException { - byte type = this.readByte(); + byte type = this.in.readByte(); Id id = IdFactory.createId(type); - id.read(this); + id.read(this.in); return id; } @@ -135,191 +135,10 @@ public Id readId() throws IOException { public Value readValue() throws IOException { ValueType valueType = this.config.valueType(); Value value = this.valueFactory.createValue(valueType); - value.read(this); - return value; - } - - public long position() { - return this.in.position(); - } - - public void seek(long position) throws IOException { - this.in.seek(position); - } - - public long skip(long n) throws IOException { - return this.in.skip(n); - } - - public int readVInt() throws IOException { - byte leading = this.readByte(); - E.checkArgument(leading != 0x80, - "Unexpected varint with leading byte '0x%s'", - Bytes.toHex(leading)); - int value = leading & 0x7f; - if (leading >= 0) { - assert (leading & 0x80) == 0; - return value; - } - - int i = 1; - for (; i < 5; i++) { - byte b = this.readByte(); - if (b >= 0) { - value = b | (value << 7); - break; - } else { - value = (b & 0x7f) | (value << 7); - } - } - - E.checkArgument(i < 5, - "Unexpected varint %s with too many bytes(%s)", - value, i + 1); - E.checkArgument(i < 4 || (leading & 0x70) == 0, - "Unexpected varint %s with leading byte '0x%s'", - value, Bytes.toHex(leading)); - return value; - } - - public long readVLong() throws IOException { - byte leading = this.readByte(); - E.checkArgument(leading != 0x80, - "Unexpected varlong with leading byte '0x%s'", - Bytes.toHex(leading)); - long value = leading & 0x7fL; - if (leading >= 0) { - assert (leading & 0x80) == 0; - return value; - } - - int i = 1; - for (; i < 10; i++) { - byte b = this.readByte(); - if (b >= 0) { - value = b | (value << 7); - break; - } else { - value = (b & 0x7f) | (value << 7); - } - } - - E.checkArgument(i < 10, - "Unexpected varlong %s with too many bytes(%s)", - value, i + 1); - E.checkArgument(i < 9 || (leading & 0x7e) == 0, - "Unexpected varlong %s with leading byte '0x%s'", - value, Bytes.toHex(leading)); + value.read(this.in); return value; } - public int readUInt8() throws IOException { - return this.readUnsignedByte(); - } - - public int readUInt16() throws IOException { - return this.readUnsignedShort(); - } - - public long readUInt32() throws IOException { - return this.readInt() & 0xffffffffL; - } - - public String readString() throws IOException { - return CoderUtil.decode(this.readBytes()); - } - - public byte[] readBytes() throws IOException { - int length = this.readVInt(); - assert length >= 0; - byte[] bytes = new byte[length]; - this.readFully(bytes, 0, length); - return bytes; - } - - @Override - public void readFully(byte[] b) throws IOException { - this.in.readFully(b); - } - - @Override - public void readFully(byte[] b, int off, int len) throws IOException { - this.in.readFully(b, off, len); - } - - @Override - public int skipBytes(int n) throws IOException { - return this.in.skipBytes(n); - } - - @Override - public boolean readBoolean() throws IOException { - return this.in.readBoolean(); - } - - @Override - public byte readByte() throws IOException { - return this.in.readByte(); - } - - @Override - public int readUnsignedByte() throws IOException { - return this.in.readUnsignedByte(); - } - - @Override - public short readShort() throws IOException { - return this.in.readShort(); - } - - @Override - public int readUnsignedShort() throws IOException { - return this.in.readUnsignedShort(); - } - - @Override - public char readChar() throws IOException { - return this.in.readChar(); - } - - @Override - public int readInt() throws IOException { - return this.in.readInt(); - } - - public final int readFullInt() throws IOException { - return this.in.readInt(); - } - - @Override - public long readLong() throws IOException { - return this.in.readLong(); - } - - public final long readFullLong() throws IOException { - return this.in.readLong(); - } - - @Override - public float readFloat() throws IOException { - return this.in.readFloat(); - } - - @Override - public double readDouble() throws IOException { - return this.in.readDouble(); - } - - @Override - public String readLine() throws IOException { - return this.in.readLine(); - } - - @Override - public String readUTF() throws IOException { - return this.in.readUTF(); - } - @Override public void close() throws IOException { this.in.close(); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java index 141516e01..c2a06ca56 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutput.java @@ -19,8 +19,6 @@ package com.baidu.hugegraph.computer.core.io; -import static com.baidu.hugegraph.computer.core.common.Constants.UINT16_MAX; - import java.io.IOException; import java.util.Map; @@ -33,8 +31,6 @@ import com.baidu.hugegraph.computer.core.graph.properties.Properties; import com.baidu.hugegraph.computer.core.graph.value.Value; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; -import com.baidu.hugegraph.computer.core.util.CoderUtil; -import com.baidu.hugegraph.util.E; public class StreamGraphOutput implements GraphOutput { @@ -63,17 +59,17 @@ public void writeVertex(Vertex vertex) throws IOException { @Override public void writeEdges(Edges edges) throws IOException { int size = edges.size(); - this.writeInt(size); + this.out.writeInt(size); if (size == 0) { return; } - long startPosition = this.writeFullInt(0); + long startPosition = this.out.writeIntLength(0); for (Edge edge : edges) { this.writeEdge(edge); } - long endPosition = this.position(); + long endPosition = this.out.position(); long length = endPosition - startPosition - Constants.INT_LEN; - this.writeFullInt(startPosition, (int) length); + this.out.writeIntLength(startPosition, (int) length); } @Override @@ -92,195 +88,24 @@ public void writeEdge(Edge edge) throws IOException { @Override public void writeProperties(Properties properties) throws IOException { Map> keyValues = properties.get(); - this.writeInt(keyValues.size()); + this.out.writeInt(keyValues.size()); for (Map.Entry> entry : keyValues.entrySet()) { - this.writeString(entry.getKey()); - this.writeValue(entry.getValue()); + this.out.writeUTF(entry.getKey()); + Value value = entry.getValue(); + this.out.writeByte(value.type().code()); + value.write(this.out); } } @Override public void writeId(Id id) throws IOException { - this.writeByte(id.type().code()); - id.write(this); + this.out.writeByte(id.type().code()); + id.write(this.out); } @Override public void writeValue(Value value) throws IOException { - value.write(this); - } - - public void writeVInt(int value) throws IOException { - // NOTE: negative numbers are not compressed - if (value > 0x0fffffff || value < 0) { - this.writeByte(0x80 | ((value >>> 28) & 0x7f)); - } - if (value > 0x1fffff || value < 0) { - this.writeByte(0x80 | ((value >>> 21) & 0x7f)); - } - if (value > 0x3fff || value < 0) { - this.writeByte(0x80 | ((value >>> 14) & 0x7f)); - } - if (value > 0x7f || value < 0) { - this.writeByte(0x80 | ((value >>> 7) & 0x7f)); - } - this.writeByte(value & 0x7f); - } - - public void writeVLong(long value) throws IOException { - if (value < 0) { - this.writeByte((byte) 0x81); - } - if (value > 0xffffffffffffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 56) & 0x7f)); - } - if (value > 0x1ffffffffffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 49) & 0x7f)); - } - if (value > 0x3ffffffffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 42) & 0x7f)); - } - if (value > 0x7ffffffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 35) & 0x7f)); - } - if (value > 0xfffffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 28) & 0x7f)); - } - if (value > 0x1fffffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 21) & 0x7f)); - } - if (value > 0x3fffL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 14) & 0x7f)); - } - if (value > 0x7fL || value < 0L) { - this.writeByte(0x80 | ((int) (value >>> 7) & 0x7f)); - } - this.write((int) value & 0x7f); - } - - public void writeUInt8(int val) throws IOException { - assert val <= Constants.UINT8_MAX; - this.write(val); - } - - public void writeUInt16(int val) throws IOException { - assert val <= UINT16_MAX; - this.writeShort(val); - } - - public void writeUInt32(long val) throws IOException { - assert val <= Constants.UINT32_MAX; - this.writeInt((int) val); - } - - public void writeString(String val) throws IOException { - this.writeBytes(CoderUtil.encode(val)); - } - - public void writeBytes(byte[] bytes) throws IOException { - E.checkArgument(bytes.length <= UINT16_MAX, - "The max length of bytes is %s, but got %s", - UINT16_MAX, bytes.length); - this.writeVInt(bytes.length); - this.write(bytes); - } - - @Override - public void write(int b) throws IOException { - this.out.write(b); - } - - @Override - public void write(byte[] b) throws IOException { - this.out.write(b); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - this.out.write(b, off, len); - } - - @Override - public void writeBoolean(boolean v) throws IOException { - this.out.writeBoolean(v); - } - - @Override - public void writeByte(int v) throws IOException { - this.out.writeByte(v); - } - - @Override - public void writeShort(int v) throws IOException { - this.out.writeShort(v); - } - - @Override - public void writeChar(int v) throws IOException { - this.out.writeChar(v); - } - - @Override - public void writeInt(int v) throws IOException { - this.out.writeInt(v); - } - - public final long writeFullInt(int v) throws IOException { - long position = this.position(); - this.out.writeInt(v); - return position; - } - - public final void writeFullInt(long position, int v) throws IOException { - this.out.writeInt(position, v); - } - - @Override - public void writeLong(long v) throws IOException { - this.out.writeLong(v); - } - - public final long writeFullLong(long v) throws IOException { - long position = this.position(); - this.out.writeLong(v); - return position; - } - - @Override - public void writeFloat(float v) throws IOException { - this.out.writeFloat(v); - } - - @Override - public void writeDouble(double v) throws IOException { - this.out.writeDouble(v); - } - - @Override - public void writeBytes(String s) throws IOException { - this.out.writeBytes(s); - } - - @Override - public void writeChars(String s) throws IOException { - this.out.writeChars(s); - } - - @Override - public void writeUTF(String s) throws IOException { - this.out.writeUTF(s); - } - - public long position() { - return this.out.position(); - } - - public void seek(long position) throws IOException { - this.out.seek(position); - } - - public long skip(long n) throws IOException { - return this.out.skip(n); + value.write(this.out); } @Override diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructGraphOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructGraphOutput.java index 23e61a862..71797d138 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructGraphOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructGraphOutput.java @@ -19,12 +19,8 @@ package com.baidu.hugegraph.computer.core.io; -import java.io.Closeable; -import java.io.DataOutput; import java.io.IOException; -import org.apache.commons.lang3.StringEscapeUtils; - import com.baidu.hugegraph.computer.core.common.ComputerContext; import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.config.Config; @@ -33,14 +29,14 @@ import com.baidu.hugegraph.computer.core.graph.value.ListValue; import com.baidu.hugegraph.computer.core.graph.value.Value; import com.baidu.hugegraph.computer.core.util.IdValueUtil; -import com.baidu.hugegraph.computer.core.util.StringEncoding; public abstract class StructGraphOutput implements GraphOutput { - protected final DataOutput out; protected final Config config; + protected final StructRandomAccessOutput out; - public StructGraphOutput(ComputerContext context, DataOutput out) { + public StructGraphOutput(ComputerContext context, + StructRandomAccessOutput out) { this.config = context.config(); this.out = out; } @@ -64,12 +60,12 @@ public void writeLineStart() throws IOException { } public void writeLineEnd() throws IOException { - this.writeRawString(System.lineSeparator()); + this.out.writeRawString(System.lineSeparator()); } @Override public void writeId(Id id) throws IOException { - id.write(this); + id.write(this.out); } @Override @@ -89,7 +85,7 @@ public void writeValue(Value value) throws IOException { case LONG: case FLOAT: case DOUBLE: - value.write(this); + value.write(this.out); break; default: throw new ComputerException("Unexpected value type %s", @@ -97,6 +93,11 @@ public void writeValue(Value value) throws IOException { } } + @Override + public void close() throws IOException { + this.out.close(); + } + private void writeIdValue(IdValue idValue) throws IOException { /* * The idValue is shown as bytes in computation, @@ -117,97 +118,4 @@ private void writeListValue(ListValue listValue) throws IOException { } this.writeArrayEnd(); } - - @Override - public void write(int b) throws IOException { - this.writeNumber(b); - } - - @Override - public void write(byte[] b) throws IOException { - this.writeString(StringEncoding.encodeBase64(b)); - } - - @Override - public void write(byte[] b, int off, int len) throws IOException { - byte[] dest = new byte[len]; - System.arraycopy(b, off, dest, 0, len); - this.writeString(StringEncoding.encodeBase64(dest)); - } - - @Override - public void writeBoolean(boolean v) throws IOException { - this.writeRawString(v ? "true" : "false"); - } - - @Override - public void writeByte(int v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeShort(int v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeChar(int v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeInt(int v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeLong(long v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeFloat(float v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeDouble(double v) throws IOException { - this.writeNumber(v); - } - - @Override - public void writeBytes(String s) throws IOException { - this.writeString(s); - } - - @Override - public void writeChars(String s) throws IOException { - this.writeString(s); - } - - @Override - public void writeUTF(String s) throws IOException { - this.writeString(s); - } - - @Override - public void close() throws IOException { - if (this.out instanceof Closeable) { - ((Closeable) this.out).close(); - } - } - - protected void writeNumber(Number number) throws IOException { - this.out.writeBytes(number.toString()); - } - - protected void writeRawString(String s) throws IOException { - this.out.writeBytes(s); - } - - protected void writeString(String s) throws IOException { - this.out.writeBytes("\""); - this.out.writeBytes(StringEscapeUtils.escapeJson(s)); - this.out.writeBytes("\""); - } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutput.java new file mode 100644 index 000000000..76d2b4ed3 --- /dev/null +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutput.java @@ -0,0 +1,161 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import java.io.IOException; + +import org.apache.commons.lang3.StringEscapeUtils; + +import com.baidu.hugegraph.computer.core.util.StringEncoding; +import com.baidu.hugegraph.testutil.Whitebox; + +public class StructRandomAccessOutput implements RandomAccessOutput { + + private final RandomAccessOutput output; + + public StructRandomAccessOutput(RandomAccessOutput output) { + this.output = output; + } + + @Override + public long position() { + return this.output.position(); + } + + @Override + public void seek(long position) throws IOException { + this.output.seek(position); + } + + @Override + public long skip(long n) throws IOException { + return this.output.skip(n); + } + + @Override + public void writeInt(long position, int v) throws IOException { + this.output.seek(position); + this.writeNumber(v); + } + + @Override + public void write(RandomAccessInput input, long offset, long length) + throws IOException { + if (UnsafeBytesInput.class == input.getClass()) { + byte[] buffer = Whitebox.getInternalState(input, "buffer"); + this.write(buffer, (int) offset, (int) length); + } else { + input.seek(offset); + byte[] bytes = input.readBytes((int) length); + this.write(bytes); + } + } + + @Override + public void write(int b) throws IOException { + this.writeNumber(b); + } + + @Override + public void write(byte[] b) throws IOException { + this.writeString(StringEncoding.encodeBase64(b)); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + byte[] dest = new byte[len]; + System.arraycopy(b, off, dest, 0, len); + this.writeString(StringEncoding.encodeBase64(dest)); + } + + @Override + public void writeBoolean(boolean v) throws IOException { + this.writeRawString(v ? "true" : "false"); + } + + @Override + public void writeByte(int v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeShort(int v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeChar(int v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeInt(int v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeLong(long v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeFloat(float v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeDouble(double v) throws IOException { + this.writeNumber(v); + } + + @Override + public void writeBytes(String s) throws IOException { + this.writeString(s); + } + + @Override + public void writeChars(String s) throws IOException { + this.writeString(s); + } + + @Override + public void writeUTF(String s) throws IOException { + this.writeString(s); + } + + @Override + public void close() throws IOException { + this.output.close(); + } + + protected void writeNumber(Number number) throws IOException { + this.output.writeBytes(number.toString()); + } + + protected void writeRawString(String s) throws IOException { + this.output.writeBytes(s); + } + + protected void writeString(String s) throws IOException { + this.output.writeBytes("\""); + this.output.writeBytes(StringEscapeUtils.escapeJson(s)); + this.output.writeBytes("\""); + } +} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayInput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesInput.java similarity index 93% rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayInput.java rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesInput.java index 65548cd91..4d2eb09c4 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayInput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesInput.java @@ -32,7 +32,7 @@ import sun.misc.Unsafe; @SuppressWarnings("deprecation") // Unsafe.getXx -public class UnsafeByteArrayInput implements RandomAccessInput, Closeable { +public class UnsafeBytesInput implements RandomAccessInput, Closeable { private static final sun.misc.Unsafe UNSAFE; @@ -50,19 +50,19 @@ public class UnsafeByteArrayInput implements RandomAccessInput, Closeable { } } - public UnsafeByteArrayInput(byte[] buffer) { + public UnsafeBytesInput(byte[] buffer) { this(buffer, buffer.length); } - public UnsafeByteArrayInput(byte[] buffer, int limit) { + public UnsafeBytesInput(byte[] buffer, int limit) { this(buffer, 0, limit); } - public UnsafeByteArrayInput(byte[] buffer, long limit) { + public UnsafeBytesInput(byte[] buffer, long limit) { this(buffer, 0, (int) limit); } - public UnsafeByteArrayInput(byte[] buffer, int position, int limit) { + public UnsafeBytesInput(byte[] buffer, int position, int limit) { E.checkArgumentNotNull(buffer, "The buffer can't be null"); this.buffer = buffer; this.limit = limit; @@ -218,6 +218,11 @@ public void close() throws IOException { // pass } + @Override + public UnsafeBytesInput duplicate() throws IOException { + return new UnsafeBytesInput(this.buffer, this.position, this.limit); + } + @Override public int compare(long offset, long length, RandomAccessInput other, long otherOffset, long otherLength) throws IOException { @@ -228,9 +233,9 @@ public int compare(long offset, long length, RandomAccessInput other, "Invalid length parameter %s, expect <= %s", length, this.buffer.length - offset); - if (other.getClass() == UnsafeByteArrayInput.class) { + if (other.getClass() == UnsafeBytesInput.class) { return BytesUtil.compare(this.buffer, (int) offset, (int) length, - ((UnsafeByteArrayInput) other).buffer, + ((UnsafeBytesInput) other).buffer, (int) otherOffset, (int) otherLength); } else { long otherPosition = other.position(); diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayOutput.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesOutput.java similarity index 97% rename from computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayOutput.java rename to computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesOutput.java index 0c8a3a0b4..97ac3e963 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayOutput.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesOutput.java @@ -37,7 +37,7 @@ * Use unsafe method to write the value to the buffer to improve the write * performance. The buffer is auto extendable. */ -public class UnsafeByteArrayOutput implements RandomAccessOutput, Closeable { +public class UnsafeBytesOutput implements RandomAccessOutput, Closeable { private static final sun.misc.Unsafe UNSAFE; private static final int DEFAULT_SIZE = 32; @@ -55,11 +55,11 @@ public class UnsafeByteArrayOutput implements RandomAccessOutput, Closeable { } } - public UnsafeByteArrayOutput() { + public UnsafeBytesOutput() { this(DEFAULT_SIZE); } - public UnsafeByteArrayOutput(int size) { + public UnsafeBytesOutput(int size) { this.buffer = new byte[size]; this.position = 0; } @@ -212,7 +212,7 @@ public long skip(long bytesToSkip) throws IOException { @Override public void write(RandomAccessInput input, long offset, long length) throws IOException { - if (UnsafeByteArrayInput.class == input.getClass()) { + if (UnsafeBytesInput.class == input.getClass()) { byte[] buffer = Whitebox.getInternalState(input, "buffer"); this.write(buffer, (int) offset, (int) length); } else { diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java index bff2cab5a..cf0770b1c 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/TransportUtil.java @@ -30,7 +30,7 @@ import org.slf4j.Logger; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.util.StringEncoding; import com.baidu.hugegraph.util.E; import com.baidu.hugegraph.util.Log; @@ -69,8 +69,8 @@ public static InetAddress resolvedAddress(String host) { try { return InetAddress.getByName(host); } catch (UnknownHostException e) { - throw new ComputeException("Failed to parse address from '%s'", e, - host); + throw new ComputerException("Failed to parse address from '%s'", e, + host); } } @@ -112,7 +112,7 @@ public static List getLocalIPAddress() { } return ips; } catch (Exception e) { - throw new ComputeException("Failed to getLocalIPAddress", e); + throw new ComputerException("Failed to getLocalIPAddress", e); } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java index bd5ba1d91..5f036a8db 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServer.java @@ -27,7 +27,7 @@ import org.slf4j.Logger; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.config.Config; import com.baidu.hugegraph.computer.core.network.IOMode; import com.baidu.hugegraph.computer.core.network.MessageHandler; @@ -176,7 +176,7 @@ public void shutdown() { try { this.close(); } catch (IOException e) { - throw new ComputeException("Failed to shutdown server", e); + throw new ComputerException("Failed to shutdown server", e); } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java index 8d643938b..25e86ba0e 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/network/session/ClientSession.java @@ -25,7 +25,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.Function; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.common.exception.TransportException; import com.baidu.hugegraph.computer.core.network.TransportConf; import com.baidu.hugegraph.computer.core.network.TransportState; @@ -152,9 +152,9 @@ public void onRecvAck(int ackId) { this.onRecvDataAck(ackId); break; default: - throw new ComputeException("Receive one ack message, but the " + - "state not match, state: %s, " + - "ackId: %s", this.state, ackId); + throw new ComputerException("Receive one ack message, but " + + "the state not match, state: %s, " + + "ackId: %s", this.state, ackId); } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/ByteArrayUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/ByteArrayUtil.java deleted file mode 100644 index 1931b5e37..000000000 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/ByteArrayUtil.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.computer.core.util; - -import com.google.common.base.Preconditions; - -public class ByteArrayUtil { - - public static byte[] ensureCapacityWithoutCopy(byte[] bytes, int length) { - if (bytes == null || bytes.length < length) { - return new byte[length]; - } else { - return bytes; - } - } - - public static int hashBytes(byte[] bytes, int length) { - return hashBytes(bytes, 0, length); - } - - public static int hashBytes(byte[] bytes, int offset, int length) { - int hash = 1; - for (int i = offset; i < offset + length; i++) { - hash = (31 * hash) + (int) bytes[i]; - } - return hash; - } - - public static int compare(byte[] bytes1, int length1, - byte[] bytes2, int length2) { - return compare(bytes1, 0, length1, bytes2, 0, length2); - } - - // TODO: use google comparator(unsafe) to improve perf - public static int compare(byte[] bytes1, int offset1, int length1, - byte[] bytes2, int offset2, int length2) { - Preconditions.checkNotNull(bytes1); - Preconditions.checkNotNull(bytes2); - if (bytes1 == bytes2 && offset1 == offset2 && length1 == length2) { - return 0; - } - if (length1 != length2) { - return length1 - length2; - } - - int end1 = offset1 + length1; - int end2 = offset2 + length2; - for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) { - int a = (bytes1[i] & 0xff); - int b = (bytes2[j] & 0xff); - if (a != b) { - return a - b; - } - } - return 0; - } -} diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/IdValueUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/IdValueUtil.java index 9ba0564d1..1f0b2cdb2 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/IdValueUtil.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/IdValueUtil.java @@ -25,12 +25,14 @@ import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.graph.id.Id; import com.baidu.hugegraph.computer.core.graph.value.IdValue; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphInput; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphOutput; +import com.baidu.hugegraph.computer.core.io.GraphInput; +import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesOutput; import com.baidu.hugegraph.computer.core.io.StreamGraphInput; import com.baidu.hugegraph.computer.core.io.StreamGraphOutput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayInput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayOutput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesOutput; public class IdValueUtil { @@ -39,9 +41,12 @@ public class IdValueUtil { public static Id toId(IdValue idValue) { byte[] bytes = idValue.bytes(); - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = new OptimizedStreamGraphInput(CONTEXT, - bai)) { + /* + * NOTE: must use OptimizedUnsafeByteArrayInput, it make sure to + * write bytes in big-end-aligned way + */ + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes); + GraphInput input = new StreamGraphInput(CONTEXT, bai)) { return input.readId(); } catch (IOException e) { throw new ComputerException("Failed to get id from idValue '%s'", @@ -50,9 +55,8 @@ public static Id toId(IdValue idValue) { } public static IdValue toIdValue(Id id, int len) { - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(len); - StreamGraphOutput output = new OptimizedStreamGraphOutput( - CONTEXT, bao)) { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput(len); + GraphOutput output = new StreamGraphOutput(CONTEXT, bao)) { output.writeId(id); return new IdValue(bao.toByteArray()); } catch (IOException e) { diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/SerializeUtil.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/SerializeUtil.java index c873dac6f..3c9e8cdd3 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/SerializeUtil.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/util/SerializeUtil.java @@ -22,14 +22,12 @@ import java.io.IOException; import com.baidu.hugegraph.computer.core.common.ComputerContext; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphInput; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphOutput; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesOutput; import com.baidu.hugegraph.computer.core.io.Readable; -import com.baidu.hugegraph.computer.core.io.StreamGraphInput; -import com.baidu.hugegraph.computer.core.io.StreamGraphOutput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayInput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayOutput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesOutput; import com.baidu.hugegraph.computer.core.io.Writable; public final class SerializeUtil { @@ -38,24 +36,20 @@ public final class SerializeUtil { private static final ComputerContext CONTEXT = ComputerContext.instance(); public static byte[] toBytes(Writable obj) { - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = new OptimizedStreamGraphOutput(CONTEXT, - bao)) { - obj.write(output); + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + obj.write(bao); return bao.toByteArray(); } catch (IOException e) { - throw new ComputeException( + throw new ComputerException( "Failed to create byte array with writable '%s'", e, obj); } } public static void fromBytes(byte[] bytes, Readable obj) { - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = new OptimizedStreamGraphInput(CONTEXT, - bai)) { - obj.read(input); + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + obj.read(bai); } catch (IOException e) { - throw new ComputeException("Failed to read from byte array", e); + throw new ComputerException("Failed to read from byte array", e); } } } diff --git a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/worker/WorkerStat.java b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/worker/WorkerStat.java index 1cd5667b8..609d68156 100644 --- a/computer-core/src/main/java/com/baidu/hugegraph/computer/core/worker/WorkerStat.java +++ b/computer-core/src/main/java/com/baidu/hugegraph/computer/core/worker/WorkerStat.java @@ -25,8 +25,8 @@ import java.util.List; import com.baidu.hugegraph.computer.core.graph.partition.PartitionStat; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; import com.baidu.hugegraph.computer.core.io.Readable; import com.baidu.hugegraph.computer.core.io.Writable; import com.baidu.hugegraph.computer.core.util.JsonUtil; @@ -64,7 +64,7 @@ public int size() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.workerId = in.readInt(); int size = in.readInt(); this.partitionStats = new ArrayList<>(size); @@ -76,7 +76,7 @@ public void read(GraphInput in) throws IOException { } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeInt(this.workerId); out.writeInt(this.partitionStats.size()); for (PartitionStat stat : this.partitionStats) { diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/UnitTestBase.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/UnitTestBase.java index ce73dfab5..0f42eeb04 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/UnitTestBase.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/UnitTestBase.java @@ -33,13 +33,13 @@ import com.baidu.hugegraph.computer.core.graph.id.IdFactory; import com.baidu.hugegraph.computer.core.graph.value.Value; import com.baidu.hugegraph.computer.core.graph.value.ValueFactory; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphInput; -import com.baidu.hugegraph.computer.core.io.OptimizedStreamGraphOutput; +import com.baidu.hugegraph.computer.core.io.GraphOutputFactory; +import com.baidu.hugegraph.computer.core.io.OutputFormat; import com.baidu.hugegraph.computer.core.io.Readable; import com.baidu.hugegraph.computer.core.io.StreamGraphInput; import com.baidu.hugegraph.computer.core.io.StreamGraphOutput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayInput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayOutput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesOutput; import com.baidu.hugegraph.computer.core.io.Writable; import com.baidu.hugegraph.computer.core.util.ComputerContextUtil; import com.baidu.hugegraph.config.ConfigOption; @@ -55,16 +55,14 @@ public class UnitTestBase { public static void assertIdEqualAfterWriteAndRead(Id oldId) throws IOException { byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - oldId.write(output); + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + oldId.write(bao); bytes = bao.toByteArray(); } Id newId = IdFactory.createId(oldId.type()); - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - newId.read(input); + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + newId.read(bai); Assert.assertEquals(oldId, newId); } } @@ -72,15 +70,14 @@ public static void assertIdEqualAfterWriteAndRead(Id oldId) public static void assertValueEqualAfterWriteAndRead(Value oldValue) throws IOException { byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - oldValue.write(output); + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + oldValue.write(bao); bytes = bao.toByteArray(); } + Value newValue = valueFactory().createValue(oldValue.type()); - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - newValue.read(input); + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + newValue.read(bai); Assert.assertEquals(oldValue, newValue); } } @@ -125,15 +122,13 @@ public static void assertEqualAfterWriteAndRead(Writable writeObj, Readable readObj) throws IOException { byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - writeObj.write(output); + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + writeObj.write(bao); bytes = bao.toByteArray(); } - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - readObj.read(input); + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + readObj.read(bai); Assert.assertEquals(writeObj, readObj); } } @@ -177,22 +172,14 @@ protected static GraphFactory graphFactory() { } protected static StreamGraphInput newStreamGraphInput( - UnsafeByteArrayInput bai) { + UnsafeBytesInput bai) { return new StreamGraphInput(context(), bai); } protected static StreamGraphOutput newStreamGraphOutput( - UnsafeByteArrayOutput bao) { - return new StreamGraphOutput(context(), bao); - } - - protected static OptimizedStreamGraphInput newOptimizedStreamGraphInput( - UnsafeByteArrayInput bai) { - return new OptimizedStreamGraphInput(context(), bai); - } - - protected static OptimizedStreamGraphOutput newOptimizedStreamGraphOutput( - UnsafeByteArrayOutput bao) { - return new OptimizedStreamGraphOutput(context(), bao); + UnsafeBytesOutput bao) { + return (StreamGraphOutput) GraphOutputFactory.create(context(), + OutputFormat.BIN, + bao); } } diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongIdTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongIdTest.java index cbd707b7f..4690dbc2b 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongIdTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/id/LongIdTest.java @@ -34,27 +34,43 @@ public class LongIdTest extends UnitTestBase { @Test public void test() { - LongId longId1 = new LongId(-100L); - LongId longId2 = new LongId(); - LongId longId3 = new LongId(-100L); + LongId longId1 = new LongId(123L); + LongId longId2 = new LongId(321L); + LongId longId3 = new LongId(123L); + LongId longId4 = new LongId(322L); + + LongId longId5 = new LongId(-100L); + LongId longId6 = new LongId(); + LongId longId7 = new LongId(-100L); Assert.assertEquals(IdType.LONG, longId1.type()); IdValue idValue = longId1.idValue(); Assert.assertEquals(ValueType.ID_VALUE, idValue.type()); Assert.assertEquals(longId1, IdValueUtil.toId(idValue)); - Assert.assertEquals(new Long(-100L), longId1.asObject()); - Assert.assertEquals(-100L, longId1.asLong()); - Assert.assertArrayEquals(NumericUtil.longToBytes(-100L), + Assert.assertEquals(new Long(123L), longId1.asObject()); + Assert.assertEquals(123L, longId1.asLong()); + Assert.assertArrayEquals(NumericUtil.longToBytes(123L), longId1.asBytes()); + Assert.assertEquals(new Long(-100L), longId5.asObject()); + Assert.assertEquals(-100L, longId5.asLong()); + Assert.assertArrayEquals(NumericUtil.longToBytes(-100L), + longId5.asBytes()); + Assert.assertTrue(longId1.compareTo(longId2) < 0); Assert.assertTrue(longId2.compareTo(longId1) > 0); Assert.assertTrue(longId1.compareTo(longId3) == 0); + Assert.assertTrue(longId2.compareTo(longId4) < 0); + + Assert.assertTrue(longId5.compareTo(longId6) < 0); + Assert.assertTrue(longId6.compareTo(longId5) > 0); + Assert.assertTrue(longId5.compareTo(longId7) == 0); Assert.assertEquals(longId1, longId3); Assert.assertNotEquals(longId1, longId2); - Assert.assertEquals(Long.hashCode(-100L), longId1.hashCode()); + Assert.assertEquals(Long.hashCode(123L), longId1.hashCode()); + Assert.assertEquals(Long.hashCode(-100L), longId5.hashCode()); Assert.assertEquals(longId1, new LongId(longId1.asLong())); } diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueTest.java index a6ce40fb9..4f57897bc 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/IdValueTest.java @@ -27,10 +27,13 @@ import com.baidu.hugegraph.computer.core.graph.id.Id; import com.baidu.hugegraph.computer.core.graph.id.LongId; import com.baidu.hugegraph.computer.core.graph.id.Utf8Id; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.OptimizedUnsafeBytesOutput; import com.baidu.hugegraph.computer.core.io.StreamGraphInput; import com.baidu.hugegraph.computer.core.io.StreamGraphOutput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayInput; -import com.baidu.hugegraph.computer.core.io.UnsafeByteArrayOutput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesInput; +import com.baidu.hugegraph.computer.core.io.UnsafeBytesOutput; +import com.baidu.hugegraph.computer.core.util.IdValueUtil; import com.baidu.hugegraph.testutil.Assert; public class IdValueTest extends UnitTestBase { @@ -40,9 +43,11 @@ public void testCompare() { IdValue value1 = new LongId(123L).idValue(); IdValue value2 = new LongId(123L).idValue(); IdValue value3 = new LongId(321L).idValue(); + IdValue value4 = new LongId(322L).idValue(); Assert.assertEquals(0, value1.compareTo(value2)); Assert.assertLt(0, value2.compareTo(value3)); Assert.assertGt(0, value3.compareTo(value1)); + Assert.assertLt(0, value3.compareTo(value4)); } @Test @@ -50,18 +55,16 @@ public void testBytes() throws IOException { IdValue value1 = new Utf8Id("long id").idValue(); IdValue value2 = new Utf8Id("short").idValue(); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - value1.write(output); - value2.write(output); + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + value1.write(bao); + value2.write(bao); bytes = bao.toByteArray(); } IdValue value3 = new Utf8Id().idValue(); - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - value3.read(input); + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + value3.read(bai); Assert.assertEquals(value1, value3); - value3.read(input); + value3.read(bai); Assert.assertEquals(value2, value3); } } @@ -73,14 +76,14 @@ public void testWriteId() throws IOException { IdValue value1 = id1.idValue(); IdValue value2 = id2.idValue(); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - value1.writeId(output); - value2.writeId(output); + try (UnsafeBytesOutput bao = new UnsafeBytesOutput(); + StreamGraphOutput output = newStreamGraphOutput(bao)) { + output.writeId(IdValueUtil.toId(value1)); + output.writeId(IdValueUtil.toId(value2)); bytes = bao.toByteArray(); } - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes); + StreamGraphInput input = newStreamGraphInput(bai)) { Id id3 = input.readId(); Assert.assertEquals(id1, id3); Id id4 = input.readId(); diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/MockCustomValue.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/MockCustomValue.java index 10a94bc2b..1281ce6d8 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/MockCustomValue.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/graph/value/MockCustomValue.java @@ -22,8 +22,8 @@ import java.io.IOException; import java.util.Date; -import com.baidu.hugegraph.computer.core.io.GraphInput; -import com.baidu.hugegraph.computer.core.io.GraphOutput; +import com.baidu.hugegraph.computer.core.io.RandomAccessInput; +import com.baidu.hugegraph.computer.core.io.RandomAccessOutput; public class MockCustomValue implements Value { @@ -39,12 +39,12 @@ public ValueType type() { } @Override - public void read(GraphInput in) throws IOException { + public void read(RandomAccessInput in) throws IOException { this.value.setTime(in.readLong()); } @Override - public void write(GraphOutput out) throws IOException { + public void write(RandomAccessOutput out) throws IOException { out.writeLong(this.value.getTime()); } diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java index de1c93f5d..c8175b5cb 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/BufferedFileTest.java @@ -640,9 +640,11 @@ public void testCompare() throws IOException { // UnsafeByteArrayInput compare to BufferedFileInput File file3 = createTempFile(); try (BufferedFileInput fileInput = inputByString(file3, "apple")) { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); + @SuppressWarnings("resource") + UnsafeBytesOutput output = new UnsafeBytesOutput(); output.writeBytes("banana"); - RandomAccessInput input = new UnsafeByteArrayInput(output.buffer()); + @SuppressWarnings("resource") + RandomAccessInput input = new UnsafeBytesInput(output.buffer()); int result = input.compare(0, input.available(), fileInput, 0, fileInput.available()); Assert.assertTrue(result > 0); diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutputTest.java index 2aadfa2da..8fc63205f 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutputTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/CsvStructGraphOutputTest.java @@ -23,7 +23,6 @@ import java.io.IOException; import org.apache.commons.io.FileUtils; -import org.junit.Assert; import org.junit.Test; import com.baidu.hugegraph.computer.core.UnitTestBase; @@ -40,6 +39,7 @@ import com.baidu.hugegraph.computer.core.graph.value.IntValue; import com.baidu.hugegraph.computer.core.graph.value.LongValue; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; +import com.baidu.hugegraph.testutil.Assert; public class CsvStructGraphOutputTest extends UnitTestBase { diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/IOTestSuite.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/IOTestSuite.java index d1f799a1a..3b4c1fb23 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/IOTestSuite.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/IOTestSuite.java @@ -28,7 +28,9 @@ StreamGraphOutputInputTest.class, JsonStructGraphOutputTest.class, CsvStructGraphOutputTest.class, - UnsafeByteArrayTest.class, + StructRandomAccessOutputTest.class, + UnsafeBytesTest.class, + OptimizedUnsafeBytesTest.class, BufferedFileTest.class, BufferedStreamTest.class }) diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutputTest.java index 07aed7f65..c85b80db8 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutputTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/JsonStructGraphOutputTest.java @@ -23,7 +23,6 @@ import java.io.IOException; import org.apache.commons.io.FileUtils; -import org.junit.Assert; import org.junit.Test; import com.baidu.hugegraph.computer.core.UnitTestBase; @@ -40,6 +39,7 @@ import com.baidu.hugegraph.computer.core.graph.value.IntValue; import com.baidu.hugegraph.computer.core.graph.value.LongValue; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; +import com.baidu.hugegraph.testutil.Assert; public class JsonStructGraphOutputTest extends UnitTestBase { diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesTest.java new file mode 100644 index 000000000..b62d00158 --- /dev/null +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/OptimizedUnsafeBytesTest.java @@ -0,0 +1,62 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import java.io.IOException; + +import org.junit.Test; + +import com.baidu.hugegraph.testutil.Assert; + +@SuppressWarnings("resource") +public class OptimizedUnsafeBytesTest { + + @Test + public void testConstructor() { + UnsafeBytesOutput output = new OptimizedUnsafeBytesOutput(); + Assert.assertEquals(0, output.position()); + + UnsafeBytesOutput output2 = new OptimizedUnsafeBytesOutput(16); + Assert.assertEquals(0, output2.position()); + + UnsafeBytesInput input = new OptimizedUnsafeBytesInput(output.buffer()); + Assert.assertEquals(0, input.position()); + + UnsafeBytesInput input2 = new OptimizedUnsafeBytesInput(output.buffer(), + 4L); + Assert.assertEquals(0, input2.position()); + } + + @Test + public void testDuplicate() throws IOException { + OptimizedUnsafeBytesInput raw = inputByString("apple"); + OptimizedUnsafeBytesInput dup = raw.duplicate(); + raw.readByte(); + Assert.assertEquals(1, raw.position()); + Assert.assertEquals(0, dup.position()); + } + + private static OptimizedUnsafeBytesInput inputByString(String s) + throws IOException { + OptimizedUnsafeBytesOutput output = new OptimizedUnsafeBytesOutput(); + output.writeBytes(s); + return new OptimizedUnsafeBytesInput(output.toByteArray()); + } +} diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutputInputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutputInputTest.java index 3aae03e84..d88b65132 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutputInputTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StreamGraphOutputInputTest.java @@ -33,7 +33,9 @@ import com.baidu.hugegraph.computer.core.graph.id.LongId; import com.baidu.hugegraph.computer.core.graph.id.Utf8Id; import com.baidu.hugegraph.computer.core.graph.properties.Properties; +import com.baidu.hugegraph.computer.core.graph.value.DoubleValue; import com.baidu.hugegraph.computer.core.graph.value.IdValueList; +import com.baidu.hugegraph.computer.core.graph.value.IntValue; import com.baidu.hugegraph.computer.core.graph.value.LongValue; import com.baidu.hugegraph.computer.core.graph.vertex.Vertex; import com.baidu.hugegraph.testutil.Assert; @@ -53,15 +55,15 @@ public void testWriteReadVertex() throws IOException { LongValue longValue = new LongValue(999L); Vertex vertex1 = graphFactory().createVertex(longId, longValue); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeVertex(vertex1); bytes = bao.toByteArray(); } Vertex vertex2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); vertex2 = input.readVertex(); } Assert.assertEquals(vertex1, vertex2); @@ -81,15 +83,15 @@ public void testWriteReadEdges() throws IOException { edges1.add(factory.createEdge(new LongId(300), new LongValue(-1))); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeEdges(edges1); bytes = bao.toByteArray(); } Edges edges2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); edges2 = input.readEdges(); } Assert.assertEquals(edges1, edges2); @@ -105,15 +107,15 @@ public void testWriteReadEmptyEdges() throws IOException { Edges edges1 = graphFactory().createEdges(0); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeEdges(edges1); bytes = bao.toByteArray(); } Edges edges2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); edges2 = input.readEdges(); } Assert.assertEquals(edges1, edges2); @@ -133,41 +135,34 @@ public void testWriteReadProperties() throws IOException { */ Properties properties1 = graphFactory().createProperties(); properties1.put("age", new LongValue(18L)); - properties1.put("salary", new LongValue(20000L)); + properties1.put("salary", new DoubleValue(20000.50D)); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeProperties(properties1); bytes = bao.toByteArray(); } Properties properties2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); properties2 = input.readProperties(); } Assert.assertEquals(properties1, properties2); - // Let ValueType as ID_VALUE - UnitTestBase.updateOptions( - ComputerOptions.VALUE_TYPE, "ID_VALUE", - ComputerOptions.VALUE_NAME, "value", - ComputerOptions.EDGES_NAME, "value" - ); - properties1 = graphFactory().createProperties(); properties1.put("name", new Utf8Id("marko").idValue()); - properties1.put("city", new Utf8Id("Beijing").idValue()); + properties1.put("age", new IntValue(18)); - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeProperties(properties1); bytes = bao.toByteArray(); } - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); properties2 = input.readProperties(); } Assert.assertEquals(properties1, properties2); @@ -177,8 +172,8 @@ public void testWriteReadProperties() throws IOException { public void testWriteReadId() throws IOException { LongId longId1 = new LongId(100L); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeId(longId1); bytes = bao.toByteArray(); } @@ -188,8 +183,9 @@ public void testWriteReadId() throws IOException { Id longId2 = new LongId(); Assert.assertEquals(0L, longId2.asLong()); - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = + new OptimizedUnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); longId2 = input.readId(); } Assert.assertEquals(100L, longId2.asLong()); @@ -205,8 +201,8 @@ public void testWriteReadValue() throws IOException { LongValue longValue1 = new LongValue(100L); byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeValue(longValue1); bytes = bao.toByteArray(); } @@ -215,8 +211,8 @@ public void testWriteReadValue() throws IOException { Assert.assertArrayEquals(expect, bytes); LongValue longValue2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); longValue2 = (LongValue) input.readValue(); } Assert.assertEquals(100L, longValue2.value()); @@ -233,8 +229,8 @@ public void testWriteReadValue() throws IOException { IdValueList idValueList1 = new IdValueList(); idValueList1.add(longId1.idValue()); idValueList1.add(longId2.idValue()); - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + StreamGraphOutput output = newStreamGraphOutput(bao); output.writeValue(idValueList1); bytes = bao.toByteArray(); } @@ -243,8 +239,8 @@ public void testWriteReadValue() throws IOException { Assert.assertArrayEquals(expect, bytes); IdValueList idValueList2; - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + StreamGraphInput input = newStreamGraphInput(bai); idValueList2 = (IdValueList) input.readValue(); } Assert.assertTrue(ListUtils.isEqualList( @@ -252,310 +248,4 @@ public void testWriteReadValue() throws IOException { longId2.idValue()), idValueList2.values())); } - - @Test - public void testReadWriteFullInt() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeFullInt(Integer.MIN_VALUE); - output.writeFullInt(Integer.MAX_VALUE); - output.writeFullInt(0); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(Integer.MIN_VALUE, input.readFullInt()); - Assert.assertEquals(Integer.MAX_VALUE, input.readFullInt()); - Assert.assertEquals(0, input.readFullInt()); - } - } - - @Test - public void testReadWriteFullLong() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeFullLong(Long.MIN_VALUE); - output.writeFullLong(Long.MAX_VALUE); - output.writeFullLong(0L); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(Long.MIN_VALUE, input.readFullLong()); - Assert.assertEquals(Long.MAX_VALUE, input.readFullLong()); - Assert.assertEquals(0L, input.readFullLong()); - } - } - - @Test - public void testWriteReadVInt() throws IOException { - testBytesStreamWriteReadVInt(new byte[]{0}, 0); - testBytesStreamWriteReadVInt(new byte[]{1}, 1); - testBytesStreamWriteReadVInt(new byte[]{(byte) 0x7f}, 127); - testBytesStreamWriteReadVInt(new byte[]{(byte) 0x81, 0}, 128); - testBytesStreamWriteReadVInt(new byte[]{(byte) 0xff, (byte) 0x7f}, - 16383); - testBytesStreamWriteReadVInt(new byte[]{(byte) 0x81, (byte) 0x80, 0}, - 16384); - testBytesStreamWriteReadVInt(new byte[]{(byte) 0x81, (byte) 0x80, 1}, - 16385); - testBytesStreamWriteReadVInt(new byte[]{-113, -1, -1, -1, 127}, -1); - testBytesStreamWriteReadVInt(new byte[]{-121, -1, -1, -1, 127}, - Integer.MAX_VALUE); - testBytesStreamWriteReadVInt(new byte[]{-120, -128, -128, -128, 0}, - Integer.MIN_VALUE); - } - - @Test - public void testWriteReadVLong() throws IOException { - testBytesStreamWriteReadVLong(new byte[]{0}, 0L); - testBytesStreamWriteReadVLong(new byte[]{1}, 1L); - testBytesStreamWriteReadVLong(new byte[]{(byte) 0x7f}, 127L); - testBytesStreamWriteReadVLong(new byte[]{(byte) 0x81, 0}, 128L); - testBytesStreamWriteReadVLong(new byte[]{(byte) 0xff, (byte) 0x7f}, - 16383L); - testBytesStreamWriteReadVLong(new byte[]{(byte) 0x81, (byte) 0x80, 0}, - 16384L); - testBytesStreamWriteReadVLong(new byte[]{(byte) 0x81, (byte) 0x80, 1}, - 16385L); - testBytesStreamWriteReadVLong(new byte[]{-127, -1, -1, -1, -1, - -1, -1, -1, -1, 127}, -1L); - testBytesStreamWriteReadVLong(new byte[]{-121, -1, -1, -1, 127}, - Integer.MAX_VALUE); - testBytesStreamWriteReadVLong(new byte[]{-127, -1, -1, -1, -1, - -8, -128, -128, -128, 0}, - Integer.MIN_VALUE); - testBytesStreamWriteReadVLong(new byte[]{-1, -1, -1, -1, -1, - -1, -1, -1, 127}, - Long.MAX_VALUE); - testBytesStreamWriteReadVLong(new byte[]{-127, -128, -128, -128, -128, - -128, -128, -128, -128, 0}, - Long.MIN_VALUE); - } - - @Test - public void testWriteReadString() throws IOException { - testBytesStreamWriteReadString(new byte[]{0}, ""); - testBytesStreamWriteReadString(new byte[]{1, 49}, "1"); - testBytesStreamWriteReadString(new byte[]{3, 55, 56, 57}, "789"); - testBytesStreamWriteReadString(new byte[]{5, 65, 66, 67, 68, 69}, - "ABCDE"); - } - - public static void testBytesStreamWriteReadVInt(byte[] bytes, int value) - throws IOException { - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeVInt(value); - Assert.assertArrayEquals(bytes, bao.toByteArray()); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - int readValue = input.readVInt(); - Assert.assertEquals(value, readValue); - } - } - - public static void testBytesStreamWriteReadVLong(byte[] bytes, long value) - throws IOException { - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeVLong(value); - Assert.assertArrayEquals(bytes, bao.toByteArray()); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - long readValue = input.readVLong(); - Assert.assertEquals(value, readValue); - } - } - - public static void testBytesStreamWriteReadString(byte[] bytes, - String value) - throws IOException { - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeString(value); - output.close(); - Assert.assertArrayEquals(bytes, bao.toByteArray()); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - String readValue = input.readString(); - Assert.assertEquals(value, readValue); - } - } - - @Test - public void testPosition() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - Assert.assertEquals(0L, output.position()); - output.writeFullLong(Long.MAX_VALUE); - Assert.assertEquals(8L, output.position()); - output.close(); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(0L, input.position()); - Assert.assertEquals(Long.MAX_VALUE, input.readFullLong()); - Assert.assertEquals(8L, input.position()); - } - } - - @Test - public void testSeek() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeFullLong(Long.MAX_VALUE); - output.seek(0L); - output.writeFullLong(Long.MIN_VALUE); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(Long.MIN_VALUE, input.readFullLong()); - input.seek(0L); - Assert.assertEquals(Long.MIN_VALUE, input.readFullLong()); - } - } - - @Test - public void testSkip() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeFullLong(Long.MAX_VALUE); - output.skip(4L); - output.writeFullLong(Long.MIN_VALUE); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(Long.MAX_VALUE, input.readFullLong()); - input.skip(4L); - Assert.assertEquals(Long.MIN_VALUE, input.readFullLong()); - } - } - - @Test - public void testUInt8() throws IOException { - byte[] bytes; - int value1 = 0; - int value2 = 255; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newOptimizedStreamGraphOutput(bao)) { - output.writeUInt8(value1); - output.writeUInt8(value2); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newOptimizedStreamGraphInput(bai)) { - Assert.assertEquals(value1, input.readUInt8()); - Assert.assertEquals(value2, input.readUInt8()); - } - } - - @Test - public void testUInt16() throws IOException { - byte[] bytes; - int value1 = 0; - int value2 = 65535; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newStreamGraphOutput(bao)) { - output.writeUInt16(value1); - output.writeUInt16(value2); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newStreamGraphInput(bai)) { - Assert.assertEquals(value1, input.readUInt16()); - Assert.assertEquals(value2, input.readUInt16()); - } - } - - @Test - public void testUInt32() throws IOException { - byte[] bytes; - long value1 = 0L; - long value2 = Integer.MAX_VALUE + 0L; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newStreamGraphOutput(bao)) { - output.writeUInt32(value1); - output.writeUInt32(value2); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newStreamGraphInput(bai)) { - Assert.assertEquals(value1, input.readUInt32()); - Assert.assertEquals(value2, input.readUInt32()); - } - } - - @Test - public void testShort() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newStreamGraphOutput(bao)) { - output.writeShort(Short.MAX_VALUE); - output.writeShort(Short.MIN_VALUE); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newStreamGraphInput(bai)) { - Assert.assertEquals(Short.MAX_VALUE, input.readShort()); - Assert.assertEquals(Short.MIN_VALUE, input.readShort()); - } - } - - @Test - public void testInt() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newStreamGraphOutput(bao)) { - output.writeInt(Integer.MAX_VALUE); - output.writeInt(Integer.MIN_VALUE); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newStreamGraphInput(bai)) { - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); - } - } - - @Test - public void testLong() throws IOException { - byte[] bytes; - try (UnsafeByteArrayOutput bao = new UnsafeByteArrayOutput(); - StreamGraphOutput output = newStreamGraphOutput(bao)) { - output.writeLong(Long.MAX_VALUE); - output.writeLong(Long.MIN_VALUE); - bytes = bao.toByteArray(); - } - - try (UnsafeByteArrayInput bai = new UnsafeByteArrayInput(bytes); - StreamGraphInput input = newStreamGraphInput(bai)) { - Assert.assertEquals(Long.MAX_VALUE, input.readLong()); - Assert.assertEquals(Long.MIN_VALUE, input.readLong()); - } - } } diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutputTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutputTest.java new file mode 100644 index 000000000..5f6fd73e2 --- /dev/null +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/StructRandomAccessOutputTest.java @@ -0,0 +1,218 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import java.io.File; +import java.io.IOException; +import java.util.UUID; + +import org.apache.commons.io.FileUtils; +import org.junit.Test; + +import com.baidu.hugegraph.testutil.Assert; + +public class StructRandomAccessOutputTest { + + @Test + public void testPosition() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + Assert.assertEquals(0L, srao.position()); + srao.writeLong(12345678); + Assert.assertEquals(8L, srao.position()); + } + } + + @Test + public void testSeek() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (int i = -128; i <= 127; i++) { + srao.writeInt(i); + } + Assert.assertEquals(678L, srao.position()); + // Overwrite last 8 bytes + srao.seek(srao.position() - 8); + srao.writeInt(1000); + srao.writeInt(2000); + Assert.assertEquals(678L, srao.position()); + } + } + + @Test + public void testSkip() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + Assert.assertEquals(0L, srao.position()); + output.writeInt(1000); + output.writeInt(2000); + Assert.assertEquals(8L, srao.position()); + srao.skip(4); + Assert.assertEquals(12L, srao.position()); + } + } + + @Test + public void testWriteIntWithPosition() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + Assert.assertEquals(0L, srao.position()); + srao.writeInt(4, 1000); + Assert.assertEquals(8L, srao.position()); + srao.writeInt(4, 2000); + Assert.assertEquals(8L, srao.position()); + } + } + + @Test + public void testWriteByInput() throws IOException { + // Input class is UnsafeByteArrayInput + String uuid = UUID.randomUUID().toString(); + UnsafeBytesInput input = inputByString(uuid); + UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output); + srao.write(input, 0, input.available()); + Assert.assertEquals(50L, srao.position()); + srao.close(); + + // Input class isn't UnsafeByteArrayInput + File tempFile = File.createTempFile(UUID.randomUUID().toString(), ""); + BufferedFileOutput fileOutput = null; + BufferedFileInput fileInput = null; + try { + fileOutput = new BufferedFileOutput(tempFile); + fileOutput.writeBytes(uuid); + fileOutput.close(); + fileInput = new BufferedFileInput(tempFile); + srao.write(fileInput, 0, fileInput.available()); + Assert.assertEquals(100L, srao.position()); + srao.close(); + } finally { + if (fileInput != null) { + fileInput.close(); + } + if (fileOutput != null) { + fileOutput.close(); + } + FileUtils.deleteQuietly(tempFile); + } + } + + @Test + public void testByte() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (int i = -128; i <= 127; i++) { + srao.write(i); + } + Assert.assertEquals(678L, srao.position()); + } + + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (int i = -128; i <= 127; i++) { + srao.writeByte(i); + } + Assert.assertEquals(678L, srao.position()); + } + } + + @Test + public void testByteArray() throws IOException { + byte[] bytes = "testByteArray".getBytes(); + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + srao.write(bytes); + Assert.assertEquals(22L, srao.position()); + } + } + + @Test + public void testWritePartByteArray() throws IOException { + byte[] bytes = "testByteArray".getBytes(); + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + srao.write(bytes, 1, bytes.length - 1); + Assert.assertEquals(18L, srao.position()); + } + } + + @Test + public void testShort() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (short i = -128; i <= 127; i++) { + srao.writeShort(i); + } + Assert.assertEquals(678L, srao.position()); + srao.writeShort(1000); + Assert.assertEquals(682L, srao.position()); + } + } + + @Test + public void testChar() throws IOException { + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (char i = 'a'; i <= 'z'; i++) { + srao.writeChar(i); + } + Assert.assertEquals(75L, srao.position()); + } + } + + @Test + public void testBytes() throws IOException { + String bytes = "testByteArray"; + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + srao.writeBytes(bytes); + Assert.assertEquals(15L, srao.position()); + } + } + + @Test + public void testChars() throws IOException { + String chars = "testByteArray"; + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + srao.writeChars(chars); + Assert.assertEquals(15L, srao.position()); + } + } + + @Test + public void testUTF() throws IOException { + String prefix = "random string"; + try (UnsafeBytesOutput output = new UnsafeBytesOutput(); + RandomAccessOutput srao = new StructRandomAccessOutput(output)) { + for (int i = 0; i <= 9; i++) { + srao.writeUTF(prefix + i); + } + Assert.assertEquals(160L, srao.position()); + } + } + + private static UnsafeBytesInput inputByString(String s) throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeBytes(s); + return new UnsafeBytesInput(output.toByteArray()); + } +} diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayTest.java deleted file mode 100644 index f93ae06f1..000000000 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeByteArrayTest.java +++ /dev/null @@ -1,474 +0,0 @@ -/* - * Copyright 2017 HugeGraph Authors - * - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with this - * work for additional information regarding copyright ownership. The ASF - * licenses this file to You under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations - * under the License. - */ - -package com.baidu.hugegraph.computer.core.io; - -import java.io.File; -import java.io.IOException; -import java.io.UTFDataFormatException; -import java.util.UUID; - -import org.apache.commons.io.FileUtils; -import org.junit.Test; - -import com.baidu.hugegraph.computer.core.UnitTestBase; -import com.baidu.hugegraph.computer.core.common.Constants; -import com.baidu.hugegraph.computer.core.common.exception.ComputerException; -import com.baidu.hugegraph.testutil.Assert; - -@SuppressWarnings("resource") -public class UnsafeByteArrayTest { - - @Test - public void testConstructor() { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - Assert.assertEquals(0, output.position()); - - UnsafeByteArrayOutput output2 = new UnsafeByteArrayOutput(16); - Assert.assertEquals(0, output2.position()); - - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.buffer()); - Assert.assertEquals(0, input.position()); - } - - @Test - public void testBoolean() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeBoolean(true); - output.writeBoolean(false); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - Assert.assertTrue(input.readBoolean()); - Assert.assertFalse(input.readBoolean()); - } - - @Test - public void testByte() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.write(i); - } - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 127; i++) { - int value = input.readByte(); - Assert.assertEquals(i, value); - } - } - - @Test - public void testUnsignedByte() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = 0; i <= 255; i++) { - output.write(i); - } - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = 0; i <= 255; i++) { - int value = input.readUnsignedByte(); - Assert.assertEquals(i, value); - } - } - - @Test - public void testShort() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (short i = -128; i <= 127; i++) { - output.writeShort(i); - } - output.writeShort(Short.MAX_VALUE); - output.writeShort(Short.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 127; i++) { - Assert.assertEquals(i, input.readShort()); - } - Assert.assertEquals(Short.MAX_VALUE, input.readShort()); - Assert.assertEquals(Short.MIN_VALUE, input.readShort()); - } - - @Test - public void testUnsignedShort() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (short i = 0; i <= 255; i++) { - output.writeShort(i); - } - output.writeShort(Short.MAX_VALUE); - output.writeShort(Short.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = 0; i <= 255; i++) { - Assert.assertEquals(i, input.readUnsignedShort()); - } - Assert.assertEquals(Short.MAX_VALUE, input.readUnsignedShort()); - Assert.assertEquals(Short.MIN_VALUE & 0xFFFF, - input.readUnsignedShort()); - } - - @Test - public void testChar() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (char i = 'a'; i <= 'z'; i++) { - output.writeChar(i); - } - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (char i = 'a'; i <= 'z'; i++) { - char value = input.readChar(); - Assert.assertEquals(i, value); - } - } - - @Test - public void testInt() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.writeInt(i); - } - output.writeInt(Integer.MAX_VALUE); - output.writeInt(Integer.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 127; i++) { - Assert.assertEquals(i, input.readInt()); - } - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); - } - - @Test - public void testWriteIntWithPosition() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - long position = output.skip(Constants.INT_LEN); - output.writeInt(2); - output.writeInt(position, 1); - - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - Assert.assertEquals(1, input.readInt()); - Assert.assertEquals(2, input.readInt()); - } - - @Test - public void testOverWriteWithPosition() { - Assert.assertThrows(ComputerException.class, () -> { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(4); - output.writeInt(1, 100); - }); - - Assert.assertThrows(ComputerException.class, () -> { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(4); - output.writeInt(2, 100); - }); - - Assert.assertThrows(ComputerException.class, () -> { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(4); - output.writeInt(3, 100); - }); - - Assert.assertThrows(ComputerException.class, () -> { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(4); - output.writeInt(4, 100); - }); - } - - @Test - public void testLong() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (long i = -128; i <= 127; i++) { - output.writeLong(i); - } - output.writeLong(Long.MAX_VALUE); - output.writeLong(Long.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (long i = -128; i <= 127; i++) { - Assert.assertEquals(i, input.readLong()); - } - Assert.assertEquals(Long.MAX_VALUE, input.readLong()); - Assert.assertEquals(Long.MIN_VALUE, input.readLong()); - } - - @Test - public void testFloat() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.writeFloat(i); - } - output.writeFloat(Float.MAX_VALUE); - output.writeFloat(Float.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 127; i++) { - Assert.assertEquals(i, input.readFloat(), 0.0D); - } - Assert.assertEquals(Float.MAX_VALUE, input.readFloat(), 0.0D); - Assert.assertEquals(Float.MIN_VALUE, input.readFloat(), 0.0D); - } - - @Test - public void testDouble() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.writeDouble(i); - } - output.writeDouble(Double.MAX_VALUE); - output.writeDouble(Double.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 127; i++) { - Assert.assertEquals(i, input.readDouble(), 0.0D); - } - Assert.assertEquals(Double.MAX_VALUE, input.readDouble(), 0.0D); - Assert.assertEquals(Double.MIN_VALUE, input.readDouble(), 0.0D); - } - - @Test - public void testByteArray() throws IOException { - byte[] bytes = "testByteArray".getBytes(); - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.write(bytes); - byte[] bytesRead = new byte[bytes.length]; - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - input.readFully(bytesRead); - Assert.assertArrayEquals(bytes, bytesRead); - } - - @Test - public void testWritePartByteArray() throws IOException { - byte[] bytes = "testByteArray".getBytes(); - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.write(bytes, 1, bytes.length - 1); - byte[] bytesRead = new byte[bytes.length]; - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - input.readFully(bytesRead, 1, bytes.length - 1); - bytesRead[0] = bytes[0]; - Assert.assertArrayEquals(bytes, bytesRead); - } - - @Test - public void testWriteChars() throws IOException { - String chars = "testByteArray"; - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeChars(chars); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = 0; i < chars.length(); i++) { - char c = input.readChar(); - Assert.assertEquals(chars.charAt(i), c); - } - } - - @Test - public void testReadLine() { - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - Constants.EMPTY_BYTES); - Assert.assertThrows(ComputerException.class, () -> { - input.readLine(); - }); - } - - @Test - public void testUTFWithChineseCharacters() throws IOException { - String s = "带汉字的字符串"; - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeUTF(s); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - String value = input.readUTF(); - Assert.assertEquals(s, value); - } - - @Test - public void testUTF() throws IOException { - String prefix = "random string"; - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.writeUTF(prefix + i); - } - UnsafeByteArrayInput input = new UnsafeByteArrayInput(output.buffer()); - for (int i = -128; i <= 127; i++) { - String value = input.readUTF(); - Assert.assertEquals(prefix + i, value); - } - } - - @Test - public void testUTFBoundary() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - String s1 = UnitTestBase.randomString(65535); - output.writeUTF(s1); - String s2 = UnitTestBase.randomString(65536); - Assert.assertThrows(UTFDataFormatException.class, () -> { - output.writeUTF(s2); - }); - UnsafeByteArrayInput input = new UnsafeByteArrayInput(output.buffer()); - String value = input.readUTF(); - Assert.assertEquals(s1, value); - } - - @Test - public void testSkipBytes() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - long position = output.skip(4); - Assert.assertEquals(0, position); - output.writeInt(Integer.MAX_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - int bytesSkipped = input.skipBytes(4); - Assert.assertEquals(4, bytesSkipped); - Assert.assertEquals(4, input.remaining()); - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(0, input.remaining()); - Assert.assertEquals(0, input.skipBytes(1)); - } - - @Test - public void testSkip() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.skip(4); - output.writeInt(Integer.MAX_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - input.skipBytes(4); - Assert.assertEquals(4, input.remaining()); - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(0, input.remaining()); - Assert.assertEquals(0, input.skipBytes(1)); - } - - @Test - public void testBuffer() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeInt(Integer.MAX_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.buffer(), output.position()); - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - } - - @Test - public void testOverRead() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeInt(Integer.MAX_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.buffer(), output.position()); - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertThrows(ComputerException.class, () -> { - input.readInt(); - }); - } - - @Test - public void testSeek() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.writeInt(i); - } - // Overwrite last 2 elements - output.seek(256 * 4 - 8); - output.writeInt(Integer.MAX_VALUE); - output.writeInt(Integer.MIN_VALUE); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = -128; i <= 125; i++) { - Assert.assertEquals(i, input.readInt()); - } - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); - input.seek(256 * 4 - 8); - Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); - Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); - } - - @Test - public void testAvailable() throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - for (int i = -128; i <= 127; i++) { - output.write(i); - } - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - for (int i = 0; i < 256; i++) { - Assert.assertEquals(256 - i, input.available()); - input.readByte(); - } - } - - @Test - public void testWriteByInput() throws IOException { - // Input class is UnsafeByteArrayInput - String uuid = UUID.randomUUID().toString(); - UnsafeByteArrayInput input = inputByString(uuid); - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.write(input, 0, input.available()); - Assert.assertEquals(uuid, new String(output.toByteArray())); - - // Input class isn't UnsafeByteArrayInput - File tempFile = File.createTempFile(UUID.randomUUID().toString(), ""); - BufferedFileOutput fileOutput = null; - BufferedFileInput fileInput = null; - try { - fileOutput = new BufferedFileOutput(tempFile); - fileOutput.writeBytes(uuid); - fileOutput.close(); - fileInput = new BufferedFileInput(tempFile); - output = new UnsafeByteArrayOutput(); - output.write(fileInput, 0, fileInput.available()); - Assert.assertEquals(uuid, new String(output.toByteArray())); - } finally { - if (fileInput != null) { - fileInput.close(); - } - if (fileOutput != null) { - fileOutput.close(); - } - FileUtils.deleteQuietly(tempFile); - } - } - - @Test - public void testReadBytes() throws IOException { - String uuid = UUID.randomUUID().toString(); - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeBytes(uuid); - UnsafeByteArrayInput input = new UnsafeByteArrayInput( - output.toByteArray()); - byte[] bytes = input.readBytes(uuid.length()); - Assert.assertEquals(uuid, new String(bytes)); - } - - @Test - public void testCompare() throws IOException { - UnsafeByteArrayInput apple = inputByString("apple"); - UnsafeByteArrayInput egg = inputByString("egg"); - Assert.assertTrue(apple.compare(0, 2, egg, 0, 2) < 0); - Assert.assertTrue(apple.compare(1, 3, egg, 0, 2) > 0); - Assert.assertEquals(0, apple.compare(4, 1, egg, 0, 1)); - } - - private static UnsafeByteArrayInput inputByString(String s) - throws IOException { - UnsafeByteArrayOutput output = new UnsafeByteArrayOutput(); - output.writeBytes(s); - return new UnsafeByteArrayInput(output.toByteArray()); - } -} diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesTest.java new file mode 100644 index 000000000..54326bed9 --- /dev/null +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/io/UnsafeBytesTest.java @@ -0,0 +1,645 @@ +/* + * Copyright 2017 HugeGraph Authors + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations + * under the License. + */ + +package com.baidu.hugegraph.computer.core.io; + +import java.io.File; +import java.io.IOException; +import java.io.UTFDataFormatException; +import java.util.UUID; + +import org.apache.commons.io.FileUtils; +import org.junit.Test; + +import com.baidu.hugegraph.computer.core.UnitTestBase; +import com.baidu.hugegraph.computer.core.common.Constants; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; +import com.baidu.hugegraph.testutil.Assert; + +@SuppressWarnings("resource") +public class UnsafeBytesTest { + + @Test + public void testConstructor() { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + Assert.assertEquals(0, output.position()); + + UnsafeBytesOutput output2 = new UnsafeBytesOutput(16); + Assert.assertEquals(0, output2.position()); + + UnsafeBytesInput input = new UnsafeBytesInput(output.buffer()); + Assert.assertEquals(0, input.position()); + } + + @Test + public void testBoolean() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeBoolean(true); + output.writeBoolean(false); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + Assert.assertTrue(input.readBoolean()); + Assert.assertFalse(input.readBoolean()); + } + + @Test + public void testByte() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.write(i); + } + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 127; i++) { + int value = input.readByte(); + Assert.assertEquals(i, value); + } + } + + @Test + public void testUnsignedByte() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = 0; i <= 255; i++) { + output.write(i); + } + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = 0; i <= 255; i++) { + int value = input.readUnsignedByte(); + Assert.assertEquals(i, value); + } + } + + @Test + public void testShort() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (short i = -128; i <= 127; i++) { + output.writeShort(i); + } + output.writeShort(Short.MAX_VALUE); + output.writeShort(Short.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 127; i++) { + Assert.assertEquals(i, input.readShort()); + } + Assert.assertEquals(Short.MAX_VALUE, input.readShort()); + Assert.assertEquals(Short.MIN_VALUE, input.readShort()); + } + + @Test + public void testUnsignedShort() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (short i = 0; i <= 255; i++) { + output.writeShort(i); + } + output.writeShort(Short.MAX_VALUE); + output.writeShort(Short.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = 0; i <= 255; i++) { + Assert.assertEquals(i, input.readUnsignedShort()); + } + Assert.assertEquals(Short.MAX_VALUE, input.readUnsignedShort()); + Assert.assertEquals(Short.MIN_VALUE & 0xFFFF, + input.readUnsignedShort()); + } + + @Test + public void testChar() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (char i = 'a'; i <= 'z'; i++) { + output.writeChar(i); + } + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (char i = 'a'; i <= 'z'; i++) { + char value = input.readChar(); + Assert.assertEquals(i, value); + } + } + + @Test + public void testInt() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.writeInt(i); + } + output.writeInt(Integer.MAX_VALUE); + output.writeInt(Integer.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 127; i++) { + Assert.assertEquals(i, input.readInt()); + } + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); + } + + @Test + public void testWriteIntWithPosition() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + long position = output.skip(Constants.INT_LEN); + output.writeInt(2); + output.writeInt(position, 1); + + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + Assert.assertEquals(1, input.readInt()); + Assert.assertEquals(2, input.readInt()); + } + + @Test + public void testOverWriteWithPosition() { + Assert.assertThrows(ComputerException.class, () -> { + UnsafeBytesOutput output = new UnsafeBytesOutput(4); + output.writeInt(1, 100); + }); + + Assert.assertThrows(ComputerException.class, () -> { + UnsafeBytesOutput output = new UnsafeBytesOutput(4); + output.writeInt(2, 100); + }); + + Assert.assertThrows(ComputerException.class, () -> { + UnsafeBytesOutput output = new UnsafeBytesOutput(4); + output.writeInt(3, 100); + }); + + Assert.assertThrows(ComputerException.class, () -> { + UnsafeBytesOutput output = new UnsafeBytesOutput(4); + output.writeInt(4, 100); + }); + } + + @Test + public void testLong() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (long i = -128; i <= 127; i++) { + output.writeLong(i); + } + output.writeLong(Long.MAX_VALUE); + output.writeLong(Long.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (long i = -128; i <= 127; i++) { + Assert.assertEquals(i, input.readLong()); + } + Assert.assertEquals(Long.MAX_VALUE, input.readLong()); + Assert.assertEquals(Long.MIN_VALUE, input.readLong()); + } + + @Test + public void testFloat() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.writeFloat(i); + } + output.writeFloat(Float.MAX_VALUE); + output.writeFloat(Float.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 127; i++) { + Assert.assertEquals(i, input.readFloat(), 0.0D); + } + Assert.assertEquals(Float.MAX_VALUE, input.readFloat(), 0.0D); + Assert.assertEquals(Float.MIN_VALUE, input.readFloat(), 0.0D); + } + + @Test + public void testDouble() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.writeDouble(i); + } + output.writeDouble(Double.MAX_VALUE); + output.writeDouble(Double.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 127; i++) { + Assert.assertEquals(i, input.readDouble(), 0.0D); + } + Assert.assertEquals(Double.MAX_VALUE, input.readDouble(), 0.0D); + Assert.assertEquals(Double.MIN_VALUE, input.readDouble(), 0.0D); + } + + @Test + public void testByteArray() throws IOException { + byte[] bytes = "testByteArray".getBytes(); + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.write(bytes); + byte[] bytesRead = new byte[bytes.length]; + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + input.readFully(bytesRead); + Assert.assertArrayEquals(bytes, bytesRead); + } + + @Test + public void testWritePartByteArray() throws IOException { + byte[] bytes = "testByteArray".getBytes(); + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.write(bytes, 1, bytes.length - 1); + byte[] bytesRead = new byte[bytes.length]; + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + input.readFully(bytesRead, 1, bytes.length - 1); + bytesRead[0] = bytes[0]; + Assert.assertArrayEquals(bytes, bytesRead); + } + + @Test + public void testWriteChars() throws IOException { + String chars = "testByteArray"; + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeChars(chars); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = 0; i < chars.length(); i++) { + char c = input.readChar(); + Assert.assertEquals(chars.charAt(i), c); + } + } + + @Test + public void testReadLine() { + UnsafeBytesInput input = new UnsafeBytesInput(Constants.EMPTY_BYTES); + Assert.assertThrows(ComputerException.class, () -> { + input.readLine(); + }); + } + + @Test + public void testUTFWithChineseCharacters() throws IOException { + String s = "带汉字的字符串"; + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeUTF(s); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + String value = input.readUTF(); + Assert.assertEquals(s, value); + } + + @Test + public void testUTF() throws IOException { + String prefix = "random string"; + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.writeUTF(prefix + i); + } + UnsafeBytesInput input = new UnsafeBytesInput(output.buffer()); + for (int i = -128; i <= 127; i++) { + String value = input.readUTF(); + Assert.assertEquals(prefix + i, value); + } + } + + @Test + public void testUTFBoundary() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + String s1 = UnitTestBase.randomString(65535); + output.writeUTF(s1); + String s2 = UnitTestBase.randomString(65536); + Assert.assertThrows(UTFDataFormatException.class, () -> { + output.writeUTF(s2); + }); + UnsafeBytesInput input = new UnsafeBytesInput(output.buffer()); + String value = input.readUTF(); + Assert.assertEquals(s1, value); + } + + @Test + public void testSkipBytes() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + long position = output.skip(4); + Assert.assertEquals(0, position); + output.writeInt(Integer.MAX_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + int bytesSkipped = input.skipBytes(4); + Assert.assertEquals(4, bytesSkipped); + Assert.assertEquals(4, input.remaining()); + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertEquals(0, input.remaining()); + Assert.assertEquals(0, input.skipBytes(1)); + } + + @Test + public void testSkip() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.skip(4); + output.writeInt(Integer.MAX_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + input.skipBytes(4); + Assert.assertEquals(4, input.remaining()); + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertEquals(0, input.remaining()); + Assert.assertEquals(0, input.skipBytes(1)); + } + + @Test + public void testBuffer() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeInt(Integer.MAX_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.buffer(), + output.position()); + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + } + + @Test + public void testOverRead() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeInt(Integer.MAX_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.buffer(), + output.position()); + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertThrows(ComputerException.class, () -> { + input.readInt(); + }); + } + + @Test + public void testSeek() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.writeInt(i); + } + // Overwrite last 2 elements + output.seek(256 * 4 - 8); + output.writeInt(Integer.MAX_VALUE); + output.writeInt(Integer.MIN_VALUE); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = -128; i <= 125; i++) { + Assert.assertEquals(i, input.readInt()); + } + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); + input.seek(256 * 4 - 8); + Assert.assertEquals(Integer.MAX_VALUE, input.readInt()); + Assert.assertEquals(Integer.MIN_VALUE, input.readInt()); + } + + @Test + public void testAvailable() throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + for (int i = -128; i <= 127; i++) { + output.write(i); + } + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + for (int i = 0; i < 256; i++) { + Assert.assertEquals(256 - i, input.available()); + input.readByte(); + } + } + + @Test + public void testPosition() throws IOException { + byte[] bytes; + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + Assert.assertEquals(0L, bao.position()); + bao.writeLong(Long.MAX_VALUE); + Assert.assertEquals(8L, bao.position()); + bytes = bao.toByteArray(); + } + + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + Assert.assertEquals(0L, bai.position()); + Assert.assertEquals(Long.MAX_VALUE, bai.readLong()); + Assert.assertEquals(8L, bai.position()); + } + } + + @Test + public void testWriteByInput() throws IOException { + // Input class is UnsafeByteArrayInput + String uuid = UUID.randomUUID().toString(); + UnsafeBytesInput input = inputByString(uuid); + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.write(input, 0, input.available()); + Assert.assertEquals(uuid, new String(output.toByteArray())); + + // Input class isn't UnsafeByteArrayInput + File tempFile = File.createTempFile(UUID.randomUUID().toString(), ""); + BufferedFileOutput fileOutput = null; + BufferedFileInput fileInput = null; + try { + fileOutput = new BufferedFileOutput(tempFile); + fileOutput.writeBytes(uuid); + fileOutput.close(); + fileInput = new BufferedFileInput(tempFile); + output = new UnsafeBytesOutput(); + output.write(fileInput, 0, fileInput.available()); + Assert.assertEquals(uuid, new String(output.toByteArray())); + } finally { + if (fileInput != null) { + fileInput.close(); + } + if (fileOutput != null) { + fileOutput.close(); + } + FileUtils.deleteQuietly(tempFile); + } + } + + @Test + public void testReadBytes() throws IOException { + String uuid = UUID.randomUUID().toString(); + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeBytes(uuid); + UnsafeBytesInput input = new UnsafeBytesInput(output.toByteArray()); + byte[] bytes = input.readBytes(uuid.length()); + Assert.assertEquals(uuid, new String(bytes)); + } + + @Test + public void testCompare() throws IOException { + UnsafeBytesInput apple = inputByString("apple"); + UnsafeBytesInput egg = inputByString("egg"); + Assert.assertTrue(apple.compare(0, 2, egg, 0, 2) < 0); + Assert.assertTrue(apple.compare(1, 3, egg, 0, 2) > 0); + Assert.assertEquals(0, apple.compare(4, 1, egg, 0, 1)); + } + + @Test + public void testDuplicate() throws IOException { + UnsafeBytesInput raw = inputByString("apple"); + UnsafeBytesInput dup = raw.duplicate(); + raw.readByte(); + Assert.assertEquals(1, raw.position()); + Assert.assertEquals(0, dup.position()); + + String uuid = UUID.randomUUID().toString(); + File tempFile = File.createTempFile(UUID.randomUUID().toString(), ""); + BufferedFileOutput fileOutput = null; + BufferedFileInput fileInput = null; + try { + fileOutput = new BufferedFileOutput(tempFile); + fileOutput.writeBytes(uuid); + fileOutput.close(); + fileInput = new BufferedFileInput(tempFile); + dup = fileInput.duplicate(); + + fileInput.readChar(); + Assert.assertEquals(2, fileInput.position()); + Assert.assertEquals(0, dup.position()); + } finally { + if (fileInput != null) { + fileInput.close(); + } + if (fileOutput != null) { + fileOutput.close(); + } + FileUtils.deleteQuietly(tempFile); + } + } + + @Test + public void testReadWriteFullInt() throws IOException { + byte[] bytes; + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + bao.writeIntLength(Integer.MIN_VALUE); + bao.writeIntLength(Integer.MAX_VALUE); + bao.writeIntLength(0); + bytes = bao.toByteArray(); + } + + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + Assert.assertEquals(Integer.MIN_VALUE, bai.readIntLength()); + Assert.assertEquals(Integer.MAX_VALUE, bai.readIntLength()); + Assert.assertEquals(0, bai.readIntLength()); + } + } + + @Test + public void testReadWriteFullLong() throws IOException { + byte[] bytes; + try (UnsafeBytesOutput bao = new UnsafeBytesOutput()) { + bao.writeLongLength(Long.MIN_VALUE); + bao.writeLongLength(Long.MAX_VALUE); + bao.writeLongLength(0L); + bytes = bao.toByteArray(); + } + + try (UnsafeBytesInput bai = new UnsafeBytesInput(bytes)) { + Assert.assertEquals(Long.MIN_VALUE, bai.readLongLength()); + Assert.assertEquals(Long.MAX_VALUE, bai.readLongLength()); + Assert.assertEquals(0L, bai.readLongLength()); + } + } + + @Test + public void testWriteReadVInt() throws IOException { + assertEqualAfterWriteAndReadVInt(0, new byte[]{0}); + assertEqualAfterWriteAndReadVInt(1, new byte[]{1}); + assertEqualAfterWriteAndReadVInt(127, new byte[]{(byte) 0x7f}); + assertEqualAfterWriteAndReadVInt(128, new byte[]{(byte) 0x81, 0}); + assertEqualAfterWriteAndReadVInt(16383, + new byte[]{(byte) 0xff, (byte) 0x7f}); + assertEqualAfterWriteAndReadVInt(16384, + new byte[]{(byte) 0x81, (byte) 0x80, + 0} + ); + assertEqualAfterWriteAndReadVInt(16385, + new byte[]{(byte) 0x81, (byte) 0x80, + 1}); + assertEqualAfterWriteAndReadVInt(-1, new byte[]{-113, -1, -1, -1, 127}); + assertEqualAfterWriteAndReadVInt(Integer.MAX_VALUE, + new byte[]{-121, -1, -1, -1, 127} + ); + assertEqualAfterWriteAndReadVInt(Integer.MIN_VALUE, + new byte[]{-120, -128, -128, -128, 0} + ); + } + + @Test + public void testWriteReadVLong() throws IOException { + assertEqualAfterWriteAndReadVLong(0L, new byte[]{0}); + assertEqualAfterWriteAndReadVLong(1L, new byte[]{1}); + assertEqualAfterWriteAndReadVLong(127L, new byte[]{(byte) 0x7f}); + assertEqualAfterWriteAndReadVLong(128L, new byte[]{(byte) 0x81, 0}); + assertEqualAfterWriteAndReadVLong(16383L, + new byte[]{(byte) 0xff, (byte) 0x7f}); + assertEqualAfterWriteAndReadVLong(16384L, + new byte[]{(byte) 0x81, (byte) 0x80, + 0} + ); + assertEqualAfterWriteAndReadVLong(16385L, + new byte[]{(byte) 0x81, (byte) 0x80, + 1} + ); + assertEqualAfterWriteAndReadVLong(-1L, new byte[]{-127, -1, -1, -1, -1, + -1, -1, -1, -1, 127}); + assertEqualAfterWriteAndReadVLong(Integer.MAX_VALUE, + new byte[]{-121, -1, -1, -1, 127} + ); + assertEqualAfterWriteAndReadVLong(Integer.MIN_VALUE, + new byte[]{-127, -1, -1, -1, -1, + -8, -128, -128, -128, 0} + ); + assertEqualAfterWriteAndReadVLong(Long.MAX_VALUE, + new byte[]{-1, -1, -1, -1, -1, + -1, -1, -1, 127} + ); + assertEqualAfterWriteAndReadVLong(Long.MIN_VALUE, + new byte[]{-127, -128, -128, -128, + -128, -128, -128, -128, + -128, 0} + ); + } + + @Test + public void testWriteReadString() throws IOException { + assertEqualAfterWriteAndReadString("", new byte[]{0}); + assertEqualAfterWriteAndReadString("1", new byte[]{1, 49}); + assertEqualAfterWriteAndReadString("789", new byte[]{3, 55, 56, 57}); + assertEqualAfterWriteAndReadString("ABCDE", + new byte[]{5, 65, 66, 67, 68, 69}); + } + + public static void assertEqualAfterWriteAndReadVInt(int value, byte[] bytes) + throws IOException { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput(5)) { + bao.writeInt(value); + Assert.assertArrayEquals(bytes, bao.toByteArray()); + } + + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + int readValue = bai.readInt(); + Assert.assertEquals(value, readValue); + } + } + + public static void assertEqualAfterWriteAndReadVLong(long value, + byte[] bytes) + throws IOException { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput(9)) { + bao.writeLong(value); + Assert.assertArrayEquals(bytes, bao.toByteArray()); + } + + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + long readValue = bai.readLong(); + Assert.assertEquals(value, readValue); + } + } + + public static void assertEqualAfterWriteAndReadString(String value, + byte[] bytes) + throws IOException { + try (UnsafeBytesOutput bao = new OptimizedUnsafeBytesOutput()) { + bao.writeUTF(value); + Assert.assertArrayEquals(bytes, bao.toByteArray()); + } + + try (UnsafeBytesInput bai = new OptimizedUnsafeBytesInput(bytes)) { + String readValue = bai.readUTF(); + Assert.assertEquals(value, readValue); + } + } + + private static UnsafeBytesInput inputByString(String s) throws IOException { + UnsafeBytesOutput output = new UnsafeBytesOutput(); + output.writeBytes(s); + return new UnsafeBytesInput(output.toByteArray()); + } +} diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java index d839f9de6..5ad25b42c 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportClientTest.java @@ -33,7 +33,7 @@ import com.baidu.hugegraph.computer.core.UnitTestBase; import com.baidu.hugegraph.computer.core.common.ComputerContext; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.common.exception.TransportException; import com.baidu.hugegraph.computer.core.config.ComputerOptions; import com.baidu.hugegraph.computer.core.network.ConnectionId; @@ -275,8 +275,8 @@ public void testTransportPerformance() throws IOException, LOG.info("Current send unavailable"); i--; if (!BARRIER_EVENT.await(timout)) { - throw new ComputeException("Timeout(%sms) to wait sendable", - timout); + throw new ComputerException("Timeout(%sms) to wait " + + "sendable", timout); } BARRIER_EVENT.reset(); } diff --git a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServerTest.java b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServerTest.java index d06c45794..61390520f 100644 --- a/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServerTest.java +++ b/computer-test/src/main/java/com/baidu/hugegraph/computer/core/network/netty/NettyTransportServerTest.java @@ -30,7 +30,7 @@ import org.slf4j.Logger; import com.baidu.hugegraph.computer.core.UnitTestBase; -import com.baidu.hugegraph.computer.core.common.exception.ComputeException; +import com.baidu.hugegraph.computer.core.common.exception.ComputerException; import com.baidu.hugegraph.computer.core.config.ComputerOptions; import com.baidu.hugegraph.computer.core.config.Config; import com.baidu.hugegraph.computer.core.network.IOMode; @@ -193,7 +193,7 @@ public void testListenWithInvalidHost() { ComputerOptions.TRANSPORT_IO_MODE, "NIO" ); - Assert.assertThrows(ComputeException.class, () -> { + Assert.assertThrows(ComputerException.class, () -> { this.server.listen(config, messageHandler); }, e -> { Assert.assertContains("Failed to parse", e.getMessage());