From 911a221bd0640623abc2a40aac5f656b0587117b Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 12:14:41 +0300 Subject: [PATCH 1/9] added BVEdges and IntervalResidualEdges --- giraph-core/pom.xml | 4 + .../java/org/apache/giraph/edge/BVEdges.java | 409 ++++++++++++++++++ .../giraph/edge/IntervalResidualEdges.java | 224 ++++++++++ .../apache/giraph/edge/BVEdgesEdgesTest.java | 276 ++++++++++++ .../edge/IntervalResidualEdgesTest.java | 262 +++++++++++ pom.xml | 6 + 6 files changed, 1181 insertions(+) create mode 100644 giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java create mode 100644 giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java create mode 100644 giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java create mode 100644 giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java diff --git a/giraph-core/pom.xml b/giraph-core/pom.xml index 9a5d89015..9ff837be6 100644 --- a/giraph-core/pom.xml +++ b/giraph-core/pom.xml @@ -521,6 +521,10 @@ under the License. it.unimi.dsi fastutil + + + it.unimi.dsi + webgraph org.apache.zookeeper diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java new file mode 100644 index 000000000..3474a8a6c --- /dev/null +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -0,0 +1,409 @@ +package org.apache.giraph.edge; + +import it.unimi.dsi.bits.Fast; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.io.InputBitStream; +import it.unimi.dsi.io.OutputBitStream; +import it.unimi.dsi.webgraph.AbstractLazyIntIterator; +import it.unimi.dsi.webgraph.BVGraph; +import it.unimi.dsi.webgraph.IntIntervalSequenceIterator; +import it.unimi.dsi.webgraph.LazyIntIterator; +import it.unimi.dsi.webgraph.LazyIntIterators; +import it.unimi.dsi.webgraph.MergedIntIterator; + +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.stream.StreamSupport; + +import org.apache.giraph.utils.Trimmable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.weakref.jmx.com.google.common.collect.Iterators; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.UnmodifiableIterator; + +public class BVEdges extends ConfigurableOutEdges + implements ReuseObjectsOutEdges, Trimmable { + + /** Serialized Intervals and Residuals */ + private byte[] intervalsAndResiduals; + /** Number of edges. */ + private int size; + + @Override + public void initialize(Iterable> edges) { + compress(StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId) + .mapToInt(IntWritable::get).toArray()); + } + + @Override + public void initialize(int capacity) { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void initialize() { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void add(Edge edge) { + // Note that ths is very expensive (decompresses all edges and recompresses them again). + compress(StreamSupport + .stream(((Iterable>) () -> Iterators.concat(this.iterator(), + ImmutableSet.of(edge).iterator())).spliterator(), false) + .map(Edge::getTargetVertexId).mapToInt(IntWritable::get).sorted().toArray()); + } + + @Override + public void remove(IntWritable targetVertexId) { + // Note that this is very expensive (decompresses all edges and recompresses them again). + initialize(Iterables.filter(this, + edge -> !((Edge) edge).getTargetVertexId().equals(targetVertexId))); + } + + @Override + public int size() { + return size; + } + + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new BVEdgesIterator(); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int intervalResidualEdgesBytesUsed = in.readInt(); + if (intervalResidualEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (intervalsAndResiduals == null || intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { + intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; + } + in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(intervalsAndResiduals.length); + if (intervalsAndResiduals.length > 0) { + out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); + } + } + + @Override + public void trim() { + // Nothing to do + } + + + /** + * Receives an integer array of successors and compresses them in the + * intervalsAndResiduals byte array + * + * @param edgesArray an integer array of successors + */ + private void compress(final int[] edgesArray) { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + diffComp(edgesArray, new OutputBitStream(baos)); + intervalsAndResiduals = baos.toByteArray(); + size = edgesArray.length; + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + /** + * See also {@link it.unimi.dsi.webgraph.BVGraph#diffComp(int[], OutputBitStream)}. + * This method is given an integer successors array and produces, onto a given output + * bit stream, the differentially compressed successor list. + * + * @param edgesArray the integer successors array + * @param obs an output bit stream where the compressed data will be + * stored. + */ + public static void diffComp(final int[] edgesArray, OutputBitStream obs) throws IOException { + // We write the degree. + obs.writeInt(edgesArray.length, Integer.SIZE); + final int residual[], residualCount; + IntArrayList left = new IntArrayList(); + IntArrayList len = new IntArrayList(); + IntArrayList residuals = new IntArrayList(); + // If we are to produce intervals, we first compute them. + int intervalCount; + try { + intervalCount = BVEdges.intervalize(edgesArray, BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, + left, len, residuals); + } catch (IllegalArgumentException e) { + // array was not sorted, sorting and retrying + Arrays.sort(edgesArray); + left = new IntArrayList(); + len = new IntArrayList(); + residuals = new IntArrayList(); + intervalCount = BVEdges.intervalize(edgesArray, BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, + left, len, residuals); + } + // We write the number of intervals. + obs.writeGamma(intervalCount); + int currIntLen; + int prev = 0; + if (intervalCount > 0) { + obs.writeInt(left.getInt(0), Integer.SIZE); + currIntLen = len.getInt(0); + prev = left.getInt(0) + currIntLen; + obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); + } + // We write out the intervals. + for (int i = 1; i < intervalCount; i++) { + obs.writeGamma(left.getInt(i) - prev - 1); + currIntLen = len.getInt(i); + prev = left.getInt(i) + currIntLen; + obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); + } + residual = residuals.elements(); + residualCount = residuals.size(); + // Now we write out the residuals, if any + if (residualCount != 0) { + if (intervalCount > 0) { + obs.writeLongZeta(Fast.int2nat((long) (prev = residual[0]) - left.getInt(0)), BVGraph.DEFAULT_ZETA_K); + } else { + obs.writeInt((prev = residual[0]), Integer.SIZE); + } + for (int i = 1; i < residualCount; i++) { + if (residual[i] == prev) { + throw new IllegalArgumentException( + "Repeated successor " + prev + " in successor list of this node"); + } + obs.writeLongZeta(residual[i] - prev - 1, BVGraph.DEFAULT_ZETA_K); + prev = residual[i]; + } + } + obs.flush(); + } + + /** + * See also + * {@link it.unimi.dsi.webgraph.BVGraph#intervalize(IntArrayList, int, IntArrayList, IntArrayList, IntArrayList)}. + * This method tries to express an increasing sequence of natural numbers + * x as a union of an increasing sequence of intervals and an + * increasing sequence of residual elements. More precisely, this + * intervalization works as follows: first, one looks at edgesArray + * as a sequence of intervals (i.e., maximal sequences of consecutive elements); + * those intervals whose length is ≥ minInterval are stored in + * the lists left (the list of left extremes) and len + * (the list of lengths; the length of an integer interval is the number of + * integers in that interval). The remaining integers, called residuals + * are stored in the residual list. + * + *

+ * Note that the previous content of left, len and + * residual is lost. + * + * @param edgesArray the array to be intervalized (an increasing list of + * natural numbers). + * @param minInterval the least length that a maximal sequence of consecutive + * elements must have in order for it to be considered as an + * interval. + * @param left the resulting list of left extremes of the intervals. + * @param len the resulting list of interval lengths. + * @param residuals the resulting list of residuals. + * @return the number of intervals. + */ + protected static int intervalize(final int[] edgesArray, final int minInterval, + final int maxInterval, final IntArrayList left, final IntArrayList len, final IntArrayList residuals) { + int nInterval = 0; + int i, j; + + for (i = 0; i < edgesArray.length; i++) { + j = 0; + checkIsSorted(edgesArray, i); + if (i < edgesArray.length - 1 && edgesArray[i] + 1 == edgesArray[i + 1]) { + do + j++; + while (i + j < edgesArray.length - 1 && j < maxInterval + && edgesArray[i + j] + 1 == edgesArray[i + j + 1]); + checkIsSorted(edgesArray, i + j); + j++; + // Now j is the number of integers in the interval. + if (j >= minInterval) { + left.add(edgesArray[i]); + len.add(j); + nInterval++; + i += j - 1; + } + } + if (j < minInterval) + residuals.add(edgesArray[i]); + } + return nInterval; + } + + /** + * Given an integer array and an index, this method throws an exception in case + * the element of the array the index points at is equal or larger than its + * next. + * + * @param edgesArray the integer array + * @param i the index + */ + private static void checkIsSorted(int[] edgesArray, int i) { + if (i < edgesArray.length - 1 && edgesArray[i] == edgesArray[i + 1]) { + throw new IllegalArgumentException("Parallel edges are not allowed."); + } + if (i < edgesArray.length - 1 && edgesArray[i] > edgesArray[i + 1]) { + throw new IllegalArgumentException("Edges are not sorted."); + } + } + + /** + * Iterator that reuses the same Edge object. + */ + private class BVEdgesIterator extends UnmodifiableIterator> { + /** Wrapped map iterator. */ + LazyIntIterator liIter = successors(new InputBitStream(intervalsAndResiduals)); + /** Representative edge object. */ + private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + + @Override + public boolean hasNext() { + return currentEdge < size; + } + + @Override + public Edge next() { + representativeEdge.getTargetVertexId().set(liIter.nextInt()); + currentEdge++; + return representativeEdge; + } + + private LazyIntIterator successors(InputBitStream ibs) { + try { + final int d; + int extraCount; + int firstIntervalNode = -1; + ibs.position(0); + d = ibs.readInt(Integer.SIZE); + if (d == 0) + return LazyIntIterators.EMPTY_ITERATOR; + extraCount = d; + int intervalCount = 0; // Number of intervals + int[] left = null; + int[] len = null; + if (extraCount > 0) { + // Prepare to read intervals, if any + if ((intervalCount = ibs.readGamma()) != 0) { + int prev = 0; // Holds the last integer in the last + // interval. + left = new int[intervalCount]; + len = new int[intervalCount]; + // Now we read intervals + left[0] = firstIntervalNode = prev = ibs.readInt(Integer.SIZE); + len[0] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + + prev += len[0]; + extraCount -= len[0]; + for (int i = 1; i < intervalCount; i++) { + left[i] = prev = ibs.readGamma() + prev + 1; + len[i] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + prev += len[i]; + extraCount -= len[i]; + } + } + } + + final int residualCount = extraCount; // Just to be able to use + // an + // anonymous class. + final LazyIntIterator residualIterator = residualCount == 0 ? null + : new ResidualIntIterator(ibs, residualCount, firstIntervalNode); + // The extra part is made by the contribution of intervals, if + // any, and by the residuals iterator. + final LazyIntIterator extraIterator = intervalCount == 0 ? residualIterator + : (residualCount == 0 ? (LazyIntIterator) new IntIntervalSequenceIterator(left, len) + : (LazyIntIterator) new MergedIntIterator(new IntIntervalSequenceIterator(left, len), + residualIterator)); + return extraIterator; + } catch (IOException e) { + e.printStackTrace(); + return null; + } + } + + /** An iterator returning the residuals of a node. */ + private final class ResidualIntIterator extends AbstractLazyIntIterator { + /** The input bit stream from which residuals will be read. */ + private final InputBitStream ibs; + /** The last residual returned. */ + private int next; + /** The number of remaining residuals. */ + private int remaining; + + private ResidualIntIterator(final InputBitStream ibs, final int residualCount, int x) { + this.remaining = residualCount; + this.ibs = ibs; + try { + if (x >= 0) { + long temp = Fast.nat2int(ibs.readLongZeta(BVGraph.DEFAULT_ZETA_K)); + this.next = (int) (x + temp); + } else { + this.next = ibs.readInt(Integer.SIZE); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public int nextInt() { + if (remaining == 0) + return -1; + try { + final int result = next; + if (--remaining != 0) + next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; + return result; + } catch (IOException cantHappen) { + throw new RuntimeException(cantHappen); + } + } + + @Override + public int skip(int n) { + if (n >= remaining) { + n = remaining; + remaining = 0; + return n; + } + try { + for (int i = n; i-- != 0;) + next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; + remaining -= n; + return n; + } catch (IOException cantHappen) { + throw new RuntimeException(cantHappen); + } + } + + } + + } + +} diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java new file mode 100644 index 000000000..316f59924 --- /dev/null +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -0,0 +1,224 @@ +package org.apache.giraph.edge; + +import it.unimi.dsi.fastutil.ints.IntArrayList; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.stream.StreamSupport; + +import org.apache.giraph.utils.ExtendedDataInput; +import org.apache.giraph.utils.ExtendedDataOutput; +import org.apache.giraph.utils.Trimmable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.weakref.jmx.com.google.common.collect.Iterators; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.UnmodifiableIterator; + +/** + * Implementation of {@link OutEdges} with int ids and null edge values, backed + * by the IntervalResidualEdges structure proposed in "Panagiotis Liakos, Katia + * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed + * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: + * this implementation is optimized for space usage, but edge addition and + * removals are expensive. Parallel edges are not allowed. + */ +public class IntervalResidualEdges extends ConfigurableOutEdges + implements ReuseObjectsOutEdges, Trimmable { + + /** Minimum interval length is equal to 2 */ + private static final int MIN_INTERVAL_LENGTH = 2; + + /** Maximum interval length is equal to 254 */ + private static final int MAX_INTERVAL_LENGTH = 254; + + /** Serialized Intervals and Residuals */ + private byte[] intervalsAndResiduals; + + /** Number of edges stored in compressed array */ + private int size; + + @Override + public void initialize(Iterable> edges) { + compress(StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId).mapToInt(IntWritable::get).toArray()); + } + + @Override + public void initialize(int capacity) { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void initialize() { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void add(Edge edge) { + // Note that this is very expensive (decompresses all edges and recompresses them again). + compress(StreamSupport.stream(((Iterable>)() -> Iterators.concat(this.iterator(), ImmutableSet.of(edge).iterator())).spliterator(), false).map(Edge::getTargetVertexId).mapToInt(IntWritable::get).sorted().toArray()); + } + + @Override + public void remove(IntWritable targetVertexId) { + // Note that this is very expensive (decompresses all edges and recompresses them again). + initialize(Iterables.filter(this, edge -> !((Edge) edge).getTargetVertexId().equals(targetVertexId))); + } + + @Override + public int size() { + return size; + } + + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new IntervalResidualEdgeIterator(); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int intervalResidualEdgesBytesUsed = in.readInt(); + if (intervalResidualEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (intervalsAndResiduals == null || intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { + intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; + } + in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(intervalsAndResiduals.length); + if (intervalsAndResiduals.length > 0) { + out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); + } + } + + @Override + public void trim() { + /* Nothing to do */ + } + + /** Receives an integer array of successors and compresses them + * in the intervalsAndResiduals byte array + * + * @param edgesArray an integer array of successors + * */ + private void compress(final int[] edgesArray) { + try { + ExtendedDataOutput eos = getConf().createExtendedDataOutput(); + IntArrayList left = new IntArrayList(); + IntArrayList len = new IntArrayList(); + IntArrayList residuals = new IntArrayList(); + // If we are to produce intervals, we first compute them. + int intervalCount; + try { + intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, MAX_INTERVAL_LENGTH, left, len, + residuals); + } catch (IllegalArgumentException e) { + // array was not sorted, sorting and retrying + Arrays.sort(edgesArray); + left = new IntArrayList(); + len = new IntArrayList(); + residuals = new IntArrayList(); + intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, MAX_INTERVAL_LENGTH, left, len, + residuals); + } + + // We write out the intervals. + eos.writeInt(intervalCount); + for (int i = 0; i < intervalCount; i++) { + eos.writeInt(left.getInt(i)); + eos.write(len.getInt(i)); + } + final int[] residual = residuals.elements(); + final int residualCount = residuals.size(); + for (int i = 0; i < residualCount; i++) { + eos.writeInt(residual[i]); + } + intervalsAndResiduals = eos.toByteArray(); + size = edgesArray.length; + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + /** + * Iterator that reuses the same Edge object. + */ + private class IntervalResidualEdgeIterator extends UnmodifiableIterator> { + private ExtendedDataInput extendedDataInput = getConf().createExtendedDataInput(intervalsAndResiduals, 0, + intervalsAndResiduals.length); + /** Representative edge object. */ + private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + private int currentLeft; + private int currentLen = 0; + private int intervalCount; + + public IntervalResidualEdgeIterator() { + try { + intervalCount = extendedDataInput.readInt(); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean hasNext() { + return currentEdge < size; + } + + @Override + public Edge next() { + this.currentEdge++; + switch (this.currentLen) { + case 0: + switch (this.intervalCount) { + case 0: + try { + representativeEdge.getTargetVertexId().set(extendedDataInput.readInt()); + } catch (IOException canthappen) { + canthappen.printStackTrace(); + } + return representativeEdge; + default: + try { + this.currentLeft = extendedDataInput.readInt(); + this.currentLen = extendedDataInput.readByte() & 0xff; + intervalCount--; + } catch (IOException canthappen) { + canthappen.printStackTrace(); + } + final int result = this.currentLeft; + this.currentLen--; + representativeEdge.getTargetVertexId().set(result); + return representativeEdge; + + } + default: + final int result = ++this.currentLeft; + this.currentLen--; + representativeEdge.getTargetVertexId().set(result); + return representativeEdge; + } + + } + + } +} diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java new file mode 100644 index 000000000..ec9a6b2dc --- /dev/null +++ b/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java @@ -0,0 +1,276 @@ +/* + * 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 org.apache.giraph.edge; + +import com.google.common.collect.Lists; +import org.apache.giraph.conf.GiraphConfiguration; +import org.apache.giraph.conf.GiraphConstants; +import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * Implementation of {@link OutEdges} with int ids and null edge values, backed + * by the BVEdges structure proposed in "Panagiotis Liakos, Katia + * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed + * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: + * this implementation is optimized for space usage, but edge addition and + * removals are expensive. Parallel edges are not allowed. + */ +public class BVEdgesEdgesTest { + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(BVEdges edges, int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private BVEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + BVEdges ret = new BVEdges(); + ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); + return ret; + } + + @Test + public void testEdges() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + + edges.remove(new IntWritable(2)); + assertEdges(edges, 1, 3, 4); + } + + @Test + public void testInitialize() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testInitializeUnsorted() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), + createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 3, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + BVEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + BVEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation + + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); + + edges.write(tempBuffer); + tempBuffer.close(); + + byte[] binary = arrayStream.toByteArray(); + + assertTrue("Serialized version should not be empty ", binary.length > 0); + + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); + + assertEquals(5, edges.size()); + + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; + + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } + + @Test(expected = IllegalArgumentException.class) + public void testParallelEdges() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), + createEdge(2)); + + edges.initialize(initialEdges); + } + + @Test + public void testEdgeValues() { + BVEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); + + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } + + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + BVEdges edges = getEdges(); + + // Add 10 edges with id i, for i = 0..9 + // to create one large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Add a residual edge + edges.add(createEdge(23)); + + // Add 10 edges with id i, for i = 50..59 + // to create a second large interval + for (int i = 50; i < 60; ++i) { + edges.add(createEdge(i)); + } + + assertEquals(21, edges.size()); + + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of intervalResiduals byte array should be equal to 12 + ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor.forClass(byte[].class); + Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); + assertEquals(12, intervalsAndResidualsCaptop.getValue().length); + } + +} diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java new file mode 100644 index 000000000..54eeb4440 --- /dev/null +++ b/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java @@ -0,0 +1,262 @@ +/* + * 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 org.apache.giraph.edge; + +import com.google.common.collect.Lists; +import org.apache.giraph.conf.GiraphConfiguration; +import org.apache.giraph.conf.GiraphConstants; +import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class IntervalResidualEdgesTest { + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(IntervalResidualEdges edges, int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private IntervalResidualEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + IntervalResidualEdges ret = new IntervalResidualEdges(); + ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); + return ret; + } + + @Test + public void testEdges() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + + edges.remove(new IntWritable(2)); + assertEdges(edges, 3, 4, 1); + } + + @Test + public void testInitialize() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testInitializeUnsorted() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), + createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 3, 4, 1); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + IntervalResidualEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + IntervalResidualEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation + + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); + + edges.write(tempBuffer); + tempBuffer.close(); + + byte[] binary = arrayStream.toByteArray(); + + assertTrue("Serialized version should not be empty ", binary.length > 0); + + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); + + assertEquals(5, edges.size()); + + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; + + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } + + @Test(expected = IllegalArgumentException.class) + public void testParallelEdges() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), + createEdge(2)); + + edges.initialize(initialEdges); + } + + @Test + public void testEdgeValues() { + IntervalResidualEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); + + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } + + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + IntervalResidualEdges edges = getEdges(); + + // Add 10 edges with id i, for i = 0..9 + // to create on large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Add a residual edge + edges.add(createEdge(23)); + + assertEquals(11, edges.size()); + + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of intervalResiduals byte array should be equal to 13: + // 4 (intervals' size) + 4 (index of 1st interval) + 1 (length of 1st interval) + 4 (residual) + ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor.forClass(byte[].class); + Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); + assertEquals(13, intervalsAndResidualsCaptop.getValue().length); + } +} diff --git a/pom.xml b/pom.xml index 1522fd4d0..6369634ba 100644 --- a/pom.xml +++ b/pom.xml @@ -358,6 +358,7 @@ under the License. 1.7.6 2.4.0 0.2.1 + 3.6.1 2.2.0 11.0.10 3.4.5 @@ -1786,6 +1787,11 @@ under the License. fastutil ${dep.fastutil.version} + + it.unimi.dsi + webgraph + ${dep.webgraph.version} + org.apache.giraph giraph-core From 51d48895ce64a4b6bde1f8683c35b6931287831a Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 12:47:58 +0300 Subject: [PATCH 2/9] code improvements --- .../java/org/apache/giraph/edge/BVEdges.java | 78 +++++++++---------- .../giraph/edge/IntervalResidualEdges.java | 7 +- 2 files changed, 41 insertions(+), 44 deletions(-) diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java index 3474a8a6c..110a96d5d 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -22,7 +22,6 @@ import org.apache.giraph.utils.Trimmable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; import org.weakref.jmx.com.google.common.collect.Iterators; import com.google.common.collect.ImmutableSet; @@ -67,8 +66,7 @@ public void add(Edge edge) { @Override public void remove(IntWritable targetVertexId) { // Note that this is very expensive (decompresses all edges and recompresses them again). - initialize(Iterables.filter(this, - edge -> !((Edge) edge).getTargetVertexId().equals(targetVertexId))); + initialize(Iterables.filter(this, edge -> !edge.getTargetVertexId().equals(targetVertexId))); } @Override @@ -139,10 +137,9 @@ private void compress(final int[] edgesArray) { * @param obs an output bit stream where the compressed data will be * stored. */ - public static void diffComp(final int[] edgesArray, OutputBitStream obs) throws IOException { + public static void diffComp(final int[] edgesArray, OutputBitStream obs) throws IOException { // We write the degree. obs.writeInt(edgesArray.length, Integer.SIZE); - final int residual[], residualCount; IntArrayList left = new IntArrayList(); IntArrayList len = new IntArrayList(); IntArrayList residuals = new IntArrayList(); @@ -177,21 +174,22 @@ public static void diffComp(final int[] edgesArray, OutputB prev = left.getInt(i) + currIntLen; obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); } - residual = residuals.elements(); - residualCount = residuals.size(); + final int[] residual = residuals.elements(); + final int residualCount = residuals.size(); // Now we write out the residuals, if any if (residualCount != 0) { if (intervalCount > 0) { obs.writeLongZeta(Fast.int2nat((long) (prev = residual[0]) - left.getInt(0)), BVGraph.DEFAULT_ZETA_K); } else { - obs.writeInt((prev = residual[0]), Integer.SIZE); + prev = residual[0]; + obs.writeInt(prev, Integer.SIZE); } for (int i = 1; i < residualCount; i++) { if (residual[i] == prev) { throw new IllegalArgumentException( "Repeated successor " + prev + " in successor list of this node"); } - obs.writeLongZeta(residual[i] - prev - 1, BVGraph.DEFAULT_ZETA_K); + obs.writeLongZeta(residual[i] - prev - 1L, BVGraph.DEFAULT_ZETA_K); prev = residual[i]; } } @@ -226,10 +224,11 @@ public static void diffComp(final int[] edgesArray, OutputB * @param residuals the resulting list of residuals. * @return the number of intervals. */ - protected static int intervalize(final int[] edgesArray, final int minInterval, + protected static int intervalize(final int[] edgesArray, final int minInterval, final int maxInterval, final IntArrayList left, final IntArrayList len, final IntArrayList residuals) { int nInterval = 0; - int i, j; + int i; + int j; for (i = 0; i < edgesArray.length; i++) { j = 0; @@ -308,25 +307,23 @@ private LazyIntIterator successors(InputBitStream ibs) { int intervalCount = 0; // Number of intervals int[] left = null; int[] len = null; - if (extraCount > 0) { - // Prepare to read intervals, if any - if ((intervalCount = ibs.readGamma()) != 0) { - int prev = 0; // Holds the last integer in the last - // interval. - left = new int[intervalCount]; - len = new int[intervalCount]; - // Now we read intervals - left[0] = firstIntervalNode = prev = ibs.readInt(Integer.SIZE); - len[0] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; - - prev += len[0]; - extraCount -= len[0]; - for (int i = 1; i < intervalCount; i++) { - left[i] = prev = ibs.readGamma() + prev + 1; - len[i] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; - prev += len[i]; - extraCount -= len[i]; - } + // Prepare to read intervals, if any + if (extraCount > 0 && (intervalCount = ibs.readGamma()) != 0) { + int prev = 0; // Holds the last integer in the last + // interval. + left = new int[intervalCount]; + len = new int[intervalCount]; + // Now we read intervals + left[0] = firstIntervalNode = prev = ibs.readInt(Integer.SIZE); + len[0] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + + prev += len[0]; + extraCount -= len[0]; + for (int i = 1; i < intervalCount; i++) { + left[i] = prev = ibs.readGamma() + prev + 1; + len[i] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + prev += len[i]; + extraCount -= len[i]; } } @@ -337,14 +334,15 @@ private LazyIntIterator successors(InputBitStream ibs) { : new ResidualIntIterator(ibs, residualCount, firstIntervalNode); // The extra part is made by the contribution of intervals, if // any, and by the residuals iterator. - final LazyIntIterator extraIterator = intervalCount == 0 ? residualIterator - : (residualCount == 0 ? (LazyIntIterator) new IntIntervalSequenceIterator(left, len) - : (LazyIntIterator) new MergedIntIterator(new IntIntervalSequenceIterator(left, len), - residualIterator)); - return extraIterator; + if (intervalCount == 0) { + return residualIterator; + } else if (residualCount == 0){ + return new IntIntervalSequenceIterator(left, len); + } else { + return new MergedIntIterator(new IntIntervalSequenceIterator(left, len), residualIterator); + } } catch (IOException e) { - e.printStackTrace(); - return null; + throw new IllegalStateException(e); } } @@ -368,7 +366,7 @@ private ResidualIntIterator(final InputBitStream ibs, final int residualCount, i this.next = ibs.readInt(Integer.SIZE); } } catch (IOException e) { - throw new RuntimeException(e); + throw new IllegalStateException(e); } } @@ -381,7 +379,7 @@ public int nextInt() { next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; return result; } catch (IOException cantHappen) { - throw new RuntimeException(cantHappen); + throw new IllegalStateException(cantHappen); } } @@ -398,7 +396,7 @@ public int skip(int n) { remaining -= n; return n; } catch (IOException cantHappen) { - throw new RuntimeException(cantHappen); + throw new IllegalStateException(cantHappen); } } diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java index 316f59924..09214535b 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -69,7 +69,7 @@ public void add(Edge edge) { @Override public void remove(IntWritable targetVertexId) { // Note that this is very expensive (decompresses all edges and recompresses them again). - initialize(Iterables.filter(this, edge -> !((Edge) edge).getTargetVertexId().equals(targetVertexId))); + initialize(Iterables.filter(this, edge -> !edge.getTargetVertexId().equals(targetVertexId))); } @Override @@ -194,7 +194,7 @@ public Edge next() { try { representativeEdge.getTargetVertexId().set(extendedDataInput.readInt()); } catch (IOException canthappen) { - canthappen.printStackTrace(); + throw new IllegalStateException(canthappen); } return representativeEdge; default: @@ -203,7 +203,7 @@ public Edge next() { this.currentLen = extendedDataInput.readByte() & 0xff; intervalCount--; } catch (IOException canthappen) { - canthappen.printStackTrace(); + throw new IllegalStateException(canthappen); } final int result = this.currentLeft; this.currentLen--; @@ -217,7 +217,6 @@ public Edge next() { representativeEdge.getTargetVertexId().set(result); return representativeEdge; } - } } From e8b96b7d298253665d3772ba7882ca6ac19dadfe Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 12:50:49 +0300 Subject: [PATCH 3/9] added apache licence text --- .../java/org/apache/giraph/edge/BVEdges.java | 18 ++++++++++++++++++ .../giraph/edge/IntervalResidualEdges.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java index 110a96d5d..d6a5d06c2 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -1,3 +1,21 @@ +/* + * 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 org.apache.giraph.edge; import it.unimi.dsi.bits.Fast; diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java index 09214535b..d4be13e30 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -1,3 +1,21 @@ +/* + * 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 org.apache.giraph.edge; import it.unimi.dsi.fastutil.ints.IntArrayList; From b3429a566b4776007da8a92c2fedd0342f953898 Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 16:29:10 +0300 Subject: [PATCH 4/9] added IndexedBitArray --- .../java/org/apache/giraph/edge/BVEdges.java | 9 + .../giraph/edge/IndexedBitArrayEdges.java | 311 ++++++++++++++++++ .../giraph/edge/IntervalResidualEdges.java | 3 +- .../giraph/edge/IndexedBitArrayEdgesTest.java | 267 +++++++++++++++ 4 files changed, 589 insertions(+), 1 deletion(-) create mode 100644 giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java create mode 100644 giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java index d6a5d06c2..947ea6f1f 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -46,6 +46,15 @@ import com.google.common.collect.Iterables; import com.google.common.collect.UnmodifiableIterator; +/** + * Implementation of {@link OutEdges} with int ids and null edge values, backed + * by the BVEdges structure proposed in "Panagiotis Liakos, Katia + * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed + * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: + * this implementation is optimized for space usage for graphs exhibiting the + * locality of reference property, but edge addition and + * removals are expensive. Parallel edges are not allowed. + */ public class BVEdges extends ConfigurableOutEdges implements ReuseObjectsOutEdges, Trimmable { diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java new file mode 100644 index 000000000..60b92c77a --- /dev/null +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java @@ -0,0 +1,311 @@ +/* + * 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 org.apache.giraph.edge; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.UnmodifiableIterator; + +import org.apache.giraph.utils.Trimmable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.python.google.common.primitives.Bytes; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.stream.StreamSupport; + +/** + * Implementation of {@link OutEdges} with int ids and null edge values, backed + * by the IndexedBitArrayEdges structure proposed in "Panagiotis Liakos, Katia + * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed + * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: + * this implementation is optimized for space usage for graphs exhibiting the + * locality of reference property, but edge addition and + * removals are expensive. Parallel edges are not allowed. + */ +public class IndexedBitArrayEdges extends ConfigurableOutEdges + implements ReuseObjectsOutEdges, Trimmable { + + /** Serialized edges. */ + private byte[] serializedEdges; + /** Number of edges. */ + private int size; + + @Override + public void initialize(Iterable> edges) { + initialize(); + int[] sortedEdgesArray = StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId) + .mapToInt(IntWritable::get).sorted().distinct().toArray(); + if (sortedEdgesArray.length == 0) { + return; + } + int currBucket = sortedEdgesArray[0] / Byte.SIZE; + byte myByte = (byte) 0; + for (int edge : sortedEdgesArray) { + int bucket = edge / Byte.SIZE; + int pos = edge % Byte.SIZE; + if (bucket == currBucket) { + myByte = setBit(myByte, pos); + size += 1; + } else { + serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), myByte); + currBucket = bucket; + myByte = setBit((byte) 0, pos); + size += 1; + } + } + serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), myByte); + } + + @Override + public void initialize(int capacity) { + size = 0; + serializedEdges = new byte[0]; + } + + @Override + public void initialize() { + size = 0; + serializedEdges = new byte[0]; + } + + @Override + public void add(Edge edge) { + int bucket = edge.getTargetVertexId().get() / Byte.SIZE; + int pos = edge.getTargetVertexId().get() % Byte.SIZE; + int i = 0; + boolean done = false; + while (i < serializedEdges.length) { // if bucket is already there, simply set the appropriate bit + if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, i + Integer.SIZE / Byte.SIZE)) == bucket) { + int index = i + Integer.SIZE / Byte.SIZE; + if (!isSet(serializedEdges[index], pos)) { + size += 1; + serializedEdges[index] = setBit(serializedEdges[index], pos); + } + done = true; + break; + } + i += Integer.SIZE / Byte.SIZE + 1; + } + if (!done) { // we need to add a bucket + serializedEdges = addBytes(serializedEdges, toByteArray(bucket), setBit((byte) 0, pos)); + size += 1; + } + } + + @Override + public void remove(IntWritable targetVertexId) { + int bucket = targetVertexId.get() / Byte.SIZE; + int pos = targetVertexId.get() % Byte.SIZE; + int i = 0; + while (i < serializedEdges.length) { + if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, i + Integer.SIZE / Byte.SIZE)) == bucket) { + serializedEdges[i + Integer.SIZE / Byte.SIZE] = unsetBit(serializedEdges[i + Integer.SIZE / Byte.SIZE], + pos); + --size; + break; + } + i += Integer.SIZE / Byte.SIZE + 1; + } + } + + @Override + public int size() { + return size; + } + + @Override + public void trim() { + // Nothing to do + } + + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new IndexedBitmapEdgeIterator(); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int serializedEdgesBytesUsed = in.readInt(); + if (serializedEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (serializedEdges == null || serializedEdgesBytesUsed > serializedEdges.length) { + serializedEdges = new byte[serializedEdgesBytesUsed]; + } + in.readFully(serializedEdges, 0, serializedEdgesBytesUsed); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(serializedEdges.length); + if (serializedEdges.length > 0) { + out.write(serializedEdges, 0, serializedEdges.length); + } + } + + /** Iterator that reuses the same Edge object. */ + private class IndexedBitmapEdgeIterator extends UnmodifiableIterator> { + /** Representative edge object. */ + private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + /** Current position */ + private int currentBitPosition = 0; + private int currentBytePosition = 0; + /** Index int */ + private int indexInt; + /** Current byte */ + private Byte myByte; + /** Current index */ + private byte[] myBytes = new byte[Integer.SIZE / Byte.SIZE]; + + @Override + public boolean hasNext() { + return currentEdge < size; + } + + @Override + public Edge next() { + if (currentBitPosition == 8) { + currentBitPosition = 0; + } + if (currentBitPosition == 0) { + myBytes[0] = serializedEdges[currentBytePosition]; + myBytes[1] = serializedEdges[currentBytePosition + 1]; + myBytes[2] = serializedEdges[currentBytePosition + 2]; + myBytes[3] = serializedEdges[currentBytePosition + 3]; + indexInt = fromByteArray(myBytes); + myByte = serializedEdges[currentBytePosition + Integer.SIZE / Byte.SIZE]; + currentBytePosition += Integer.SIZE / Byte.SIZE + 1; + } + int pos = currentBitPosition; + int nextPos = 0; + boolean done = false; + while (!done) { + for (int i = pos; i < Byte.SIZE; i++) { + if (isSet(myByte, i)) { + done = true; + nextPos = i; + currentEdge++; + currentBitPosition = i + 1; + break; + } + } + if (!done /* && mapIterator.hasNext() */) { + myBytes[0] = serializedEdges[currentBytePosition]; + myBytes[1] = serializedEdges[currentBytePosition + 1]; + myBytes[2] = serializedEdges[currentBytePosition + 2]; + myBytes[3] = serializedEdges[currentBytePosition + 3]; + indexInt = fromByteArray(myBytes); + myByte = serializedEdges[currentBytePosition + Integer.SIZE / Byte.SIZE]; + currentBytePosition += Integer.SIZE / Byte.SIZE + 1; + pos = 0; + } + } + representativeEdge.getTargetVertexId().set(indexInt * Byte.SIZE + nextPos); + return representativeEdge; + } + + } + + /** + * tests if bit is set in a byte + * + * @param myByte the byte to be tested + * @param pos the position in the byte to be tested + * @returns true or false depending on the bit being set + * + */ + public static boolean isSet(byte myByte, int pos) { + return (myByte & (1 << pos)) != 0; + } + + /** + * tests if bit is set in a byte + * + * @param byteWritable the byte to be updated + * @param pos the position in the byte to be set + * @returns the updated byte + * + */ + public static byte setBit(byte myByte, int pos) { + return (byte) (myByte | (1 << pos)); + } + + /** + * tests if bit is set in a byte + * + * @param my_byte the byte to be updated + * @param pos the position in the byte to be unset + * @returns the updated byte + * + */ + public static byte unsetBit(byte myByte, int pos) { + return (byte) (myByte & ~(1 << pos)); + } + + /** + * converts an integer to a 4-byte length array holding + * its binary representation + * + * @param value the integer to be converted + * @return the 4-byte length byte array holding the integer's + * binary representation + */ + public static byte[] toByteArray(int value) { + return new byte[] { (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }; + } + + /** + * converts a 4-byte length byte array to its respective integer + * + * @param bytes a 4-byte length byte array + * @return the integer represented in the byte array + */ + public static int fromByteArray(byte[] bytes) { + return bytes[0] << 24 | (bytes[1] & 0xFF) << 16 | (bytes[2] & 0xFF) << 8 | (bytes[3] & 0xFF); + } + + /** + * returns a byte arrray with the concatenation of the three input parameters + * + * @param original a byte array + * @param bindex a byte array + * @param myByte a byte + * @return a byte arrray with the concatenation of the three input parameters + * */ + public static byte[] addBytes(byte[] original, byte[] bindex, byte myByte) { + byte[] destination = new byte[original.length + Integer.SIZE / Byte.SIZE + 1]; + System.arraycopy(original, 0, destination, 0, original.length); + System.arraycopy(bindex, 0, destination, original.length, Integer.SIZE / Byte.SIZE); + destination[destination.length - 1] = myByte; + return destination; + } + +} diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java index d4be13e30..f6c47cf4c 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -43,7 +43,8 @@ * by the IntervalResidualEdges structure proposed in "Panagiotis Liakos, Katia * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: - * this implementation is optimized for space usage, but edge addition and + * this implementation is optimized for space usage for graphs exhibiting the + * locality of reference property, but edge addition and * removals are expensive. Parallel edges are not allowed. */ public class IntervalResidualEdges extends ConfigurableOutEdges diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java new file mode 100644 index 000000000..dcdfd47c6 --- /dev/null +++ b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java @@ -0,0 +1,267 @@ +/* + * 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 org.apache.giraph.edge; + +import com.google.common.collect.Lists; +import org.apache.giraph.conf.GiraphConfiguration; +import org.apache.giraph.conf.GiraphConstants; +import org.apache.giraph.conf.ImmutableClassesGiraphConfiguration; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class IndexedBitArrayEdgesTest { + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(IndexedBitArrayEdges edges, int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private IndexedBitArrayEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + IndexedBitArrayEdges ret = new IndexedBitArrayEdges(); + ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); + return ret; + } + + @Test + public void testEdges() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + + edges.remove(new IntWritable(2)); + assertEdges(edges, 1, 3, 4); + } + + @Test + public void testInitialize() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), + createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testInitializeUnsorted() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), + createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 3, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation + + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); + + edges.write(tempBuffer); + tempBuffer.close(); + + byte[] binary = arrayStream.toByteArray(); + + assertTrue("Serialized version should not be empty ", binary.length > 0); + + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); + + assertEquals(5, edges.size()); + + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; + + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } + + @Test + public void testParallelEdges() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), + createEdge(2)); + + edges.initialize(initialEdges); + + assertEquals(1, edges.size()); + } + + @Test + public void testEdgeValues() { + IndexedBitArrayEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); + + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } + + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists.newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + // to create on large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Add a residual edge + edges.add(createEdge(23)); + + assertEquals(11, edges.size()); + + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of serializedEdges byte array should be equal to 15: + // 5 (bucket 0: 0-7) + 5 (bucket 1: 8-10) + 5 (bucket 3: 23) + ArgumentCaptor serializedEdgesCaptop = ArgumentCaptor.forClass(byte[].class); + Mockito.verify(dataOutput).write(serializedEdgesCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); + assertEquals(15, serializedEdgesCaptop.getValue().length); + } +} From 2955dd1f2da635061fedda6395e5fd0da4624f5a Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 16:33:40 +0300 Subject: [PATCH 5/9] removed unused import --- .../main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java index 60b92c77a..e6fdbd09c 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java @@ -24,7 +24,6 @@ import org.apache.giraph.utils.Trimmable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; -import org.python.google.common.primitives.Bytes; import java.io.DataInput; import java.io.DataOutput; @@ -40,7 +39,7 @@ * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: * this implementation is optimized for space usage for graphs exhibiting the * locality of reference property, but edge addition and - * removals are expensive. Parallel edges are not allowed. + * removals are expensive. Parallel edges are ignored. */ public class IndexedBitArrayEdges extends ConfigurableOutEdges implements ReuseObjectsOutEdges, Trimmable { From 8f974352de1f7c01ea026d440785ac78b7b1dc18 Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 17:47:04 +0300 Subject: [PATCH 6/9] fixed a comment --- .../java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java index dcdfd47c6..925db118a 100644 --- a/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java +++ b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java @@ -259,7 +259,7 @@ public void testCompressedSize() throws IOException { edges.write(dataOutput); // size of serializedEdges byte array should be equal to 15: - // 5 (bucket 0: 0-7) + 5 (bucket 1: 8-10) + 5 (bucket 3: 23) + // 5 (bucket 0: 0-7) + 5 (bucket 1: 8-10) + 5 (bucket 2: 23) ArgumentCaptor serializedEdgesCaptop = ArgumentCaptor.forClass(byte[].class); Mockito.verify(dataOutput).write(serializedEdgesCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); assertEquals(15, serializedEdgesCaptop.getValue().length); From fc958e13fa9e9f55fbf53d5a7020af5783deaabf Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 4 Sep 2018 18:08:32 +0300 Subject: [PATCH 7/9] fixed a comment --- .../java/org/apache/giraph/edge/IndexedBitArrayEdges.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java index e6fdbd09c..072445adf 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java @@ -292,12 +292,12 @@ public static int fromByteArray(byte[] bytes) { } /** - * returns a byte arrray with the concatenation of the three input parameters + * returns a byte array with the concatenation of the three input parameters * * @param original a byte array * @param bindex a byte array * @param myByte a byte - * @return a byte arrray with the concatenation of the three input parameters + * @return a byte array with the concatenation of the three input parameters * */ public static byte[] addBytes(byte[] original, byte[] bindex, byte myByte) { byte[] destination = new byte[original.length + Integer.SIZE / Byte.SIZE + 1]; From 9e8dbb33063c9b3d846139b2794496fd27e4906e Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 11 Sep 2018 10:54:50 +0300 Subject: [PATCH 8/9] javadoc improvements --- .../src/main/java/org/apache/giraph/edge/BVEdges.java | 11 +++++++---- .../org/apache/giraph/edge/IndexedBitArrayEdges.java | 11 +++++++---- .../org/apache/giraph/edge/IntervalResidualEdges.java | 11 +++++++---- 3 files changed, 21 insertions(+), 12 deletions(-) diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java index 947ea6f1f..21463bbb5 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -48,10 +48,13 @@ /** * Implementation of {@link OutEdges} with int ids and null edge values, backed - * by the BVEdges structure proposed in "Panagiotis Liakos, Katia - * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed - * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: - * this implementation is optimized for space usage for graphs exhibiting the + * by the BVEdges structure proposed in: + * + * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: + * Realizing Memory-Optimized Distributed Graph Processing. + * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). + * + * Note: this implementation is optimized for space usage for graphs exhibiting the * locality of reference property, but edge addition and * removals are expensive. Parallel edges are not allowed. */ diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java index 072445adf..f55dbaca5 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java @@ -34,10 +34,13 @@ /** * Implementation of {@link OutEdges} with int ids and null edge values, backed - * by the IndexedBitArrayEdges structure proposed in "Panagiotis Liakos, Katia - * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed - * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: - * this implementation is optimized for space usage for graphs exhibiting the + * by the IndexedBitArrayEdges structure proposed in: + * + * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: + * Realizing Memory-Optimized Distributed Graph Processing. + * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). + * + * Note: this implementation is optimized for space usage for graphs exhibiting the * locality of reference property, but edge addition and * removals are expensive. Parallel edges are ignored. */ diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java index f6c47cf4c..52c8be518 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -40,10 +40,13 @@ /** * Implementation of {@link OutEdges} with int ids and null edge values, backed - * by the IntervalResidualEdges structure proposed in "Panagiotis Liakos, Katia - * Papakonstantinopoulou, Alex Delis: Realizing Memory-Optimized Distributed - * Graph Processing. IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018)". Note: - * this implementation is optimized for space usage for graphs exhibiting the + * by the IntervalResidualEdges structure proposed in: + * + * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: + * Realizing Memory-Optimized Distributed Graph Processing. + * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). + * + * Note: this implementation is optimized for space usage for graphs exhibiting the * locality of reference property, but edge addition and * removals are expensive. Parallel edges are not allowed. */ From e045198388c9efc05fdfe09613d9781926c4a6b5 Mon Sep 17 00:00:00 2001 From: Panagiotis Liakos Date: Tue, 11 Sep 2018 17:42:40 +0300 Subject: [PATCH 9/9] compliation efforts --- giraph-core/pom.xml | 4 + .../java/org/apache/giraph/edge/BVEdges.java | 818 ++++++++++-------- .../giraph/edge/IndexedBitArrayEdges.java | 533 ++++++------ .../giraph/edge/IntervalResidualEdges.java | 429 ++++----- .../apache/giraph/edge/BVEdgesEdgesTest.java | 408 ++++----- .../giraph/edge/IndexedBitArrayEdgesTest.java | 422 ++++----- .../edge/IntervalResidualEdgesTest.java | 385 +++++---- pom.xml | 72 +- 8 files changed, 1641 insertions(+), 1430 deletions(-) diff --git a/giraph-core/pom.xml b/giraph-core/pom.xml index 9ff837be6..3703ade88 100644 --- a/giraph-core/pom.xml +++ b/giraph-core/pom.xml @@ -518,6 +518,10 @@ under the License. net.iharder base64 + + it.unimi.dsi + dsiutils + it.unimi.dsi fastutil diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java index 21463bbb5..6d31b4470 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/BVEdges.java @@ -35,13 +35,12 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; -import java.util.stream.StreamSupport; import org.apache.giraph.utils.Trimmable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; -import org.weakref.jmx.com.google.common.collect.Iterators; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.UnmodifiableIterator; @@ -49,389 +48,446 @@ /** * Implementation of {@link OutEdges} with int ids and null edge values, backed * by the BVEdges structure proposed in: - * + * * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: * Realizing Memory-Optimized Distributed Graph Processing. * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). - * - * Note: this implementation is optimized for space usage for graphs exhibiting the - * locality of reference property, but edge addition and - * removals are expensive. Parallel edges are not allowed. + * + * Note: this implementation is optimized for space usage for graphs exhibiting + * the locality of reference property, but edge addition and removals are + * expensive. Parallel edges are not allowed. */ public class BVEdges extends ConfigurableOutEdges - implements ReuseObjectsOutEdges, Trimmable { - - /** Serialized Intervals and Residuals */ - private byte[] intervalsAndResiduals; - /** Number of edges. */ - private int size; - - @Override - public void initialize(Iterable> edges) { - compress(StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId) - .mapToInt(IntWritable::get).toArray()); - } - - @Override - public void initialize(int capacity) { - size = 0; - intervalsAndResiduals = null; - } - - @Override - public void initialize() { - size = 0; - intervalsAndResiduals = null; - } - - @Override - public void add(Edge edge) { - // Note that ths is very expensive (decompresses all edges and recompresses them again). - compress(StreamSupport - .stream(((Iterable>) () -> Iterators.concat(this.iterator(), - ImmutableSet.of(edge).iterator())).spliterator(), false) - .map(Edge::getTargetVertexId).mapToInt(IntWritable::get).sorted().toArray()); - } - - @Override - public void remove(IntWritable targetVertexId) { - // Note that this is very expensive (decompresses all edges and recompresses them again). - initialize(Iterables.filter(this, edge -> !edge.getTargetVertexId().equals(targetVertexId))); - } - - @Override - public int size() { - return size; - } - - @Override - public Iterator> iterator() { - if (size == 0) { - return ImmutableSet.>of().iterator(); - } else { - return new BVEdgesIterator(); - } - } - - @Override - public void readFields(DataInput in) throws IOException { - size = in.readInt(); - int intervalResidualEdgesBytesUsed = in.readInt(); - if (intervalResidualEdgesBytesUsed > 0) { - // Only create a new buffer if the old one isn't big enough - if (intervalsAndResiduals == null || intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { - intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; - } - in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); - } - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(size); - out.writeInt(intervalsAndResiduals.length); - if (intervalsAndResiduals.length > 0) { - out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); - } - } - - @Override - public void trim() { - // Nothing to do - } - - - /** - * Receives an integer array of successors and compresses them in the - * intervalsAndResiduals byte array - * - * @param edgesArray an integer array of successors - */ - private void compress(final int[] edgesArray) { - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - diffComp(edgesArray, new OutputBitStream(baos)); - intervalsAndResiduals = baos.toByteArray(); - size = edgesArray.length; - } catch (IOException e) { - throw new IllegalArgumentException(e); - } - } - - /** - * See also {@link it.unimi.dsi.webgraph.BVGraph#diffComp(int[], OutputBitStream)}. - * This method is given an integer successors array and produces, onto a given output - * bit stream, the differentially compressed successor list. - * - * @param edgesArray the integer successors array - * @param obs an output bit stream where the compressed data will be - * stored. - */ - public static void diffComp(final int[] edgesArray, OutputBitStream obs) throws IOException { - // We write the degree. - obs.writeInt(edgesArray.length, Integer.SIZE); - IntArrayList left = new IntArrayList(); - IntArrayList len = new IntArrayList(); - IntArrayList residuals = new IntArrayList(); - // If we are to produce intervals, we first compute them. - int intervalCount; - try { - intervalCount = BVEdges.intervalize(edgesArray, BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, - left, len, residuals); - } catch (IllegalArgumentException e) { - // array was not sorted, sorting and retrying - Arrays.sort(edgesArray); - left = new IntArrayList(); - len = new IntArrayList(); - residuals = new IntArrayList(); - intervalCount = BVEdges.intervalize(edgesArray, BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, - left, len, residuals); - } - // We write the number of intervals. - obs.writeGamma(intervalCount); - int currIntLen; - int prev = 0; - if (intervalCount > 0) { - obs.writeInt(left.getInt(0), Integer.SIZE); - currIntLen = len.getInt(0); - prev = left.getInt(0) + currIntLen; - obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); - } - // We write out the intervals. - for (int i = 1; i < intervalCount; i++) { - obs.writeGamma(left.getInt(i) - prev - 1); - currIntLen = len.getInt(i); - prev = left.getInt(i) + currIntLen; - obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); - } - final int[] residual = residuals.elements(); - final int residualCount = residuals.size(); - // Now we write out the residuals, if any - if (residualCount != 0) { - if (intervalCount > 0) { - obs.writeLongZeta(Fast.int2nat((long) (prev = residual[0]) - left.getInt(0)), BVGraph.DEFAULT_ZETA_K); - } else { - prev = residual[0]; - obs.writeInt(prev, Integer.SIZE); - } - for (int i = 1; i < residualCount; i++) { - if (residual[i] == prev) { - throw new IllegalArgumentException( - "Repeated successor " + prev + " in successor list of this node"); - } - obs.writeLongZeta(residual[i] - prev - 1L, BVGraph.DEFAULT_ZETA_K); - prev = residual[i]; - } - } - obs.flush(); - } - - /** - * See also - * {@link it.unimi.dsi.webgraph.BVGraph#intervalize(IntArrayList, int, IntArrayList, IntArrayList, IntArrayList)}. - * This method tries to express an increasing sequence of natural numbers - * x as a union of an increasing sequence of intervals and an - * increasing sequence of residual elements. More precisely, this - * intervalization works as follows: first, one looks at edgesArray - * as a sequence of intervals (i.e., maximal sequences of consecutive elements); - * those intervals whose length is ≥ minInterval are stored in - * the lists left (the list of left extremes) and len - * (the list of lengths; the length of an integer interval is the number of - * integers in that interval). The remaining integers, called residuals - * are stored in the residual list. - * - *

- * Note that the previous content of left, len and - * residual is lost. - * - * @param edgesArray the array to be intervalized (an increasing list of - * natural numbers). - * @param minInterval the least length that a maximal sequence of consecutive - * elements must have in order for it to be considered as an - * interval. - * @param left the resulting list of left extremes of the intervals. - * @param len the resulting list of interval lengths. - * @param residuals the resulting list of residuals. - * @return the number of intervals. - */ - protected static int intervalize(final int[] edgesArray, final int minInterval, - final int maxInterval, final IntArrayList left, final IntArrayList len, final IntArrayList residuals) { - int nInterval = 0; - int i; - int j; - - for (i = 0; i < edgesArray.length; i++) { - j = 0; - checkIsSorted(edgesArray, i); - if (i < edgesArray.length - 1 && edgesArray[i] + 1 == edgesArray[i + 1]) { - do - j++; - while (i + j < edgesArray.length - 1 && j < maxInterval - && edgesArray[i + j] + 1 == edgesArray[i + j + 1]); - checkIsSorted(edgesArray, i + j); - j++; - // Now j is the number of integers in the interval. - if (j >= minInterval) { - left.add(edgesArray[i]); - len.add(j); - nInterval++; - i += j - 1; - } - } - if (j < minInterval) - residuals.add(edgesArray[i]); - } - return nInterval; - } - - /** - * Given an integer array and an index, this method throws an exception in case - * the element of the array the index points at is equal or larger than its - * next. - * - * @param edgesArray the integer array - * @param i the index - */ - private static void checkIsSorted(int[] edgesArray, int i) { - if (i < edgesArray.length - 1 && edgesArray[i] == edgesArray[i + 1]) { - throw new IllegalArgumentException("Parallel edges are not allowed."); - } - if (i < edgesArray.length - 1 && edgesArray[i] > edgesArray[i + 1]) { - throw new IllegalArgumentException("Edges are not sorted."); - } - } - - /** - * Iterator that reuses the same Edge object. - */ - private class BVEdgesIterator extends UnmodifiableIterator> { - /** Wrapped map iterator. */ - LazyIntIterator liIter = successors(new InputBitStream(intervalsAndResiduals)); - /** Representative edge object. */ - private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); - /** Current edge count */ - private int currentEdge = 0; - - @Override - public boolean hasNext() { - return currentEdge < size; - } - - @Override - public Edge next() { - representativeEdge.getTargetVertexId().set(liIter.nextInt()); - currentEdge++; - return representativeEdge; - } - - private LazyIntIterator successors(InputBitStream ibs) { - try { - final int d; - int extraCount; - int firstIntervalNode = -1; - ibs.position(0); - d = ibs.readInt(Integer.SIZE); - if (d == 0) - return LazyIntIterators.EMPTY_ITERATOR; - extraCount = d; - int intervalCount = 0; // Number of intervals - int[] left = null; - int[] len = null; - // Prepare to read intervals, if any - if (extraCount > 0 && (intervalCount = ibs.readGamma()) != 0) { - int prev = 0; // Holds the last integer in the last - // interval. - left = new int[intervalCount]; - len = new int[intervalCount]; - // Now we read intervals - left[0] = firstIntervalNode = prev = ibs.readInt(Integer.SIZE); - len[0] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; - - prev += len[0]; - extraCount -= len[0]; - for (int i = 1; i < intervalCount; i++) { - left[i] = prev = ibs.readGamma() + prev + 1; - len[i] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; - prev += len[i]; - extraCount -= len[i]; - } - } - - final int residualCount = extraCount; // Just to be able to use - // an - // anonymous class. - final LazyIntIterator residualIterator = residualCount == 0 ? null - : new ResidualIntIterator(ibs, residualCount, firstIntervalNode); - // The extra part is made by the contribution of intervals, if - // any, and by the residuals iterator. - if (intervalCount == 0) { - return residualIterator; - } else if (residualCount == 0){ - return new IntIntervalSequenceIterator(left, len); - } else { - return new MergedIntIterator(new IntIntervalSequenceIterator(left, len), residualIterator); - } - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - /** An iterator returning the residuals of a node. */ - private final class ResidualIntIterator extends AbstractLazyIntIterator { - /** The input bit stream from which residuals will be read. */ - private final InputBitStream ibs; - /** The last residual returned. */ - private int next; - /** The number of remaining residuals. */ - private int remaining; - - private ResidualIntIterator(final InputBitStream ibs, final int residualCount, int x) { - this.remaining = residualCount; - this.ibs = ibs; - try { - if (x >= 0) { - long temp = Fast.nat2int(ibs.readLongZeta(BVGraph.DEFAULT_ZETA_K)); - this.next = (int) (x + temp); - } else { - this.next = ibs.readInt(Integer.SIZE); - } - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - public int nextInt() { - if (remaining == 0) - return -1; - try { - final int result = next; - if (--remaining != 0) - next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; - return result; - } catch (IOException cantHappen) { - throw new IllegalStateException(cantHappen); - } - } - - @Override - public int skip(int n) { - if (n >= remaining) { - n = remaining; - remaining = 0; - return n; - } - try { - for (int i = n; i-- != 0;) - next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; - remaining -= n; - return n; - } catch (IOException cantHappen) { - throw new IllegalStateException(cantHappen); - } - } - - } - - } + implements ReuseObjectsOutEdges, Trimmable { + + /** Serialized Intervals and Residuals */ + private byte[] intervalsAndResiduals; + /** Number of edges. */ + private int size; + + @Override + public void initialize(Iterable> edges) { + IntArrayList edgesList = new IntArrayList(); + for (Iterator> iter = edges.iterator(); iter + .hasNext();) { + edgesList.add(iter.next().getTargetVertexId().get()); + } + compress(Arrays.copyOfRange(edgesList.elements(), 0, edgesList.size())); + } + + @Override + public void initialize(int capacity) { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void initialize() { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void add(Edge edge) { + // Note that this is very expensive (decompresses all edges and recompresses + // them again). + IntArrayList edgesList = new IntArrayList(); + for (Iterator> iter = this.iterator(); iter + .hasNext();) { + edgesList.add(iter.next().getTargetVertexId().get()); + } + edgesList.add(edge.getTargetVertexId().get()); + compress(Arrays.copyOfRange(edgesList.elements(), 0, edgesList.size())); + } + + @Override + public void remove(IntWritable targetVertexId) { + // Note that this is very expensive (decompresses all edges and recompresses + // them again). + final int id = targetVertexId.get(); + initialize(Iterables.filter(this, + new Predicate>() { + @Override + public boolean apply(Edge edge) { + return edge.getTargetVertexId().get() != id; + } + })); + } + + @Override + public int size() { + return size; + } + + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new BVEdgesIterator(); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int intervalResidualEdgesBytesUsed = in.readInt(); + if (intervalResidualEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (intervalsAndResiduals == null || + intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { + intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; + } + in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(intervalsAndResiduals.length); + if (intervalsAndResiduals.length > 0) { + out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); + } + } + + @Override + public void trim() { + // Nothing to do + } + + /** + * Receives an integer array of successors and compresses them in the + * intervalsAndResiduals byte array + * + * @param edgesArray an integer array of successors + */ + private void compress(final int[] edgesArray) { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + diffComp(edgesArray, new OutputBitStream(baos)); + intervalsAndResiduals = baos.toByteArray(); + size = edgesArray.length; + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + /** + * See also + * {@link it.unimi.dsi.webgraph.BVGraph#diffComp(int[], OutputBitStream)}. + * This method is given an integer successors array and produces, onto a given + * output bit stream, the differentially compressed successor list. + * + * @param edgesArray the integer successors array + * @param obs an output bit stream where the compressed data will be + * stored. + */ + public static void diffComp(final int[] edgesArray, OutputBitStream obs) + throws IOException { + // We write the degree. + obs.writeInt(edgesArray.length, Integer.SIZE); + IntArrayList left = new IntArrayList(); + IntArrayList len = new IntArrayList(); + IntArrayList residuals = new IntArrayList(); + // If we are to produce intervals, we first compute them. + int intervalCount; + try { + intervalCount = BVEdges.intervalize(edgesArray, + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, left, len, + residuals); + } catch (IllegalArgumentException e) { + // array was not sorted, sorting and retrying + Arrays.sort(edgesArray); + left = new IntArrayList(); + len = new IntArrayList(); + residuals = new IntArrayList(); + intervalCount = BVEdges.intervalize(edgesArray, + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH, Integer.MAX_VALUE, left, len, + residuals); + } + // We write the number of intervals. + obs.writeGamma(intervalCount); + int currIntLen; + int prev = 0; + if (intervalCount > 0) { + obs.writeInt(left.getInt(0), Integer.SIZE); + currIntLen = len.getInt(0); + prev = left.getInt(0) + currIntLen; + obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); + } + // We write out the intervals. + for (int i = 1; i < intervalCount; i++) { + obs.writeGamma(left.getInt(i) - prev - 1); + currIntLen = len.getInt(i); + prev = left.getInt(i) + currIntLen; + obs.writeGamma(currIntLen - BVGraph.DEFAULT_MIN_INTERVAL_LENGTH); + } + final int[] residual = residuals.elements(); + final int residualCount = residuals.size(); + // Now we write out the residuals, if any + if (residualCount != 0) { + if (intervalCount > 0) { + prev = residual[0]; + obs.writeLongZeta( + Fast.int2nat((long) prev - left.getInt(0)), + BVGraph.DEFAULT_ZETA_K); + } else { + prev = residual[0]; + obs.writeInt(prev, Integer.SIZE); + } + for (int i = 1; i < residualCount; i++) { + if (residual[i] == prev) { + throw new IllegalArgumentException( + "Repeated successor " + prev + " in successor list of this node"); + } + obs.writeLongZeta(residual[i] - prev - 1L, BVGraph.DEFAULT_ZETA_K); + prev = residual[i]; + } + } + obs.flush(); + } + + /** + * See also {@link it.unimi.dsi.webgraph.BVGraph#intervalize(IntArrayList, + * int, IntArrayList, IntArrayList, IntArrayList)}. + * This method tries to express an increasing sequence of natural numbers + * x as a union of an increasing sequence of intervals and an + * increasing sequence of residual elements. More precisely, this + * intervalization works as follows: first, one looks at + * edgesArray as a sequence of intervals (i.e., maximal sequences + * of consecutive elements); those intervals whose length is ≥ + * minInterval are stored in the lists left (the + * list of left extremes) and len (the list of lengths; the + * length of an integer interval is the number of integers in that interval). + * The remaining integers, called residuals are stored in the + * residual list. + * + *

+ * Note that the previous content of left, len and + * residual is lost. + * + * @param edgesArray the array to be intervalized (an increasing list of + * natural numbers). + * @param minInterval the least length that a maximal sequence of consecutive + * elements must have in order for it to be considered as + * an interval. + * @param maxInterval the maximum length that a maximal sequence of + * consecutive elements must have in order for it to be + * considered as an interval. + * @param left the resulting list of left extremes of the intervals. + * @param len the resulting list of interval lengths. + * @param residuals the resulting list of residuals. + * @return the number of intervals. + */ + protected static int intervalize(final int[] edgesArray, + final int minInterval, final int maxInterval, final IntArrayList left, + final IntArrayList len, final IntArrayList residuals) { + int nInterval = 0; + int i; + int j; + + for (i = 0; i < edgesArray.length; i++) { + j = 0; + checkIsSorted(edgesArray, i); + if (i < edgesArray.length - 1 && edgesArray[i] + 1 == edgesArray[i + 1]) { + do { + j++; + } while (i + j < edgesArray.length - 1 && j < maxInterval && + edgesArray[i + j] + 1 == edgesArray[i + j + 1]); + checkIsSorted(edgesArray, i + j); + j++; + // Now j is the number of integers in the interval. + if (j >= minInterval) { + left.add(edgesArray[i]); + len.add(j); + nInterval++; + i += j - 1; + } + } + if (j < minInterval) { + residuals.add(edgesArray[i]); + } + } + return nInterval; + } + + /** + * Given an integer array and an index, this method throws an exception in + * case the element of the array the index points at is equal or larger than + * its next. + * + * @param edgesArray the integer array + * @param i the index + */ + private static void checkIsSorted(int[] edgesArray, int i) { + if (i < edgesArray.length - 1 && edgesArray[i] == edgesArray[i + 1]) { + throw new IllegalArgumentException("Parallel edges are not allowed."); + } + if (i < edgesArray.length - 1 && edgesArray[i] > edgesArray[i + 1]) { + throw new IllegalArgumentException("Edges are not sorted."); + } + } + + /** + * Iterator that reuses the same Edge object. + */ + private class BVEdgesIterator + extends UnmodifiableIterator> { + /** Wrapped map iterator. */ + private LazyIntIterator liIter = successors( + new InputBitStream(intervalsAndResiduals)); + /** Representative edge object. */ + private final Edge representativeEdge = + EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + + @Override + public boolean hasNext() { + return currentEdge < size; + } + + @Override + public Edge next() { + representativeEdge.getTargetVertexId().set(liIter.nextInt()); + currentEdge++; + return representativeEdge; + } + + /** Creates an iterator from an input bit stream + * + * @param ibs input bit stream with compressed intervals and residuals + * @return an iterator with the successors of the input bit stream + * + * */ + private LazyIntIterator successors(InputBitStream ibs) { + try { + final int d; + int extraCount; + int firstIntervalNode = -1; + ibs.position(0); + d = ibs.readInt(Integer.SIZE); + if (d == 0) { + return LazyIntIterators.EMPTY_ITERATOR; + } + extraCount = d; + int intervalCount = 0; // Number of intervals + int[] left = null; + int[] len = null; + // Prepare to read intervals, if any + intervalCount = ibs.readGamma(); + if (extraCount > 0 && intervalCount != 0) { + int prev = 0; // Holds the last integer in the last + // interval. + left = new int[intervalCount]; + len = new int[intervalCount]; + // Now we read intervals + firstIntervalNode = ibs.readInt(Integer.SIZE); + left[0] = firstIntervalNode; + len[0] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + + prev = left[0] + len[0]; + extraCount -= len[0]; + for (int i = 1; i < intervalCount; i++) { + left[i] = ibs.readGamma() + prev + 1; + len[i] = ibs.readGamma() + BVGraph.DEFAULT_MIN_INTERVAL_LENGTH; + prev = left[i] + len[i]; + extraCount -= len[i]; + } + } + + final int residualCount = extraCount; // Just to be able to use + // an + // anonymous class. + final LazyIntIterator residualIterator = residualCount == 0 ? null : + new ResidualIntIterator(ibs, residualCount, firstIntervalNode); + // The extra part is made by the contribution of intervals, if + // any, and by the residuals iterator. + if (intervalCount == 0) { + return residualIterator; + } else if (residualCount == 0) { + return new IntIntervalSequenceIterator(left, len); + } else { + return new MergedIntIterator( + new IntIntervalSequenceIterator(left, len), residualIterator); + } + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + /** An iterator returning the residuals of a node. */ + private final class ResidualIntIterator extends AbstractLazyIntIterator { + /** The input bit stream from which residuals will be read. */ + private final InputBitStream ibs; + /** The last residual returned. */ + private int next; + /** The number of remaining residuals. */ + private int remaining; + + /** Constructor + * + * @param ibs the input bit stream with the residuals + * @param remaining the total elements in the stream + * @param x the value of the previous (or first) edge id + * + * */ + private ResidualIntIterator(final InputBitStream ibs, + final int remaining, int x) { + this.remaining = remaining; + this.ibs = ibs; + try { + if (x >= 0) { + long temp = Fast.nat2int(ibs.readLongZeta(BVGraph.DEFAULT_ZETA_K)); + this.next = (int) (x + temp); + } else { + this.next = ibs.readInt(Integer.SIZE); + } + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + /** Return the next integer of the iterator + * + * @return the next integer of the iterator + * */ + public int nextInt() { + if (remaining == 0) { + return -1; + } + try { + final int result = next; + if (--remaining != 0) { + next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; + } + return result; + } catch (IOException cantHappen) { + throw new IllegalStateException(cantHappen); + } + } + + @Override + public int skip(int n) { + if (n >= remaining) { + n = remaining; + remaining = 0; + return n; + } + try { + for (int i = n; i-- != 0;) { + next += ibs.readZeta(BVGraph.DEFAULT_ZETA_K) + 1; + } + remaining -= n; + return n; + } catch (IOException cantHappen) { + throw new IllegalStateException(cantHappen); + } + } + + } + + } } diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java index f55dbaca5..7254a7a99 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IndexedBitArrayEdges.java @@ -21,6 +21,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.UnmodifiableIterator; +import it.unimi.dsi.fastutil.ints.IntArrayList; + import org.apache.giraph.utils.Trimmable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; @@ -30,284 +32,313 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; -import java.util.stream.StreamSupport; /** * Implementation of {@link OutEdges} with int ids and null edge values, backed * by the IndexedBitArrayEdges structure proposed in: - * + * * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: * Realizing Memory-Optimized Distributed Graph Processing. * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). - * - * Note: this implementation is optimized for space usage for graphs exhibiting the - * locality of reference property, but edge addition and - * removals are expensive. Parallel edges are ignored. + * + * Note: this implementation is optimized for space usage for graphs exhibiting + * the locality of reference property, but edge addition and removals are + * expensive. Parallel edges are ignored. */ -public class IndexedBitArrayEdges extends ConfigurableOutEdges - implements ReuseObjectsOutEdges, Trimmable { +public class IndexedBitArrayEdges + extends ConfigurableOutEdges + implements ReuseObjectsOutEdges, Trimmable { + + /** Serialized edges. */ + private byte[] serializedEdges; + /** Number of edges. */ + private int size; - /** Serialized edges. */ - private byte[] serializedEdges; - /** Number of edges. */ - private int size; + @Override + public void initialize(Iterable> edges) { + initialize(); + IntArrayList edgesList = new IntArrayList(); + for (Iterator> iter = edges.iterator(); iter + .hasNext();) { + edgesList.add(iter.next().getTargetVertexId().get()); + } + int[] edgesArray = Arrays.copyOfRange(edgesList.elements(), 0, + edgesList.size()); + Arrays.sort(edgesArray); + if (edgesArray.length == 0) { + return; + } + int previousId = -1; + int currBucket = edgesArray[0] / Byte.SIZE; + byte myByte = (byte) 0; + for (int edge : edgesArray) { + if (edge == previousId) { + throw new IllegalArgumentException("Repeated successor " + previousId + + " in successor list of this node"); + } + previousId = edge; + int bucket = edge / Byte.SIZE; + int pos = edge % Byte.SIZE; + if (bucket == currBucket) { + myByte = setBit(myByte, pos); + size += 1; + } else { + serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), + myByte); + currBucket = bucket; + myByte = setBit((byte) 0, pos); + size += 1; + } + } + serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), + myByte); + } - @Override - public void initialize(Iterable> edges) { - initialize(); - int[] sortedEdgesArray = StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId) - .mapToInt(IntWritable::get).sorted().distinct().toArray(); - if (sortedEdgesArray.length == 0) { - return; - } - int currBucket = sortedEdgesArray[0] / Byte.SIZE; - byte myByte = (byte) 0; - for (int edge : sortedEdgesArray) { - int bucket = edge / Byte.SIZE; - int pos = edge % Byte.SIZE; - if (bucket == currBucket) { - myByte = setBit(myByte, pos); - size += 1; - } else { - serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), myByte); - currBucket = bucket; - myByte = setBit((byte) 0, pos); - size += 1; - } - } - serializedEdges = addBytes(serializedEdges, toByteArray(currBucket), myByte); - } + @Override + public void initialize(int capacity) { + size = 0; + serializedEdges = new byte[0]; + } - @Override - public void initialize(int capacity) { - size = 0; - serializedEdges = new byte[0]; - } + @Override + public void initialize() { + size = 0; + serializedEdges = new byte[0]; + } - @Override - public void initialize() { - size = 0; - serializedEdges = new byte[0]; - } + @Override + public void add(Edge edge) { + int bucket = edge.getTargetVertexId().get() / Byte.SIZE; + int pos = edge.getTargetVertexId().get() % Byte.SIZE; + int i = 0; + boolean done = false; + while (i < serializedEdges.length) { // if bucket is already there, simply + // set the appropriate bit + if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, + i + Integer.SIZE / Byte.SIZE)) == bucket) { + int index = i + Integer.SIZE / Byte.SIZE; + if (!isSet(serializedEdges[index], pos)) { + size += 1; + serializedEdges[index] = setBit(serializedEdges[index], pos); + } + done = true; + break; + } + i += Integer.SIZE / Byte.SIZE + 1; + } + if (!done) { // we need to add a bucket + serializedEdges = addBytes(serializedEdges, toByteArray(bucket), + setBit((byte) 0, pos)); + size += 1; + } + } - @Override - public void add(Edge edge) { - int bucket = edge.getTargetVertexId().get() / Byte.SIZE; - int pos = edge.getTargetVertexId().get() % Byte.SIZE; - int i = 0; - boolean done = false; - while (i < serializedEdges.length) { // if bucket is already there, simply set the appropriate bit - if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, i + Integer.SIZE / Byte.SIZE)) == bucket) { - int index = i + Integer.SIZE / Byte.SIZE; - if (!isSet(serializedEdges[index], pos)) { - size += 1; - serializedEdges[index] = setBit(serializedEdges[index], pos); - } - done = true; - break; - } - i += Integer.SIZE / Byte.SIZE + 1; - } - if (!done) { // we need to add a bucket - serializedEdges = addBytes(serializedEdges, toByteArray(bucket), setBit((byte) 0, pos)); - size += 1; - } - } + @Override + public void remove(IntWritable targetVertexId) { + int bucket = targetVertexId.get() / Byte.SIZE; + int pos = targetVertexId.get() % Byte.SIZE; + int i = 0; + while (i < serializedEdges.length) { + if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, + i + Integer.SIZE / Byte.SIZE)) == bucket) { + serializedEdges[i + Integer.SIZE / Byte.SIZE] = unsetBit( + serializedEdges[i + Integer.SIZE / Byte.SIZE], pos); + --size; + break; + } + i += Integer.SIZE / Byte.SIZE + 1; + } + } - @Override - public void remove(IntWritable targetVertexId) { - int bucket = targetVertexId.get() / Byte.SIZE; - int pos = targetVertexId.get() % Byte.SIZE; - int i = 0; - while (i < serializedEdges.length) { - if (fromByteArray(Arrays.copyOfRange(serializedEdges, i, i + Integer.SIZE / Byte.SIZE)) == bucket) { - serializedEdges[i + Integer.SIZE / Byte.SIZE] = unsetBit(serializedEdges[i + Integer.SIZE / Byte.SIZE], - pos); - --size; - break; - } - i += Integer.SIZE / Byte.SIZE + 1; - } - } + @Override + public int size() { + return size; + } - @Override - public int size() { - return size; - } + @Override + public void trim() { + // Nothing to do + } - @Override - public void trim() { - // Nothing to do - } - - @Override - public Iterator> iterator() { - if (size == 0) { - return ImmutableSet.>of().iterator(); - } else { - return new IndexedBitmapEdgeIterator(); - } - } + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new IndexedBitmapEdgeIterator(); + } + } - @Override - public void readFields(DataInput in) throws IOException { - size = in.readInt(); - int serializedEdgesBytesUsed = in.readInt(); - if (serializedEdgesBytesUsed > 0) { - // Only create a new buffer if the old one isn't big enough - if (serializedEdges == null || serializedEdgesBytesUsed > serializedEdges.length) { - serializedEdges = new byte[serializedEdgesBytesUsed]; - } - in.readFully(serializedEdges, 0, serializedEdgesBytesUsed); - } - } + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int serializedEdgesBytesUsed = in.readInt(); + if (serializedEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (serializedEdges == null || + serializedEdgesBytesUsed > serializedEdges.length) { + serializedEdges = new byte[serializedEdgesBytesUsed]; + } + in.readFully(serializedEdges, 0, serializedEdgesBytesUsed); + } + } - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(size); - out.writeInt(serializedEdges.length); - if (serializedEdges.length > 0) { - out.write(serializedEdges, 0, serializedEdges.length); - } - } + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(serializedEdges.length); + if (serializedEdges.length > 0) { + out.write(serializedEdges, 0, serializedEdges.length); + } + } - /** Iterator that reuses the same Edge object. */ - private class IndexedBitmapEdgeIterator extends UnmodifiableIterator> { - /** Representative edge object. */ - private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); - /** Current edge count */ - private int currentEdge = 0; - /** Current position */ - private int currentBitPosition = 0; - private int currentBytePosition = 0; - /** Index int */ - private int indexInt; - /** Current byte */ - private Byte myByte; - /** Current index */ - private byte[] myBytes = new byte[Integer.SIZE / Byte.SIZE]; + /** Iterator that reuses the same Edge object. */ + private class IndexedBitmapEdgeIterator + extends UnmodifiableIterator> { + /** Representative edge object. */ + private final Edge representativeEdge = + EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + /** Current bit position */ + private int currentBitPosition = 0; + /** Current byte position */ + private int currentBytePosition = 0; + /** Index int */ + private int indexInt; + /** Current byte */ + private Byte myByte; + /** Current index */ + private byte[] myBytes = new byte[Integer.SIZE / Byte.SIZE]; - @Override - public boolean hasNext() { - return currentEdge < size; - } + @Override + public boolean hasNext() { + return currentEdge < size; + } - @Override - public Edge next() { - if (currentBitPosition == 8) { - currentBitPosition = 0; - } - if (currentBitPosition == 0) { - myBytes[0] = serializedEdges[currentBytePosition]; - myBytes[1] = serializedEdges[currentBytePosition + 1]; - myBytes[2] = serializedEdges[currentBytePosition + 2]; - myBytes[3] = serializedEdges[currentBytePosition + 3]; - indexInt = fromByteArray(myBytes); - myByte = serializedEdges[currentBytePosition + Integer.SIZE / Byte.SIZE]; - currentBytePosition += Integer.SIZE / Byte.SIZE + 1; - } - int pos = currentBitPosition; - int nextPos = 0; - boolean done = false; - while (!done) { - for (int i = pos; i < Byte.SIZE; i++) { - if (isSet(myByte, i)) { - done = true; - nextPos = i; - currentEdge++; - currentBitPosition = i + 1; - break; - } - } - if (!done /* && mapIterator.hasNext() */) { - myBytes[0] = serializedEdges[currentBytePosition]; - myBytes[1] = serializedEdges[currentBytePosition + 1]; - myBytes[2] = serializedEdges[currentBytePosition + 2]; - myBytes[3] = serializedEdges[currentBytePosition + 3]; - indexInt = fromByteArray(myBytes); - myByte = serializedEdges[currentBytePosition + Integer.SIZE / Byte.SIZE]; - currentBytePosition += Integer.SIZE / Byte.SIZE + 1; - pos = 0; - } - } - representativeEdge.getTargetVertexId().set(indexInt * Byte.SIZE + nextPos); - return representativeEdge; - } + @Override + public Edge next() { + if (currentBitPosition == 8) { + currentBitPosition = 0; + } + if (currentBitPosition == 0) { + myBytes[0] = serializedEdges[currentBytePosition]; + myBytes[1] = serializedEdges[currentBytePosition + 1]; + myBytes[2] = serializedEdges[currentBytePosition + 2]; + myBytes[3] = serializedEdges[currentBytePosition + 3]; + indexInt = fromByteArray(myBytes); + myByte = serializedEdges[currentBytePosition + + Integer.SIZE / Byte.SIZE]; + currentBytePosition += Integer.SIZE / Byte.SIZE + 1; + } + int pos = currentBitPosition; + int nextPos = 0; + boolean done = false; + while (!done) { + for (int i = pos; i < Byte.SIZE; i++) { + if (isSet(myByte, i)) { + done = true; + nextPos = i; + currentEdge++; + currentBitPosition = i + 1; + break; + } + } + if (!done /* && mapIterator.hasNext() */) { + myBytes[0] = serializedEdges[currentBytePosition]; + myBytes[1] = serializedEdges[currentBytePosition + 1]; + myBytes[2] = serializedEdges[currentBytePosition + 2]; + myBytes[3] = serializedEdges[currentBytePosition + 3]; + indexInt = fromByteArray(myBytes); + myByte = serializedEdges[currentBytePosition + + Integer.SIZE / Byte.SIZE]; + currentBytePosition += Integer.SIZE / Byte.SIZE + 1; + pos = 0; + } + } + representativeEdge.getTargetVertexId() + .set(indexInt * Byte.SIZE + nextPos); + return representativeEdge; + } - } + } - /** - * tests if bit is set in a byte - * - * @param myByte the byte to be tested - * @param pos the position in the byte to be tested - * @returns true or false depending on the bit being set - * - */ - public static boolean isSet(byte myByte, int pos) { - return (myByte & (1 << pos)) != 0; - } + /** + * tests if bit is set in a byte + * + * @param myByte the byte to be tested + * @param pos the position in the byte to be tested + * @return true or false depending on the bit being set + * + */ + public static boolean isSet(byte myByte, int pos) { + return (myByte & (1 << pos)) != 0; + } - /** - * tests if bit is set in a byte - * - * @param byteWritable the byte to be updated - * @param pos the position in the byte to be set - * @returns the updated byte - * - */ - public static byte setBit(byte myByte, int pos) { - return (byte) (myByte | (1 << pos)); - } + /** + * tests if bit is set in a byte + * + * @param myByte the byte to be updated + * @param pos the position in the byte to be set + * @return the updated byte + * + */ + public static byte setBit(byte myByte, int pos) { + return (byte) (myByte | (1 << pos)); + } - /** - * tests if bit is set in a byte - * - * @param my_byte the byte to be updated - * @param pos the position in the byte to be unset - * @returns the updated byte - * - */ - public static byte unsetBit(byte myByte, int pos) { - return (byte) (myByte & ~(1 << pos)); - } + /** + * tests if bit is set in a byte + * + * @param myByte the byte to be updated + * @param pos the position in the byte to be unset + * @return the updated byte + * + */ + public static byte unsetBit(byte myByte, int pos) { + return (byte) (myByte & ~(1 << pos)); + } - /** - * converts an integer to a 4-byte length array holding - * its binary representation - * - * @param value the integer to be converted - * @return the 4-byte length byte array holding the integer's - * binary representation - */ - public static byte[] toByteArray(int value) { - return new byte[] { (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value }; - } + /** + * converts an integer to a 4-byte length array holding its binary + * representation + * + * @param value the integer to be converted + * @return the 4-byte length byte array holding the integer's binary + * representation + */ + public static byte[] toByteArray(int value) { + return new byte[] { (byte) (value >> 24), (byte) (value >> 16), + (byte) (value >> 8), (byte) value }; + } - /** - * converts a 4-byte length byte array to its respective integer - * - * @param bytes a 4-byte length byte array - * @return the integer represented in the byte array - */ - public static int fromByteArray(byte[] bytes) { - return bytes[0] << 24 | (bytes[1] & 0xFF) << 16 | (bytes[2] & 0xFF) << 8 | (bytes[3] & 0xFF); - } + /** + * converts a 4-byte length byte array to its respective integer + * + * @param bytes a 4-byte length byte array + * @return the integer represented in the byte array + */ + public static int fromByteArray(byte[] bytes) { + return bytes[0] << 24 | (bytes[1] & 0xFF) << 16 | (bytes[2] & 0xFF) << 8 | + (bytes[3] & 0xFF); + } - /** - * returns a byte array with the concatenation of the three input parameters - * - * @param original a byte array - * @param bindex a byte array - * @param myByte a byte - * @return a byte array with the concatenation of the three input parameters - * */ - public static byte[] addBytes(byte[] original, byte[] bindex, byte myByte) { - byte[] destination = new byte[original.length + Integer.SIZE / Byte.SIZE + 1]; - System.arraycopy(original, 0, destination, 0, original.length); - System.arraycopy(bindex, 0, destination, original.length, Integer.SIZE / Byte.SIZE); - destination[destination.length - 1] = myByte; - return destination; - } + /** + * returns a byte array with the concatenation of the three input parameters + * + * @param original a byte array + * @param bindex a byte array + * @param myByte a byte + * @return a byte array with the concatenation of the three input parameters + */ + public static byte[] addBytes(byte[] original, byte[] bindex, byte myByte) { + byte[] destination = new byte[original.length + Integer.SIZE / Byte.SIZE + + 1]; + System.arraycopy(original, 0, destination, 0, original.length); + System.arraycopy(bindex, 0, destination, original.length, + Integer.SIZE / Byte.SIZE); + destination[destination.length - 1] = myByte; + return destination; + } } diff --git a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java index 52c8be518..c63f59b6c 100644 --- a/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java +++ b/giraph-core/src/main/java/org/apache/giraph/edge/IntervalResidualEdges.java @@ -25,15 +25,14 @@ import java.io.IOException; import java.util.Arrays; import java.util.Iterator; -import java.util.stream.StreamSupport; import org.apache.giraph.utils.ExtendedDataInput; import org.apache.giraph.utils.ExtendedDataOutput; import org.apache.giraph.utils.Trimmable; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; -import org.weakref.jmx.com.google.common.collect.Iterators; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.UnmodifiableIterator; @@ -41,205 +40,237 @@ /** * Implementation of {@link OutEdges} with int ids and null edge values, backed * by the IntervalResidualEdges structure proposed in: - * + * * Panagiotis Liakos, Katia Papakonstantinopoulou, Alex Delis: * Realizing Memory-Optimized Distributed Graph Processing. * IEEE Trans. Knowl. Data Eng. 30(4): 743-756 (2018). - * - * Note: this implementation is optimized for space usage for graphs exhibiting the - * locality of reference property, but edge addition and - * removals are expensive. Parallel edges are not allowed. + * + * Note: this implementation is optimized for space usage for graphs exhibiting + * the locality of reference property, but edge addition and removals are + * expensive. Parallel edges are not allowed. */ -public class IntervalResidualEdges extends ConfigurableOutEdges - implements ReuseObjectsOutEdges, Trimmable { - - /** Minimum interval length is equal to 2 */ - private static final int MIN_INTERVAL_LENGTH = 2; - - /** Maximum interval length is equal to 254 */ - private static final int MAX_INTERVAL_LENGTH = 254; - - /** Serialized Intervals and Residuals */ - private byte[] intervalsAndResiduals; - - /** Number of edges stored in compressed array */ - private int size; - - @Override - public void initialize(Iterable> edges) { - compress(StreamSupport.stream(edges.spliterator(), false).map(Edge::getTargetVertexId).mapToInt(IntWritable::get).toArray()); - } - - @Override - public void initialize(int capacity) { - size = 0; - intervalsAndResiduals = null; - } - - @Override - public void initialize() { - size = 0; - intervalsAndResiduals = null; - } - - @Override - public void add(Edge edge) { - // Note that this is very expensive (decompresses all edges and recompresses them again). - compress(StreamSupport.stream(((Iterable>)() -> Iterators.concat(this.iterator(), ImmutableSet.of(edge).iterator())).spliterator(), false).map(Edge::getTargetVertexId).mapToInt(IntWritable::get).sorted().toArray()); - } - - @Override - public void remove(IntWritable targetVertexId) { - // Note that this is very expensive (decompresses all edges and recompresses them again). - initialize(Iterables.filter(this, edge -> !edge.getTargetVertexId().equals(targetVertexId))); - } - - @Override - public int size() { - return size; - } - - @Override - public Iterator> iterator() { - if (size == 0) { - return ImmutableSet.>of().iterator(); - } else { - return new IntervalResidualEdgeIterator(); - } - } - - @Override - public void readFields(DataInput in) throws IOException { - size = in.readInt(); - int intervalResidualEdgesBytesUsed = in.readInt(); - if (intervalResidualEdgesBytesUsed > 0) { - // Only create a new buffer if the old one isn't big enough - if (intervalsAndResiduals == null || intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { - intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; - } - in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); - } - } - - @Override - public void write(DataOutput out) throws IOException { - out.writeInt(size); - out.writeInt(intervalsAndResiduals.length); - if (intervalsAndResiduals.length > 0) { - out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); - } - } - - @Override - public void trim() { - /* Nothing to do */ - } - - /** Receives an integer array of successors and compresses them - * in the intervalsAndResiduals byte array - * - * @param edgesArray an integer array of successors - * */ - private void compress(final int[] edgesArray) { - try { - ExtendedDataOutput eos = getConf().createExtendedDataOutput(); - IntArrayList left = new IntArrayList(); - IntArrayList len = new IntArrayList(); - IntArrayList residuals = new IntArrayList(); - // If we are to produce intervals, we first compute them. - int intervalCount; - try { - intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, MAX_INTERVAL_LENGTH, left, len, - residuals); - } catch (IllegalArgumentException e) { - // array was not sorted, sorting and retrying - Arrays.sort(edgesArray); - left = new IntArrayList(); - len = new IntArrayList(); - residuals = new IntArrayList(); - intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, MAX_INTERVAL_LENGTH, left, len, - residuals); - } - - // We write out the intervals. - eos.writeInt(intervalCount); - for (int i = 0; i < intervalCount; i++) { - eos.writeInt(left.getInt(i)); - eos.write(len.getInt(i)); - } - final int[] residual = residuals.elements(); - final int residualCount = residuals.size(); - for (int i = 0; i < residualCount; i++) { - eos.writeInt(residual[i]); - } - intervalsAndResiduals = eos.toByteArray(); - size = edgesArray.length; - } catch (IOException e) { - throw new IllegalArgumentException(e); - } - } - - /** - * Iterator that reuses the same Edge object. - */ - private class IntervalResidualEdgeIterator extends UnmodifiableIterator> { - private ExtendedDataInput extendedDataInput = getConf().createExtendedDataInput(intervalsAndResiduals, 0, - intervalsAndResiduals.length); - /** Representative edge object. */ - private final Edge representativeEdge = EdgeFactory.create(new IntWritable()); - /** Current edge count */ - private int currentEdge = 0; - private int currentLeft; - private int currentLen = 0; - private int intervalCount; - - public IntervalResidualEdgeIterator() { - try { - intervalCount = extendedDataInput.readInt(); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - @Override - public boolean hasNext() { - return currentEdge < size; - } - - @Override - public Edge next() { - this.currentEdge++; - switch (this.currentLen) { - case 0: - switch (this.intervalCount) { - case 0: - try { - representativeEdge.getTargetVertexId().set(extendedDataInput.readInt()); - } catch (IOException canthappen) { - throw new IllegalStateException(canthappen); - } - return representativeEdge; - default: - try { - this.currentLeft = extendedDataInput.readInt(); - this.currentLen = extendedDataInput.readByte() & 0xff; - intervalCount--; - } catch (IOException canthappen) { - throw new IllegalStateException(canthappen); - } - final int result = this.currentLeft; - this.currentLen--; - representativeEdge.getTargetVertexId().set(result); - return representativeEdge; - - } - default: - final int result = ++this.currentLeft; - this.currentLen--; - representativeEdge.getTargetVertexId().set(result); - return representativeEdge; - } - } - - } +public class IntervalResidualEdges + extends ConfigurableOutEdges + implements ReuseObjectsOutEdges, Trimmable { + + /** Minimum interval length is equal to 2 */ + private static final int MIN_INTERVAL_LENGTH = 2; + + /** Maximum interval length is equal to 254 */ + private static final int MAX_INTERVAL_LENGTH = 254; + + /** Serialized Intervals and Residuals */ + private byte[] intervalsAndResiduals; + + /** Number of edges stored in compressed array */ + private int size; + + @Override + public void initialize(Iterable> edges) { + IntArrayList edgesList = new IntArrayList(); + for (Iterator> iter = edges.iterator(); iter + .hasNext();) { + edgesList.add(iter.next().getTargetVertexId().get()); + } + compress(Arrays.copyOfRange(edgesList.elements(), 0, edgesList.size())); + } + + @Override + public void initialize(int capacity) { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void initialize() { + size = 0; + intervalsAndResiduals = null; + } + + @Override + public void add(Edge edge) { + // Note that this is very expensive (decompresses all edges and recompresses + // them again). + IntArrayList edgesList = new IntArrayList(); + for (Iterator> iter = this.iterator(); iter + .hasNext();) { + edgesList.add(iter.next().getTargetVertexId().get()); + } + edgesList.add(edge.getTargetVertexId().get()); + compress(Arrays.copyOfRange(edgesList.elements(), 0, edgesList.size())); + } + + @Override + public void remove(IntWritable targetVertexId) { + // Note that this is very expensive (decompresses all edges and recompresses + // them again). + final int id = targetVertexId.get(); + initialize(Iterables.filter(this, + new Predicate>() { + @Override + public boolean apply(Edge edge) { + return edge.getTargetVertexId().get() != id; + } + })); + } + + @Override + public int size() { + return size; + } + + @Override + public Iterator> iterator() { + if (size == 0) { + return ImmutableSet.>of().iterator(); + } else { + return new IntervalResidualEdgeIterator(); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + size = in.readInt(); + int intervalResidualEdgesBytesUsed = in.readInt(); + if (intervalResidualEdgesBytesUsed > 0) { + // Only create a new buffer if the old one isn't big enough + if (intervalsAndResiduals == null || + intervalResidualEdgesBytesUsed > intervalsAndResiduals.length) { + intervalsAndResiduals = new byte[intervalResidualEdgesBytesUsed]; + } + in.readFully(intervalsAndResiduals, 0, intervalResidualEdgesBytesUsed); + } + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(size); + out.writeInt(intervalsAndResiduals.length); + if (intervalsAndResiduals.length > 0) { + out.write(intervalsAndResiduals, 0, intervalsAndResiduals.length); + } + } + + @Override + public void trim() { + /* Nothing to do */ + } + + /** + * Receives an integer array of successors and compresses them in the + * intervalsAndResiduals byte array + * + * @param edgesArray an integer array of successors + */ + private void compress(final int[] edgesArray) { + try { + ExtendedDataOutput eos = getConf().createExtendedDataOutput(); + IntArrayList left = new IntArrayList(); + IntArrayList len = new IntArrayList(); + IntArrayList residuals = new IntArrayList(); + // If we are to produce intervals, we first compute them. + int intervalCount; + try { + intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, + MAX_INTERVAL_LENGTH, left, len, residuals); + } catch (IllegalArgumentException e) { + // array was not sorted, sorting and retrying + Arrays.sort(edgesArray); + left = new IntArrayList(); + len = new IntArrayList(); + residuals = new IntArrayList(); + intervalCount = BVEdges.intervalize(edgesArray, MIN_INTERVAL_LENGTH, + MAX_INTERVAL_LENGTH, left, len, residuals); + } + + // We write out the intervals. + eos.writeInt(intervalCount); + for (int i = 0; i < intervalCount; i++) { + eos.writeInt(left.getInt(i)); + eos.write(len.getInt(i)); + } + final int[] residual = residuals.elements(); + final int residualCount = residuals.size(); + for (int i = 0; i < residualCount; i++) { + eos.writeInt(residual[i]); + } + intervalsAndResiduals = eos.toByteArray(); + size = edgesArray.length; + } catch (IOException e) { + throw new IllegalArgumentException(e); + } + } + + /** + * Iterator that reuses the same Edge object. + */ + private class IntervalResidualEdgeIterator + extends UnmodifiableIterator> { + /** Input for processing the bytes */ + private ExtendedDataInput extendedDataInput = getConf() + .createExtendedDataInput(intervalsAndResiduals, 0, + intervalsAndResiduals.length); + /** Representative edge object. */ + private final Edge representativeEdge = + EdgeFactory.create(new IntWritable()); + /** Current edge count */ + private int currentEdge = 0; + /** Current interval index */ + private int currentLeft; + /** Current interval length */ + private int currentLen = 0; + /** Interval counter initialized with interval size and counting down */ + private int intervalCount; + + /** Constructor */ + public IntervalResidualEdgeIterator() { + try { + intervalCount = extendedDataInput.readInt(); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean hasNext() { + return currentEdge < size; + } + + @Override + public Edge next() { + this.currentEdge++; + switch (this.currentLen) { + case 0: + switch (this.intervalCount) { + case 0: + try { + representativeEdge.getTargetVertexId() + .set(extendedDataInput.readInt()); + } catch (IOException canthappen) { + throw new IllegalStateException(canthappen); + } + return representativeEdge; + default: + try { + this.currentLeft = extendedDataInput.readInt(); + this.currentLen = extendedDataInput.readByte() & 0xff; + intervalCount--; + } catch (IOException canthappen) { + throw new IllegalStateException(canthappen); + } + final int result = this.currentLeft; + this.currentLen--; + representativeEdge.getTargetVertexId().set(result); + return representativeEdge; + + } + default: + final int result = ++this.currentLeft; + this.currentLen--; + representativeEdge.getTargetVertexId().set(result); + return representativeEdge; + } + } + + } } diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java index ec9a6b2dc..dfb752ef1 100644 --- a/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java +++ b/giraph-core/src/test/java/org/apache/giraph/edge/BVEdgesEdgesTest.java @@ -53,224 +53,230 @@ * removals are expensive. Parallel edges are not allowed. */ public class BVEdgesEdgesTest { - private static Edge createEdge(int id) { - return EdgeFactory.create(new IntWritable(id)); - } - - private static void assertEdges(BVEdges edges, int... expected) { - int index = 0; - for (Edge edge : (Iterable>) edges) { - Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); - index++; - } - Assert.assertEquals(expected.length, index); - } - - private BVEdges getEdges() { - GiraphConfiguration gc = new GiraphConfiguration(); - GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); - GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); - ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( - gc); - BVEdges ret = new BVEdges(); - ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); - return ret; - } - - @Test - public void testEdges() { - BVEdges edges = getEdges(); + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(BVEdges edges, int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private BVEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + BVEdges ret = new BVEdges(); + ret.setConf( + new ImmutableClassesGiraphConfiguration( + conf)); + return ret; + } + + @Test + public void testEdges() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + edges.remove(new IntWritable(2)); + assertEdges(edges, 1, 3, 4); + } - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + @Test + public void testInitialize() { + BVEdges edges = getEdges(); - edges.remove(new IntWritable(2)); - assertEdges(edges, 1, 3, 4); - } + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); - @Test - public void testInitialize() { - BVEdges edges = getEdges(); + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testInitializeUnsorted() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(3), createEdge(4), createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 3, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + BVEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + BVEdges edges = getEdges(); + + edges.initialize(); - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } - edges.initialize(); - assertEquals(0, edges.size()); - } + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } - @Test - public void testInitializeUnsorted() { - BVEdges edges = getEdges(); + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the + // implementation - List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), - createEdge(1)); - - edges.initialize(initialEdges); - assertEdges(edges, 1, 3, 4); + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); - edges.add(EdgeFactory.createReusable(new IntWritable(2))); - assertEdges(edges, 1, 2, 3, 4); + edges.write(tempBuffer); + tempBuffer.close(); - edges.initialize(); - assertEquals(0, edges.size()); - } + byte[] binary = arrayStream.toByteArray(); - @Test - public void testMutateEdges() { - BVEdges edges = getEdges(); + assertTrue("Serialized version should not be empty ", binary.length > 0); - edges.initialize(); + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } + assertEquals(5, edges.size()); - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; - // We should now have 5 edges - assertEquals(5, edges.size()); - // The edge ids should be all odd - for (Edge edge : (Iterable>) edges) { - assertEquals(1, edge.getTargetVertexId().get() % 2); - } - } + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } + + @Test(expected = IllegalArgumentException.class) + public void testParallelEdges() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(2), createEdge(2), createEdge(2)); + + edges.initialize(initialEdges); + } + + @Test + public void testEdgeValues() { + BVEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); + + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } + + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + BVEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + BVEdges edges = getEdges(); + + // Add 10 edges with id i, for i = 0..9 + // to create one large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Add a residual edge + edges.add(createEdge(23)); + + // Add 10 edges with id i, for i = 50..59 + // to create a second large interval + for (int i = 50; i < 60; ++i) { + edges.add(createEdge(i)); + } + + assertEquals(21, edges.size()); + + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of intervalResiduals byte array should be equal to 12 + ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor + .forClass(byte[].class); + Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), + Mockito.anyInt(), Mockito.anyInt()); + assertEquals(12, intervalsAndResidualsCaptop.getValue().length); + } - @Test - public void testSerialization() throws IOException { - BVEdges edges = getEdges(); - - edges.initialize(); - - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } - - // We should now have 5 edges - assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation - - ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); - DataOutputStream tempBuffer = new DataOutputStream(arrayStream); - - edges.write(tempBuffer); - tempBuffer.close(); - - byte[] binary = arrayStream.toByteArray(); - - assertTrue("Serialized version should not be empty ", binary.length > 0); - - edges = getEdges(); - edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); - - assertEquals(5, edges.size()); - - int[] ids = new int[] { 1, 3, 5, 7, 9 }; - int index = 0; - - for (Edge edge : (Iterable>) edges) { - assertEquals(ids[index], edge.getTargetVertexId().get()); - index++; - } - assertEquals(ids.length, index); - } - - @Test(expected = IllegalArgumentException.class) - public void testParallelEdges() { - BVEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), - createEdge(2)); - - edges.initialize(initialEdges); - } - - @Test - public void testEdgeValues() { - BVEdges edges = getEdges(); - Set testValues = new HashSet(); - testValues.add(0); - testValues.add(Integer.MAX_VALUE / 2); - testValues.add(Integer.MAX_VALUE); - - List> initialEdges = new ArrayList>(); - for (Integer id : testValues) { - initialEdges.add(createEdge(id)); - } - - edges.initialize(initialEdges); - - Iterator> edgeIt = edges.iterator(); - while (edgeIt.hasNext()) { - int value = edgeIt.next().getTargetVertexId().get(); - assertTrue("Unknown edge found " + value, testValues.remove(value)); - } - } - - @Test - public void testAddedSmallerValues() { - BVEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(100)); - - edges.initialize(initialEdges); - - for (int i = 0; i < 16; i++) { - edges.add(createEdge(i)); - } - - assertEquals(17, edges.size()); - } - - @Test - public void testCompressedSize() throws IOException { - BVEdges edges = getEdges(); - - // Add 10 edges with id i, for i = 0..9 - // to create one large interval - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Add a residual edge - edges.add(createEdge(23)); - - // Add 10 edges with id i, for i = 50..59 - // to create a second large interval - for (int i = 50; i < 60; ++i) { - edges.add(createEdge(i)); - } - - assertEquals(21, edges.size()); - - DataOutput dataOutput = Mockito.mock(DataOutput.class); - edges.write(dataOutput); - - // size of intervalResiduals byte array should be equal to 12 - ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor.forClass(byte[].class); - Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); - assertEquals(12, intervalsAndResidualsCaptop.getValue().length); - } - } diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java index 925db118a..ba5b47a31 100644 --- a/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java +++ b/giraph-core/src/test/java/org/apache/giraph/edge/IndexedBitArrayEdgesTest.java @@ -45,223 +45,227 @@ import static org.junit.Assert.assertTrue; public class IndexedBitArrayEdgesTest { - private static Edge createEdge(int id) { - return EdgeFactory.create(new IntWritable(id)); - } - - private static void assertEdges(IndexedBitArrayEdges edges, int... expected) { - int index = 0; - for (Edge edge : (Iterable>) edges) { - Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); - index++; - } - Assert.assertEquals(expected.length, index); - } - - private IndexedBitArrayEdges getEdges() { - GiraphConfiguration gc = new GiraphConfiguration(); - GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); - GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); - ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( - gc); - IndexedBitArrayEdges ret = new IndexedBitArrayEdges(); - ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); - return ret; - } - - @Test - public void testEdges() { - IndexedBitArrayEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(IndexedBitArrayEdges edges, int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private IndexedBitArrayEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + IndexedBitArrayEdges ret = new IndexedBitArrayEdges(); + ret.setConf( + new ImmutableClassesGiraphConfiguration( + conf)); + return ret; + } + + @Test + public void testEdges() { + IndexedBitArrayEdges edges = getEdges(); - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - edges.remove(new IntWritable(2)); - assertEdges(edges, 1, 3, 4); - } + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array - @Test - public void testInitialize() { - IndexedBitArrayEdges edges = getEdges(); + edges.remove(new IntWritable(2)); + assertEdges(edges, 1, 3, 4); + } - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + @Test + public void testInitialize() { + IndexedBitArrayEdges edges = getEdges(); - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - edges.initialize(); - assertEquals(0, edges.size()); - } + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); - @Test - public void testInitializeUnsorted() { - IndexedBitArrayEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), - createEdge(1)); - - edges.initialize(initialEdges); - assertEdges(edges, 1, 3, 4); - - edges.add(EdgeFactory.createReusable(new IntWritable(2))); - - assertEdges(edges, 1, 2, 3, 4); - - edges.initialize(); - assertEquals(0, edges.size()); - } - - @Test - public void testMutateEdges() { - IndexedBitArrayEdges edges = getEdges(); - - edges.initialize(); - - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } + edges.initialize(); + assertEquals(0, edges.size()); + } - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } - - // We should now have 5 edges - assertEquals(5, edges.size()); - // The edge ids should be all odd - for (Edge edge : (Iterable>) edges) { - assertEquals(1, edge.getTargetVertexId().get() % 2); - } - } - - @Test - public void testSerialization() throws IOException { - IndexedBitArrayEdges edges = getEdges(); - - edges.initialize(); - - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } - - // We should now have 5 edges - assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation - - ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); - DataOutputStream tempBuffer = new DataOutputStream(arrayStream); - - edges.write(tempBuffer); - tempBuffer.close(); - - byte[] binary = arrayStream.toByteArray(); - - assertTrue("Serialized version should not be empty ", binary.length > 0); - - edges = getEdges(); - edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); - - assertEquals(5, edges.size()); - - int[] ids = new int[] { 1, 3, 5, 7, 9 }; - int index = 0; - - for (Edge edge : (Iterable>) edges) { - assertEquals(ids[index], edge.getTargetVertexId().get()); - index++; - } - assertEquals(ids.length, index); - } - - @Test - public void testParallelEdges() { - IndexedBitArrayEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), - createEdge(2)); - - edges.initialize(initialEdges); - - assertEquals(1, edges.size()); - } - - @Test - public void testEdgeValues() { - IndexedBitArrayEdges edges = getEdges(); - Set testValues = new HashSet(); - testValues.add(0); - testValues.add(Integer.MAX_VALUE / 2); - testValues.add(Integer.MAX_VALUE); - - List> initialEdges = new ArrayList>(); - for (Integer id : testValues) { - initialEdges.add(createEdge(id)); - } - - edges.initialize(initialEdges); - - Iterator> edgeIt = edges.iterator(); - while (edgeIt.hasNext()) { - int value = edgeIt.next().getTargetVertexId().get(); - assertTrue("Unknown edge found " + value, testValues.remove(value)); - } - } - - @Test - public void testAddedSmallerValues() { - IndexedBitArrayEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(100)); - - edges.initialize(initialEdges); - - for (int i = 0; i < 16; i++) { - edges.add(createEdge(i)); - } - - assertEquals(17, edges.size()); - } - - @Test - public void testCompressedSize() throws IOException { - IndexedBitArrayEdges edges = getEdges(); - - edges.initialize(); - - // Add 10 edges with id i, for i = 0..9 - // to create on large interval - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Add a residual edge - edges.add(createEdge(23)); - - assertEquals(11, edges.size()); - - DataOutput dataOutput = Mockito.mock(DataOutput.class); - edges.write(dataOutput); - - // size of serializedEdges byte array should be equal to 15: - // 5 (bucket 0: 0-7) + 5 (bucket 1: 8-10) + 5 (bucket 2: 23) - ArgumentCaptor serializedEdgesCaptop = ArgumentCaptor.forClass(byte[].class); - Mockito.verify(dataOutput).write(serializedEdgesCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); - assertEquals(15, serializedEdgesCaptop.getValue().length); - } + @Test + public void testInitializeUnsorted() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(3), createEdge(4), createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 3, 4); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the + // implementation + + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); + + edges.write(tempBuffer); + tempBuffer.close(); + + byte[] binary = arrayStream.toByteArray(); + + assertTrue("Serialized version should not be empty ", binary.length > 0); + + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); + + assertEquals(5, edges.size()); + + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; + + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } + + @Test(expected = IllegalArgumentException.class) + public void testParallelEdges() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(2), createEdge(2), createEdge(2)); + + edges.initialize(initialEdges); + } + + @Test + public void testEdgeValues() { + IndexedBitArrayEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); + + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } + + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + IndexedBitArrayEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + IndexedBitArrayEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + // to create on large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Add a residual edge + edges.add(createEdge(23)); + + assertEquals(11, edges.size()); + + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of serializedEdges byte array should be equal to 15: + // 5 (bucket 0: 0-7) + 5 (bucket 1: 8-10) + 5 (bucket 2: 23) + ArgumentCaptor serializedEdgesCaptop = ArgumentCaptor + .forClass(byte[].class); + Mockito.verify(dataOutput).write(serializedEdgesCaptop.capture(), + Mockito.anyInt(), Mockito.anyInt()); + assertEquals(15, serializedEdgesCaptop.getValue().length); + } } diff --git a/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java b/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java index 54eeb4440..e9f17ee57 100644 --- a/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java +++ b/giraph-core/src/test/java/org/apache/giraph/edge/IntervalResidualEdgesTest.java @@ -45,218 +45,227 @@ import static org.junit.Assert.assertTrue; public class IntervalResidualEdgesTest { - private static Edge createEdge(int id) { - return EdgeFactory.create(new IntWritable(id)); - } - - private static void assertEdges(IntervalResidualEdges edges, int... expected) { - int index = 0; - for (Edge edge : (Iterable>) edges) { - Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); - index++; - } - Assert.assertEquals(expected.length, index); - } - - private IntervalResidualEdges getEdges() { - GiraphConfiguration gc = new GiraphConfiguration(); - GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); - GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); - ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( - gc); - IntervalResidualEdges ret = new IntervalResidualEdges(); - ret.setConf(new ImmutableClassesGiraphConfiguration(conf)); - return ret; - } - - @Test - public void testEdges() { - IntervalResidualEdges edges = getEdges(); + private static Edge createEdge(int id) { + return EdgeFactory.create(new IntWritable(id)); + } + + private static void assertEdges(IntervalResidualEdges edges, + int... expected) { + int index = 0; + for (Edge edge : (Iterable>) edges) { + Assert.assertEquals(expected[index], edge.getTargetVertexId().get()); + index++; + } + Assert.assertEquals(expected.length, index); + } + + private IntervalResidualEdges getEdges() { + GiraphConfiguration gc = new GiraphConfiguration(); + GiraphConstants.VERTEX_ID_CLASS.set(gc, IntWritable.class); + GiraphConstants.EDGE_VALUE_CLASS.set(gc, NullWritable.class); + ImmutableClassesGiraphConfiguration conf = new ImmutableClassesGiraphConfiguration( + gc); + IntervalResidualEdges ret = new IntervalResidualEdges(); + ret.setConf( + new ImmutableClassesGiraphConfiguration( + conf)); + return ret; + } + + @Test + public void testEdges() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); + + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + edges.remove(new IntWritable(2)); + assertEdges(edges, 3, 4, 1); + } + + @Test + public void testInitialize() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(1), createEdge(2), createEdge(4)); - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); // order matters, it's an array + edges.initialize(initialEdges); + assertEdges(edges, 1, 2, 4); - edges.remove(new IntWritable(2)); - assertEdges(edges, 3, 4, 1); - } + edges.add(EdgeFactory.createReusable(new IntWritable(3))); + assertEdges(edges, 1, 2, 3, 4); - @Test - public void testInitialize() { - IntervalResidualEdges edges = getEdges(); + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testInitializeUnsorted() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(3), createEdge(4), createEdge(1)); + + edges.initialize(initialEdges); + assertEdges(edges, 3, 4, 1); + + edges.add(EdgeFactory.createReusable(new IntWritable(2))); + assertEdges(edges, 1, 2, 3, 4); + + edges.initialize(); + assertEquals(0, edges.size()); + } + + @Test + public void testMutateEdges() { + IntervalResidualEdges edges = getEdges(); + + edges.initialize(); + + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } + + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } + + // We should now have 5 edges + assertEquals(5, edges.size()); + // The edge ids should be all odd + for (Edge edge : (Iterable>) edges) { + assertEquals(1, edge.getTargetVertexId().get() % 2); + } + } + + @Test + public void testSerialization() throws IOException { + IntervalResidualEdges edges = getEdges(); + + edges.initialize(); - List> initialEdges = Lists.newArrayList(createEdge(1), createEdge(2), - createEdge(4)); + // Add 10 edges with id i, for i = 0..9 + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } - edges.initialize(initialEdges); - assertEdges(edges, 1, 2, 4); + // Remove edges with even id + for (int i = 0; i < 10; i += 2) { + edges.remove(new IntWritable(i)); + } - edges.add(EdgeFactory.createReusable(new IntWritable(3))); - assertEdges(edges, 1, 2, 3, 4); + // We should now have 5 edges + assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the + // implementation - edges.initialize(); - assertEquals(0, edges.size()); - } + ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); + DataOutputStream tempBuffer = new DataOutputStream(arrayStream); - @Test - public void testInitializeUnsorted() { - IntervalResidualEdges edges = getEdges(); + edges.write(tempBuffer); + tempBuffer.close(); - List> initialEdges = Lists.newArrayList(createEdge(3), createEdge(4), - createEdge(1)); - - edges.initialize(initialEdges); - assertEdges(edges, 3, 4, 1); + byte[] binary = arrayStream.toByteArray(); - edges.add(EdgeFactory.createReusable(new IntWritable(2))); - assertEdges(edges, 1, 2, 3, 4); + assertTrue("Serialized version should not be empty ", binary.length > 0); - edges.initialize(); - assertEquals(0, edges.size()); - } + edges = getEdges(); + edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); - @Test - public void testMutateEdges() { - IntervalResidualEdges edges = getEdges(); + assertEquals(5, edges.size()); - edges.initialize(); + int[] ids = new int[] { 1, 3, 5, 7, 9 }; + int index = 0; - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } + for (Edge edge : (Iterable>) edges) { + assertEquals(ids[index], edge.getTargetVertexId().get()); + index++; + } + assertEquals(ids.length, index); + } - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } + @Test(expected = IllegalArgumentException.class) + public void testParallelEdges() { + IntervalResidualEdges edges = getEdges(); - // We should now have 5 edges - assertEquals(5, edges.size()); - // The edge ids should be all odd - for (Edge edge : (Iterable>) edges) { - assertEquals(1, edge.getTargetVertexId().get() % 2); - } - } + List> initialEdges = Lists + .newArrayList(createEdge(2), createEdge(2), createEdge(2)); - @Test - public void testSerialization() throws IOException { - IntervalResidualEdges edges = getEdges(); - - edges.initialize(); - - // Add 10 edges with id i, for i = 0..9 - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Remove edges with even id - for (int i = 0; i < 10; i += 2) { - edges.remove(new IntWritable(i)); - } - - // We should now have 5 edges - assertEdges(edges, 1, 3, 5, 7, 9); // id order matter because of the implementation - - ByteArrayOutputStream arrayStream = new ByteArrayOutputStream(); - DataOutputStream tempBuffer = new DataOutputStream(arrayStream); + edges.initialize(initialEdges); + } - edges.write(tempBuffer); - tempBuffer.close(); + @Test + public void testEdgeValues() { + IntervalResidualEdges edges = getEdges(); + Set testValues = new HashSet(); + testValues.add(0); + testValues.add(Integer.MAX_VALUE / 2); + testValues.add(Integer.MAX_VALUE); - byte[] binary = arrayStream.toByteArray(); + List> initialEdges = new ArrayList>(); + for (Integer id : testValues) { + initialEdges.add(createEdge(id)); + } - assertTrue("Serialized version should not be empty ", binary.length > 0); + edges.initialize(initialEdges); + + Iterator> edgeIt = edges.iterator(); + while (edgeIt.hasNext()) { + int value = edgeIt.next().getTargetVertexId().get(); + assertTrue("Unknown edge found " + value, testValues.remove(value)); + } + } + + @Test + public void testAddedSmallerValues() { + IntervalResidualEdges edges = getEdges(); + + List> initialEdges = Lists + .newArrayList(createEdge(100)); + + edges.initialize(initialEdges); + + for (int i = 0; i < 16; i++) { + edges.add(createEdge(i)); + } + + assertEquals(17, edges.size()); + } + + @Test + public void testCompressedSize() throws IOException { + IntervalResidualEdges edges = getEdges(); - edges = getEdges(); - edges.readFields(new DataInputStream(new ByteArrayInputStream(binary))); + // Add 10 edges with id i, for i = 0..9 + // to create on large interval + for (int i = 0; i < 10; ++i) { + edges.add(createEdge(i)); + } - assertEquals(5, edges.size()); + // Add a residual edge + edges.add(createEdge(23)); - int[] ids = new int[] { 1, 3, 5, 7, 9 }; - int index = 0; + assertEquals(11, edges.size()); - for (Edge edge : (Iterable>) edges) { - assertEquals(ids[index], edge.getTargetVertexId().get()); - index++; - } - assertEquals(ids.length, index); - } - - @Test(expected = IllegalArgumentException.class) - public void testParallelEdges() { - IntervalResidualEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(2), createEdge(2), - createEdge(2)); - - edges.initialize(initialEdges); - } - - @Test - public void testEdgeValues() { - IntervalResidualEdges edges = getEdges(); - Set testValues = new HashSet(); - testValues.add(0); - testValues.add(Integer.MAX_VALUE / 2); - testValues.add(Integer.MAX_VALUE); - - List> initialEdges = new ArrayList>(); - for (Integer id : testValues) { - initialEdges.add(createEdge(id)); - } - - edges.initialize(initialEdges); - - Iterator> edgeIt = edges.iterator(); - while (edgeIt.hasNext()) { - int value = edgeIt.next().getTargetVertexId().get(); - assertTrue("Unknown edge found " + value, testValues.remove(value)); - } - } - - @Test - public void testAddedSmallerValues() { - IntervalResidualEdges edges = getEdges(); - - List> initialEdges = Lists.newArrayList(createEdge(100)); - - edges.initialize(initialEdges); - - for (int i = 0; i < 16; i++) { - edges.add(createEdge(i)); - } - - assertEquals(17, edges.size()); - } - - @Test - public void testCompressedSize() throws IOException { - IntervalResidualEdges edges = getEdges(); - - // Add 10 edges with id i, for i = 0..9 - // to create on large interval - for (int i = 0; i < 10; ++i) { - edges.add(createEdge(i)); - } - - // Add a residual edge - edges.add(createEdge(23)); - - assertEquals(11, edges.size()); - - DataOutput dataOutput = Mockito.mock(DataOutput.class); - edges.write(dataOutput); - - // size of intervalResiduals byte array should be equal to 13: - // 4 (intervals' size) + 4 (index of 1st interval) + 1 (length of 1st interval) + 4 (residual) - ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor.forClass(byte[].class); - Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), Mockito.anyInt(), Mockito.anyInt()); - assertEquals(13, intervalsAndResidualsCaptop.getValue().length); - } + DataOutput dataOutput = Mockito.mock(DataOutput.class); + edges.write(dataOutput); + + // size of intervalResiduals byte array should be equal to 13: + // 4 (intervals' size) + 4 (index of 1st interval) + 1 (length of 1st + // interval) + // + 4 (residual) + ArgumentCaptor intervalsAndResidualsCaptop = ArgumentCaptor + .forClass(byte[].class); + Mockito.verify(dataOutput).write(intervalsAndResidualsCaptop.capture(), + Mockito.anyInt(), Mockito.anyInt()); + assertEquals(13, intervalsAndResidualsCaptop.getValue().length); + } } diff --git a/pom.xml b/pom.xml index 6369634ba..28d1cfcd2 100644 --- a/pom.xml +++ b/pom.xml @@ -328,9 +328,10 @@ under the License. 2.1 3.1 3.4 + 2.4.2 0.14.0 2.1.2 - 6.5.4 + 8.1.0 2.0.2 18.0 0.94.16 @@ -1782,6 +1783,37 @@ under the License. base64 ${dep.base64.version} + + it.unimi.dsi + dsiutils + ${dep.dsiutils.version} + + + org.slf4j + slf4j-api + + + commons-io + commons-io + + + com.google.guava + guava + + + commons-collections + commons-collections + + + commons-configuration + commons-configuration + + + ch.qos.logback + logback-classic + + + it.unimi.dsi fastutil @@ -1791,6 +1823,44 @@ under the License. it.unimi.dsi webgraph ${dep.webgraph.version} + + + org.slf4j + slf4j-api + + + commons-io + commons-io + + + com.google.guava + guava + + + org.apache.commons + commons-lang3 + + + + commons-collections + commons-collections + + + commons-configuration + commons-configuration + + + commons-lang + commons-lang + + + ch.qos.logback + logback-classic + + org.apache.giraph