This is an automated email from the ASF dual-hosted git repository.
ctubbsii pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git
The following commit(s) were added to refs/heads/main by this push:
new 00da7799ae Collapse MutableByteSequence into ArrayByteSequence (#4745)
00da7799ae is described below
commit 00da7799ae0ec2d27a5312e7c2646e38a6112ba7
Author: John K <[email protected]>
AuthorDate: Sun Jul 28 12:59:47 2024 -0400
Collapse MutableByteSequence into ArrayByteSequence (#4745)
* Delete MutableByteSequence
* Replace uses with ArrayByteSequence
* Add bound checks to array modification methods from Objects
and update tests accordingly
* Use reset method in more places with final variables
* Simplify RelativeKey readPrefix and read methods using reset
This fixes #4738
---
.../accumulo/core/data/ArrayByteSequence.java | 20 ++----
.../org/apache/accumulo/core/file/rfile/RFile.java | 7 +-
.../accumulo/core/file/rfile/RelativeKey.java | 84 +++++++++++-----------
.../accumulo/core/util/LocalityGroupUtil.java | 6 +-
.../accumulo/core/util/MutableByteSequence.java | 47 ------------
.../accumulo/core/data/ArrayByteSequenceTest.java | 18 ++---
.../accumulo/core/file/rfile/RelativeKeyTest.java | 9 ++-
7 files changed, 66 insertions(+), 125 deletions(-)
diff --git
a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
index 6c57ba7a3e..608804a0f6 100644
--- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
+++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java
@@ -23,6 +23,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.Arrays;
+import java.util.Objects;
import org.apache.accumulo.core.util.ByteBufferUtil;
@@ -62,10 +63,7 @@ public class ArrayByteSequence extends ByteSequence
implements Serializable {
*/
public ArrayByteSequence(byte[] data, int offset, int length) {
- if (offset < 0 || offset > data.length || length < 0 || (offset + length)
> data.length) {
- throw new IllegalArgumentException(" Bad offset and/or length
data.length = " + data.length
- + " offset = " + offset + " length = " + length);
- }
+ Objects.checkFromIndexSize(offset, length, data.length);
this.data = data;
this.offset = offset;
@@ -123,13 +121,7 @@ public class ArrayByteSequence extends ByteSequence
implements Serializable {
@Override
public byte byteAt(int i) {
- if (i < 0) {
- throw new IllegalArgumentException("i < 0, " + i);
- }
-
- if (i >= length) {
- throw new IllegalArgumentException("i >= length, " + i + " >= " +
length);
- }
+ Objects.checkIndex(i, length);
return data[offset + i];
}
@@ -160,6 +152,7 @@ public class ArrayByteSequence extends ByteSequence
implements Serializable {
* @since 3.1.0
*/
public void reset(byte[] data, int offset, int length) {
+ Objects.checkFromIndexSize(offset, length, data.length);
this.data = data;
this.offset = offset;
this.length = length;
@@ -168,10 +161,7 @@ public class ArrayByteSequence extends ByteSequence
implements Serializable {
@Override
public ByteSequence subSequence(int start, int end) {
- if (start > end || start < 0 || end > length) {
- throw new IllegalArgumentException("Bad start and/end start = " + start
+ " end=" + end
- + " offset=" + offset + " length=" + length);
- }
+ Objects.checkFromToIndex(start, end, length);
return new ArrayByteSequence(data, offset + start, end - start);
}
diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
index 00d01aad9f..dfa164e9b5 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java
@@ -78,7 +78,6 @@ import
org.apache.accumulo.core.iteratorsImpl.system.LocalityGroupIterator.Local
import org.apache.accumulo.core.metadata.TabletFile;
import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl;
import org.apache.accumulo.core.util.LocalityGroupUtil;
-import org.apache.accumulo.core.util.MutableByteSequence;
import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
@@ -993,7 +992,7 @@ public class RFile {
// causing the build of an index... doing this could slow down some
use cases and
// and speed up others.
- MutableByteSequence valbs = new MutableByteSequence(new byte[64], 0,
0);
+ final var valbs = new ArrayByteSequence(new byte[64], 0, 0);
SkippR skippr =
RelativeKey.fastSkip(currBlock, startKey, valbs, prevKey,
getTopKey(), entriesLeft);
if (skippr.skipped > 0) {
@@ -1055,7 +1054,7 @@ public class RFile {
hasTop = true;
}
- MutableByteSequence valbs = new MutableByteSequence(new byte[64], 0,
0);
+ final var valbs = new ArrayByteSequence(new byte[64], 0, 0);
Key currKey = null;
@@ -1072,7 +1071,7 @@ public class RFile {
val = new Value();
val.readFields(currBlock);
- valbs = new MutableByteSequence(val.get(), 0, val.getSize());
+ valbs.reset(val.get(), 0, val.getSize());
// just consumed one key from the input stream, so subtract
one from entries left
entriesLeft = bie.getEntriesLeft() - 1;
diff --git
a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
index 46488d84cd..7340ece246 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RelativeKey.java
@@ -23,9 +23,9 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.function.Supplier;
+import org.apache.accumulo.core.data.ArrayByteSequence;
import org.apache.accumulo.core.data.ByteSequence;
import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.util.MutableByteSequence;
import org.apache.accumulo.core.util.UnsynchronizedBuffer;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableUtils;
@@ -215,12 +215,12 @@ public class RelativeKey implements Writable {
}
}
- public static SkippR fastSkip(DataInput in, Key seekKey, MutableByteSequence
value, Key prevKey,
+ public static SkippR fastSkip(DataInput in, Key seekKey, ArrayByteSequence
value, Key prevKey,
Key currKey, int entriesLeft) throws IOException {
// this method mostly avoids object allocation and only does compares when
the row changes
- MutableByteSequence row, cf, cq, cv;
- MutableByteSequence prow, pcf, pcq, pcv;
+ ArrayByteSequence row, cf, cq, cv;
+ ArrayByteSequence prow, pcf, pcq, pcv;
ByteSequence stopRow = seekKey.getRowData();
ByteSequence stopCF = seekKey.getColumnFamilyData();
@@ -234,16 +234,16 @@ public class RelativeKey implements Writable {
if (currKey != null) {
- prow = new MutableByteSequence(currKey.getRowData());
- pcf = new MutableByteSequence(currKey.getColumnFamilyData());
- pcq = new MutableByteSequence(currKey.getColumnQualifierData());
- pcv = new MutableByteSequence(currKey.getColumnVisibilityData());
+ prow = new ArrayByteSequence(currKey.getRowData());
+ pcf = new ArrayByteSequence(currKey.getColumnFamilyData());
+ pcq = new ArrayByteSequence(currKey.getColumnQualifierData());
+ pcv = new ArrayByteSequence(currKey.getColumnVisibilityData());
pts = currKey.getTimestamp();
- row = new MutableByteSequence(currKey.getRowData());
- cf = new MutableByteSequence(currKey.getColumnFamilyData());
- cq = new MutableByteSequence(currKey.getColumnQualifierData());
- cv = new MutableByteSequence(currKey.getColumnVisibilityData());
+ row = new ArrayByteSequence(currKey.getRowData());
+ cf = new ArrayByteSequence(currKey.getColumnFamilyData());
+ cq = new ArrayByteSequence(currKey.getColumnQualifierData());
+ cv = new ArrayByteSequence(currKey.getColumnVisibilityData());
ts = currKey.getTimestamp();
rowCmp = row.compareTo(stopRow);
@@ -273,15 +273,15 @@ public class RelativeKey implements Writable {
}
} else {
- row = new MutableByteSequence(new byte[64], 0, 0);
- cf = new MutableByteSequence(new byte[64], 0, 0);
- cq = new MutableByteSequence(new byte[64], 0, 0);
- cv = new MutableByteSequence(new byte[64], 0, 0);
-
- prow = new MutableByteSequence(new byte[64], 0, 0);
- pcf = new MutableByteSequence(new byte[64], 0, 0);
- pcq = new MutableByteSequence(new byte[64], 0, 0);
- pcv = new MutableByteSequence(new byte[64], 0, 0);
+ row = new ArrayByteSequence(new byte[64], 0, 0);
+ cf = new ArrayByteSequence(new byte[64], 0, 0);
+ cq = new ArrayByteSequence(new byte[64], 0, 0);
+ cv = new ArrayByteSequence(new byte[64], 0, 0);
+
+ prow = new ArrayByteSequence(new byte[64], 0, 0);
+ pcf = new ArrayByteSequence(new byte[64], 0, 0);
+ pcq = new ArrayByteSequence(new byte[64], 0, 0);
+ pcv = new ArrayByteSequence(new byte[64], 0, 0);
}
byte fieldsSame = -1;
@@ -304,7 +304,7 @@ public class RelativeKey implements Writable {
if ((fieldsSame & ROW_SAME) != ROW_SAME) {
- MutableByteSequence tmp = prow;
+ ArrayByteSequence tmp = prow;
prow = row;
row = tmp;
@@ -321,7 +321,7 @@ public class RelativeKey implements Writable {
if ((fieldsSame & CF_SAME) != CF_SAME) {
- MutableByteSequence tmp = pcf;
+ ArrayByteSequence tmp = pcf;
pcf = cf;
cf = tmp;
@@ -337,7 +337,7 @@ public class RelativeKey implements Writable {
if ((fieldsSame & CQ_SAME) != CQ_SAME) {
- MutableByteSequence tmp = pcq;
+ ArrayByteSequence tmp = pcq;
pcq = cq;
cq = tmp;
@@ -353,7 +353,7 @@ public class RelativeKey implements Writable {
if ((fieldsSame & CV_SAME) != CV_SAME) {
- MutableByteSequence tmp = pcv;
+ ArrayByteSequence tmp = pcv;
pcv = cv;
cv = tmp;
@@ -397,7 +397,7 @@ public class RelativeKey implements Writable {
}
if (count > 1) {
- MutableByteSequence trow, tcf, tcq, tcv;
+ ArrayByteSequence trow, tcf, tcq, tcv;
long tts;
// when the current keys field is same as the last, then
@@ -432,24 +432,24 @@ public class RelativeKey implements Writable {
return new SkippR(result, count, newPrevKey);
}
- private static void read(DataInput in, MutableByteSequence mbseq) throws
IOException {
+ private static void read(DataInput in, ArrayByteSequence mbseq) throws
IOException {
int len = WritableUtils.readVInt(in);
read(in, mbseq, len);
}
- private static void readValue(DataInput in, MutableByteSequence mbseq)
throws IOException {
+ private static void readValue(DataInput in, ArrayByteSequence mbseq) throws
IOException {
int len = in.readInt();
read(in, mbseq, len);
}
- private static void read(DataInput in, MutableByteSequence mbseqDestination,
int len)
+ private static void read(DataInput in, ArrayByteSequence mbseqDestination,
int len)
throws IOException {
- if (mbseqDestination.getBackingArray().length < len) {
- mbseqDestination.setArray(new
byte[UnsynchronizedBuffer.nextArraySize(len)], 0, 0);
+ byte[] buf = mbseqDestination.getBackingArray();
+ if (buf.length < len) {
+ buf = new byte[UnsynchronizedBuffer.nextArraySize(len)];
}
-
- in.readFully(mbseqDestination.getBackingArray(), 0, len);
- mbseqDestination.setLength(len);
+ in.readFully(buf, 0, len);
+ mbseqDestination.reset(buf, 0, len);
}
private static byte[] readPrefix(DataInput in, ByteSequence prefixSource)
throws IOException {
@@ -467,24 +467,24 @@ public class RelativeKey implements Writable {
return data;
}
- private static void readPrefix(DataInput in, MutableByteSequence dest,
ByteSequence prefixSource)
+ private static void readPrefix(DataInput in, ArrayByteSequence dest,
ByteSequence prefixSource)
throws IOException {
int prefixLen = WritableUtils.readVInt(in);
int remainingLen = WritableUtils.readVInt(in);
int len = prefixLen + remainingLen;
- if (dest.getBackingArray().length < len) {
- dest.setArray(new byte[UnsynchronizedBuffer.nextArraySize(len)], 0, 0);
+ byte[] buf = dest.getBackingArray();
+ if (buf.length < len) {
+ buf = new byte[UnsynchronizedBuffer.nextArraySize(len)];
}
if (prefixSource.isBackedByArray()) {
- System.arraycopy(prefixSource.getBackingArray(), prefixSource.offset(),
- dest.getBackingArray(), 0, prefixLen);
+ System.arraycopy(prefixSource.getBackingArray(), prefixSource.offset(),
buf, 0, prefixLen);
} else {
byte[] prefixArray = prefixSource.toArray();
- System.arraycopy(prefixArray, 0, dest.getBackingArray(), 0, prefixLen);
+ System.arraycopy(prefixArray, 0, buf, 0, prefixLen);
}
// read remaining
- in.readFully(dest.getBackingArray(), prefixLen, remainingLen);
- dest.setLength(len);
+ in.readFully(buf, prefixLen, remainingLen);
+ dest.reset(buf, 0, len);
}
private static byte[] read(DataInput in) throws IOException {
diff --git
a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
index 3d1baceaa6..ee27c22091 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/LocalityGroupUtil.java
@@ -306,7 +306,7 @@ public class LocalityGroupUtil {
public void partition(List<Mutation> mutations,
PreAllocatedArray<List<Mutation>> partitionedMutations) {
- MutableByteSequence mbs = new MutableByteSequence(new byte[0], 0, 0);
+ final var mbs = new ArrayByteSequence(new byte[0], 0, 0);
PreAllocatedArray<List<ColumnUpdate>> parts = new
PreAllocatedArray<>(groups.length + 1);
@@ -351,8 +351,8 @@ public class LocalityGroupUtil {
}
}
- private Integer getLgid(MutableByteSequence mbs, ColumnUpdate cu) {
- mbs.setArray(cu.getColumnFamily(), 0, cu.getColumnFamily().length);
+ private Integer getLgid(ArrayByteSequence mbs, ColumnUpdate cu) {
+ mbs.reset(cu.getColumnFamily(), 0, cu.getColumnFamily().length);
Integer lgid = colfamToLgidMap.get(mbs);
if (lgid == null) {
lgid = groups.length;
diff --git
a/core/src/main/java/org/apache/accumulo/core/util/MutableByteSequence.java
b/core/src/main/java/org/apache/accumulo/core/util/MutableByteSequence.java
deleted file mode 100644
index 83abed661a..0000000000
--- a/core/src/main/java/org/apache/accumulo/core/util/MutableByteSequence.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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
- *
- * https://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.accumulo.core.util;
-
-import org.apache.accumulo.core.data.ArrayByteSequence;
-import org.apache.accumulo.core.data.ByteSequence;
-
-public class MutableByteSequence extends ArrayByteSequence {
- private static final long serialVersionUID = 1L;
-
- public MutableByteSequence(byte[] data, int offset, int length) {
- super(data, offset, length);
- }
-
- public MutableByteSequence(ByteSequence bs) {
- super(new byte[Math.max(64, bs.length())]);
- System.arraycopy(bs.getBackingArray(), bs.offset(), data, 0, bs.length());
- this.length = bs.length();
- this.offset = 0;
- }
-
- public void setArray(byte[] data, int offset, int len) {
- this.data = data;
- this.offset = offset;
- this.length = len;
- }
-
- public void setLength(int len) {
- this.length = len;
- }
-}
diff --git
a/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
b/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
index 47d4786a80..a515ca40ac 100644
---
a/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
+++
b/core/src/test/java/org/apache/accumulo/core/data/ArrayByteSequenceTest.java
@@ -39,33 +39,33 @@ public class ArrayByteSequenceTest {
@Test
public void testInvalidByteBufferBounds0() {
- assertThrows(IllegalArgumentException.class, () -> abs = new
ArrayByteSequence(data, -1, 0));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs = new
ArrayByteSequence(data, -1, 0));
}
@Test
public void testInvalidByteBufferBounds1() {
- assertThrows(IllegalArgumentException.class,
+ assertThrows(IndexOutOfBoundsException.class,
() -> abs = new ArrayByteSequence(data, data.length + 1, 0));
}
@Test
public void testInvalidByteBufferBounds2() {
- assertThrows(IllegalArgumentException.class, () -> abs = new
ArrayByteSequence(data, 0, -1));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs = new
ArrayByteSequence(data, 0, -1));
}
@Test
public void testInvalidByteBufferBounds3() {
- assertThrows(IllegalArgumentException.class, () -> abs = new
ArrayByteSequence(data, 6, 2));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs = new
ArrayByteSequence(data, 6, 2));
}
@Test
public void testInvalidByteAt0() {
- assertThrows(IllegalArgumentException.class, () -> abs.byteAt(-1));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs.byteAt(-1));
}
@Test
public void testInvalidByteAt1() {
- assertThrows(IllegalArgumentException.class, () ->
abs.byteAt(data.length));
+ assertThrows(IndexOutOfBoundsException.class, () ->
abs.byteAt(data.length));
}
@Test
@@ -76,17 +76,17 @@ public class ArrayByteSequenceTest {
@Test
public void testInvalidSubsequence0() {
- assertThrows(IllegalArgumentException.class, () -> abs.subSequence(5, 1));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs.subSequence(5, 1));
}
@Test
public void testInvalidSubsequence1() {
- assertThrows(IllegalArgumentException.class, () -> abs.subSequence(-1, 1));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs.subSequence(-1,
1));
}
@Test
public void testInvalidSubsequence3() {
- assertThrows(IllegalArgumentException.class, () -> abs.subSequence(0, 10));
+ assertThrows(IndexOutOfBoundsException.class, () -> abs.subSequence(0,
10));
}
@Test
diff --git
a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
index ad564dce54..a0b98d78b8 100644
---
a/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
+++
b/core/src/test/java/org/apache/accumulo/core/file/rfile/RelativeKeyTest.java
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.data.Key;
import org.apache.accumulo.core.data.PartialKey;
import org.apache.accumulo.core.data.Value;
import org.apache.accumulo.core.file.rfile.RelativeKey.SkippR;
-import org.apache.accumulo.core.util.MutableByteSequence;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -158,7 +157,7 @@ public class RelativeKeyTest {
Key seekKey = new Key();
Key prevKey = new Key();
Key currKey = null;
- MutableByteSequence value = new MutableByteSequence(new byte[64], 0, 0);
+ ArrayByteSequence value = new ArrayByteSequence(new byte[64], 0, 0);
RelativeKey.SkippR skippr =
RelativeKey.fastSkip(in, seekKey, value, prevKey, currKey,
expectedKeys.size());
@@ -190,7 +189,7 @@ public class RelativeKeyTest {
Key seekKey = new Key("s", "t", "u", "v", 1);
Key prevKey = new Key();
Key currKey = null;
- MutableByteSequence value = new MutableByteSequence(new byte[64], 0, 0);
+ ArrayByteSequence value = new ArrayByteSequence(new byte[64], 0, 0);
assertThrows(EOFException.class,
() -> RelativeKey.fastSkip(in, seekKey, value, prevKey, currKey,
expectedKeys.size() + 1));
@@ -201,7 +200,7 @@ public class RelativeKeyTest {
Key seekKey = new Key("s", "t", "u", "v", 1);
Key prevKey = new Key();
Key currKey = null;
- MutableByteSequence value = new MutableByteSequence(new byte[64], 0, 0);
+ ArrayByteSequence value = new ArrayByteSequence(new byte[64], 0, 0);
SkippR skippr = RelativeKey.fastSkip(in, seekKey, value, prevKey, currKey,
expectedKeys.size());
assertEquals(expectedKeys.size(), skippr.skipped);
@@ -213,7 +212,7 @@ public class RelativeKeyTest {
Key seekKey = expectedKeys.get(seekIndex);
Key prevKey = new Key();
Key currKey = null;
- MutableByteSequence value = new MutableByteSequence(new byte[64], 0, 0);
+ ArrayByteSequence value = new ArrayByteSequence(new byte[64], 0, 0);
RelativeKey.SkippR skippr =
RelativeKey.fastSkip(in, seekKey, value, prevKey, currKey,
expectedKeys.size());