diff --git a/NOTICES b/NOTICES index 1bb6867062a..5574354ef26 100644 --- a/NOTICES +++ b/NOTICES @@ -95,10 +95,6 @@ Logback (ch.qos.logback:logback-*:1.*) - Eclipse Public License 1.0 - GNU Lesser General Public License, version 2.1 -Facebook Jcommon collections (com.facebook.jcommon:collections:0.*) - -- Apache License, Version 2.0 - Guava: Google Core Libraries For Java (com.google.guava:guava:33.*) - Apache License, Version 2.0 @@ -425,12 +421,13 @@ Maven Plugins (org.apache.maven.plugins:maven-*-plugin:3.*) - Apache License, Version 2.0 +The file java/core/src/main/java/sleeper/core/schema/type/ByteArray.java contains code that is heavily based on +ByteArray from the Facebook JCommon Collections library, licensed under the Apache License, Version 2.0. The file java/sketches/src/main/java/sleeper/sketches/SketchSerialiser.java contains code that is heavily based on ArrayOfStringsSerDe from the Apache DataSketches library, licensed under the Apache License, Version 2.0. - Sleeper also contains a Python module. This has the following dependencies. pyarrow: diff --git a/java/athena/src/main/java/sleeper/athena/record/FieldAsString.java b/java/athena/src/main/java/sleeper/athena/record/FieldAsString.java new file mode 100644 index 00000000000..22ff664b464 --- /dev/null +++ b/java/athena/src/main/java/sleeper/athena/record/FieldAsString.java @@ -0,0 +1,25 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed 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 sleeper.athena.record; + +/** + * Value for a field, held as a string along with the name of the field. + * + * @param fieldName the field name + * @param value a string representation of the value + */ +public record FieldAsString(String fieldName, String value) { +} diff --git a/java/athena/src/main/java/sleeper/athena/record/FieldAtDimension.java b/java/athena/src/main/java/sleeper/athena/record/FieldAtDimension.java new file mode 100644 index 00000000000..7ffb90a9c3e --- /dev/null +++ b/java/athena/src/main/java/sleeper/athena/record/FieldAtDimension.java @@ -0,0 +1,26 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed 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 sleeper.athena.record; + +/** + * The value for a row key field, held along with the dimension of the row key. + * The dimension is its index in the list of row keys within the schema. + * + * @param dimension the index of the row key in the schema + * @param value the value + */ +public record FieldAtDimension(int dimension, Object value) { +} diff --git a/java/athena/src/main/java/sleeper/athena/record/IteratorApplyingRecordHandler.java b/java/athena/src/main/java/sleeper/athena/record/IteratorApplyingRecordHandler.java index aa22e056111..be6e57a9daf 100644 --- a/java/athena/src/main/java/sleeper/athena/record/IteratorApplyingRecordHandler.java +++ b/java/athena/src/main/java/sleeper/athena/record/IteratorApplyingRecordHandler.java @@ -21,7 +21,6 @@ import com.amazonaws.athena.connector.lambda.domain.predicate.SortedRangeSet; import com.amazonaws.athena.connector.lambda.domain.predicate.ValueSet; import com.amazonaws.athena.connector.lambda.records.ReadRecordsRequest; -import com.facebook.collections.Pair; import com.google.gson.Gson; import org.apache.arrow.vector.types.Types; import org.apache.arrow.vector.types.pojo.ArrowType; @@ -55,7 +54,6 @@ import sleeper.query.core.rowretrieval.RowRetrievalException; import sleeper.query.runner.rowretrieval.LeafPartitionRowRetrieverImpl; -import java.io.Serializable; import java.nio.file.Path; import java.util.ArrayList; import java.util.HashMap; @@ -128,9 +126,9 @@ protected CloseableIterator createRowIterator(ReadRecordsRequest recordsReq Set relevantFiles = new HashSet<>(new Gson().fromJson(split.getProperty(RELEVANT_FILES_FIELD), List.class)); List rowKeyFields = schema.getRowKeyFields(); - List> rowKeys = split.getProperties().entrySet().stream() + List rowKeys = split.getProperties().entrySet().stream() .filter(entry -> ROW_KEY_PREFIX_TEST.test(entry.getKey())) - .map(entry -> Pair.of(entry.getKey(), entry.getValue())) + .map(entry -> new FieldAsString(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); List minRowKeys = getRowKey(rowKeys, rowKeyFields, "Min"); @@ -139,33 +137,33 @@ protected CloseableIterator createRowIterator(ReadRecordsRequest recordsReq return createIterator(relevantFiles, minRowKeys, maxRowKeys, schema, tableProperties, recordsRequest.getConstraints().getSummary()); } - private List getRowKey(List> rowKeyStream, List rowKeyFields, String indicator) { + private List getRowKey(List rowKeyStream, List rowKeyFields, String indicator) { List rowKey = new ArrayList<>(); for (int i = 0; i < rowKeyFields.size(); i++) { rowKey.add(null); } rowKeyStream.stream() - .filter(entry -> entry.getFirst().contains(indicator)) - .map(entry -> getIndexToObjectPair(rowKeyFields, entry)) - .forEach(pair -> rowKey.set(pair.getFirst(), pair.getSecond())); + .filter(entry -> entry.fieldName().contains(indicator)) + .map(entry -> getFieldAtDimension(rowKeyFields, entry)) + .forEach(valueWithIndex -> rowKey.set(valueWithIndex.dimension(), valueWithIndex.value())); return rowKey; } - private Pair getIndexToObjectPair(List rowKeyFields, Pair entry) { - String key = entry.getFirst(); + private FieldAtDimension getFieldAtDimension(List rowKeyFields, FieldAsString entry) { + String key = entry.fieldName(); Integer index = Integer.valueOf(key.substring(key.lastIndexOf("RowKey") + 6)); - String stringValue = entry.getSecond(); + String stringValue = entry.value(); Type type = rowKeyFields.get(index).getType(); if (type instanceof StringType) { - return Pair.of(index, stringValue); + return new FieldAtDimension(index, stringValue); } else if (type instanceof ByteArrayType) { - return Pair.of(index, BinaryUtils.fromBase64(stringValue)); + return new FieldAtDimension(index, BinaryUtils.fromBase64(stringValue)); } else if (type instanceof IntType) { - return Pair.of(index, Integer.parseInt(stringValue)); + return new FieldAtDimension(index, Integer.parseInt(stringValue)); } else if (type instanceof LongType) { - return Pair.of(index, Long.parseLong(stringValue)); + return new FieldAtDimension(index, Long.parseLong(stringValue)); } else { throw new RuntimeException("Unexpected Primitive type: " + type); } @@ -258,4 +256,5 @@ private CloseableIterator applyCompactionIterators(CloseableIterator m .getIterator(IteratorConfig.from(tableProperties), schema) .applyTransform(mergingIterator); } + } diff --git a/java/clients/src/main/java/sleeper/clients/table/partition/EstimateSplitPoints.java b/java/clients/src/main/java/sleeper/clients/table/partition/EstimateSplitPoints.java index d700f9244d0..1b05fb2c831 100644 --- a/java/clients/src/main/java/sleeper/clients/table/partition/EstimateSplitPoints.java +++ b/java/clients/src/main/java/sleeper/clients/table/partition/EstimateSplitPoints.java @@ -15,12 +15,12 @@ */ package sleeper.clients.table.partition; -import com.facebook.collections.ByteArray; import org.apache.datasketches.quantiles.ItemsSketch; import sleeper.core.row.Row; import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.sketches.Sketches; diff --git a/java/clients/src/main/java/sleeper/clients/table/partition/ExportSplitPoints.java b/java/clients/src/main/java/sleeper/clients/table/partition/ExportSplitPoints.java index 5002536fe4f..b3803c31ee6 100644 --- a/java/clients/src/main/java/sleeper/clients/table/partition/ExportSplitPoints.java +++ b/java/clients/src/main/java/sleeper/clients/table/partition/ExportSplitPoints.java @@ -15,7 +15,6 @@ */ package sleeper.clients.table.partition; -import com.facebook.collections.ByteArray; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.s3.S3Client; @@ -27,6 +26,7 @@ import sleeper.core.properties.table.TablePropertiesProvider; import sleeper.core.range.Range; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.LongType; diff --git a/java/compaction/compaction-job-execution/src/test/java/sleeper/compaction/job/execution/testutils/CompactionRunnerTestData.java b/java/compaction/compaction-job-execution/src/test/java/sleeper/compaction/job/execution/testutils/CompactionRunnerTestData.java index 0a9e32a5577..1e7e1864c7e 100644 --- a/java/compaction/compaction-job-execution/src/test/java/sleeper/compaction/job/execution/testutils/CompactionRunnerTestData.java +++ b/java/compaction/compaction-job-execution/src/test/java/sleeper/compaction/job/execution/testutils/CompactionRunnerTestData.java @@ -15,12 +15,12 @@ */ package sleeper.compaction.job.execution.testutils; -import com.facebook.collections.ByteArray; import org.apache.hadoop.fs.Path; import org.apache.parquet.hadoop.ParquetWriter; import sleeper.core.row.Row; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.statestore.FileReference; import sleeper.core.statestore.FileReferenceFactory; import sleeper.core.statestore.StateStore; diff --git a/java/core/pom.xml b/java/core/pom.xml index d5d1709e5fe..4b69a9ec65c 100644 --- a/java/core/pom.xml +++ b/java/core/pom.xml @@ -26,10 +26,6 @@ core - - com.facebook.jcommon - collections - org.slf4j slf4j-api diff --git a/java/core/src/main/java/sleeper/core/key/Key.java b/java/core/src/main/java/sleeper/core/key/Key.java index 4f3b10086c0..5749e27fb5a 100644 --- a/java/core/src/main/java/sleeper/core/key/Key.java +++ b/java/core/src/main/java/sleeper/core/key/Key.java @@ -15,7 +15,7 @@ */ package sleeper.core.key; -import com.facebook.collections.ByteArray; +import sleeper.core.schema.type.ByteArray; import java.util.ArrayList; import java.util.Collections; diff --git a/java/core/src/main/java/sleeper/core/partition/PartitionsFromSplitPoints.java b/java/core/src/main/java/sleeper/core/partition/PartitionsFromSplitPoints.java index 8db61a1bd34..efabe123871 100644 --- a/java/core/src/main/java/sleeper/core/partition/PartitionsFromSplitPoints.java +++ b/java/core/src/main/java/sleeper/core/partition/PartitionsFromSplitPoints.java @@ -15,7 +15,6 @@ */ package sleeper.core.partition; -import com.facebook.collections.ByteArray; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -24,6 +23,7 @@ import sleeper.core.range.Region; import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.LongType; diff --git a/java/core/src/main/java/sleeper/core/properties/local/WriteSplitPoints.java b/java/core/src/main/java/sleeper/core/properties/local/WriteSplitPoints.java index e0e077f08a9..7beeee13228 100644 --- a/java/core/src/main/java/sleeper/core/properties/local/WriteSplitPoints.java +++ b/java/core/src/main/java/sleeper/core/properties/local/WriteSplitPoints.java @@ -15,9 +15,10 @@ */ package sleeper.core.properties.local; -import com.facebook.collections.ByteArray; import org.apache.commons.codec.binary.Base64; +import sleeper.core.schema.type.ByteArray; + import java.io.IOException; import java.io.StringWriter; import java.io.UncheckedIOException; diff --git a/java/core/src/main/java/sleeper/core/range/Range.java b/java/core/src/main/java/sleeper/core/range/Range.java index 6e9d46350ba..094dd87a7f9 100644 --- a/java/core/src/main/java/sleeper/core/range/Range.java +++ b/java/core/src/main/java/sleeper/core/range/Range.java @@ -15,10 +15,9 @@ */ package sleeper.core.range; -import com.facebook.collections.ByteArray; - import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.PrimitiveType; import sleeper.core.schema.type.Type; diff --git a/java/core/src/main/java/sleeper/core/row/Row.java b/java/core/src/main/java/sleeper/core/row/Row.java index bf9ab7f9809..bf61ab97783 100644 --- a/java/core/src/main/java/sleeper/core/row/Row.java +++ b/java/core/src/main/java/sleeper/core/row/Row.java @@ -15,11 +15,10 @@ */ package sleeper.core.row; -import com.facebook.collections.ByteArray; - import sleeper.core.key.Key; import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import java.util.ArrayList; diff --git a/java/core/src/main/java/sleeper/core/schema/type/ByteArray.java b/java/core/src/main/java/sleeper/core/schema/type/ByteArray.java new file mode 100644 index 00000000000..b2f554347e9 --- /dev/null +++ b/java/core/src/main/java/sleeper/core/schema/type/ByteArray.java @@ -0,0 +1,163 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed 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. + */ +/* + * Copyright (C) 2012 Facebook, Inc. + * + * Licensed 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 sleeper.core.schema.type; + +import java.util.Arrays; +import java.util.Comparator; + +/** + * Code utilised below is a subset of the ByteArray class from facebook.collections. + * + * A wrapper object for primitive byte array type. + * Required as hashCode and equals methods required for usage of type within a HashSet. + */ +public class ByteArray implements Comparable { + + private static final ByteArrayComparator BYTE_ARRAY_COMPARATOR = new ByteArrayComparator(); + + private final byte[] array; + + private ByteArray(byte[] array) { + this.array = array; + } + + /** + * Wraps a primitive byte array. + * + * @param array the primitive byte array + * @return the wrapped object + */ + public static ByteArray wrap(byte[] array) { + return new ByteArray(array); + } + + /** + * Returns the contents of this object as a primitive byte array. + * + * @return the byte array + */ + public byte[] getArray() { + return array; + } + + /** + * Returns the length of the wrapped array. + * + * @return the length of the array + */ + public int getLength() { + return array.length; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ByteArray)) { + return false; + } + + final ByteArray that = (ByteArray) o; + + return Arrays.equals(this.getArray(), that.getArray()); + } + + @Override + public int compareTo(ByteArray o) { + return BYTE_ARRAY_COMPARATOR.compare(this, o); + } + + @Override + public int hashCode() { + return getArray() != null ? Arrays.hashCode(getArray()) : 0; + } + + @Override + public String toString() { + return "ByteArray{" + + "array=" + Arrays.toString(array) + + '}'; + } + + /** + * Comparator for ByteArray type. + */ + private static class ByteArrayComparator implements Comparator { + @Override + public int compare(ByteArray o1, ByteArray o2) { + if (o1 == null) { + if (o2 == null) { + return 0; + } else { + return -1; + } + } + + if (o2 == null) { + return 1; + } + + if (o1.getArray() == null) { + if (o2.getArray() == null) { + return 0; + } else { + return -1; + } + } + + if (o2.getArray() == null) { + return 1; + } + + int array1Length = o1.getLength(); + int array2Length = o2.getLength(); + + int length = Math.min(array1Length, array2Length); + + for (int i = 0; i < length; i++) { + if (o1.getArray()[i] < o2.getArray()[i]) { + return -1; + } else if (o1.getArray()[i] > o2.getArray()[i]) { + return 1; + } + } + + if (array1Length < array2Length) { + return -1; + } else if (array1Length > array2Length) { + return 1; + } else { + return 0; + } + } + } +} diff --git a/java/core/src/main/java/sleeper/core/schema/type/ByteArrayType.java b/java/core/src/main/java/sleeper/core/schema/type/ByteArrayType.java index 4a6d567994e..0b62ee754d7 100644 --- a/java/core/src/main/java/sleeper/core/schema/type/ByteArrayType.java +++ b/java/core/src/main/java/sleeper/core/schema/type/ByteArrayType.java @@ -15,8 +15,6 @@ */ package sleeper.core.schema.type; -import com.facebook.collections.ByteArray; - /** * A primitive type that represents a byte array. */ diff --git a/java/core/src/test/java/sleeper/core/properties/local/ReadWriteSplitPointsTest.java b/java/core/src/test/java/sleeper/core/properties/local/ReadWriteSplitPointsTest.java index ca4377a8217..41b3134ee0d 100644 --- a/java/core/src/test/java/sleeper/core/properties/local/ReadWriteSplitPointsTest.java +++ b/java/core/src/test/java/sleeper/core/properties/local/ReadWriteSplitPointsTest.java @@ -16,10 +16,10 @@ package sleeper.core.properties.local; -import com.facebook.collections.ByteArray; import org.junit.jupiter.api.Test; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.LongType; diff --git a/java/core/src/test/java/sleeper/core/range/RangeValidationTest.java b/java/core/src/test/java/sleeper/core/range/RangeValidationTest.java index 70e342b6218..ee603cdf5e9 100644 --- a/java/core/src/test/java/sleeper/core/range/RangeValidationTest.java +++ b/java/core/src/test/java/sleeper/core/range/RangeValidationTest.java @@ -15,10 +15,10 @@ */ package sleeper.core.range; -import com.facebook.collections.ByteArray; import org.junit.jupiter.api.Test; import sleeper.core.schema.Field; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.LongType; diff --git a/java/core/src/test/java/sleeper/core/schema/type/ByteArrayTest.java b/java/core/src/test/java/sleeper/core/schema/type/ByteArrayTest.java new file mode 100644 index 00000000000..f32745c4c75 --- /dev/null +++ b/java/core/src/test/java/sleeper/core/schema/type/ByteArrayTest.java @@ -0,0 +1,150 @@ +/* + * Copyright 2022-2025 Crown Copyright + * + * Licensed 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 sleeper.core.schema.type; + +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; + +import java.util.TreeSet; + +import static org.assertj.core.api.Assertions.assertThat; + +public class ByteArrayTest { + + @Nested + @DisplayName("Equality of object") + class EqualityTest { + + @Test + void shouldAllowByteArrayToBeReturnedAsPrimitiveType() { + // Given + byte[] data = {1, 2, 3}; + + // When + ByteArray result = ByteArray.wrap(data); + + // Then + assertThat(result.getArray()).isEqualTo(data); + assertThat(result.getLength()).isEqualTo(data.length); + } + + @Test + void shouldMatchWrappedByteArraysUsingSimilarData() { + // Given + ByteArray first = ByteArray.wrap(new byte[]{7, 8, 9}); + ByteArray second = ByteArray.wrap(new byte[]{7, 8, 9}); + + // When / Then + assertThat(first).isEqualTo(second); + } + + @Test + void shouldMatchWrappedByteArraysUsingSameDataSource() { + // Given + byte[] data = {1, 'b', 'c', 4, 5}; + + // When + ByteArray first = ByteArray.wrap(data); + ByteArray second = ByteArray.wrap(data); + + // Then + assertThat(first).isEqualTo(second); + } + + @Test + void shouldReturnUnequalWhenOneComparatorIsNull() { + // Given + ByteArray valid = ByteArray.wrap(new byte[]{'a', 2, 3}); + + // When / Then + assertThat(valid.equals(null)).isFalse(); + } + + @Test + void shouldReturnEqualsWhenWrappedWithNullData() { + // Given + ByteArray first = ByteArray.wrap(null); + ByteArray second = ByteArray.wrap(null); + + // When / Then + assertThat(first.equals(second)).isTrue(); + } + + @Test + void shouldReturnUnequalWhenDataDiffernt() { + // Given + ByteArray first = ByteArray.wrap(new byte[]{1, 2, 3}); + ByteArray second = ByteArray.wrap(new byte[]{'a', 'b', 'c'}); + + // When / Then + assertThat(first.equals(second)).isFalse(); + } + } + + @Nested + @DisplayName("Sorting contents") + class NestedTest { + @Test + void shouldSortByteArraysWhenContentsNumeric() { + // Given + ByteArray first = ByteArray.wrap(new byte[]{1, 2, 3}); + ByteArray second = ByteArray.wrap(new byte[]{4, 5, 6, 7, 8}); + + // When / Then + assertThat(first).isLessThan(second); + } + + @Test + void shouldSortByteArraysWhenContentsAreCharacters() { + // Given + ByteArray first = ByteArray.wrap(new byte[]{'d', 'e', 'f'}); + ByteArray second = ByteArray.wrap(new byte[]{'a', 'b', 'c'}); + + // When / Then + assertThat(first).isGreaterThan(second); + } + + @Test + void shouldSortByteArraysWhenContentsAreMixedPrimitives() { + ByteArray first = ByteArray.wrap(new byte[]{1, 'b', 3}); + ByteArray second = ByteArray.wrap(new byte[]{'a', 2, 'c', 4}); + + assertThat(first).isLessThan(second); + } + + @Test + void shouldAllowUsageWithinATreeSet() { + //Give + TreeSet treeSet = new TreeSet<>(); + ByteArray first = ByteArray.wrap(new byte[]{1, 'b', 3}); + ByteArray second = ByteArray.wrap(new byte[]{'a', 2, 'c', 4}); + + // When 1 + treeSet.add(first); + treeSet.add(second); + + // Then 1 + assertThat(treeSet).containsExactlyInAnyOrder(first, second); + + // When 2 + treeSet.remove(second); + + // Then 2 + assertThat(treeSet).containsExactly(first); + } + } +} diff --git a/java/partitions/splitter/src/main/java/sleeper/splitter/core/split/FindPartitionSplitPoint.java b/java/partitions/splitter/src/main/java/sleeper/splitter/core/split/FindPartitionSplitPoint.java index dc5c9739b1f..ab0f899b3b9 100644 --- a/java/partitions/splitter/src/main/java/sleeper/splitter/core/split/FindPartitionSplitPoint.java +++ b/java/partitions/splitter/src/main/java/sleeper/splitter/core/split/FindPartitionSplitPoint.java @@ -15,7 +15,6 @@ */ package sleeper.splitter.core.split; -import com.facebook.collections.ByteArray; import org.apache.datasketches.quantiles.ItemsSketch; import org.apache.datasketches.quantiles.ItemsUnion; import org.slf4j.Logger; @@ -23,6 +22,7 @@ import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.sketches.Sketches; import sleeper.sketches.store.SketchesStore; diff --git a/java/pom.xml b/java/pom.xml index b9cc9bcc31c..d148db5339a 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -182,7 +182,6 @@ 2.20 0.13.0 - 0.1.32 33.5.0-jre 2.45.0 @@ -1148,29 +1147,6 @@ handlebars ${handlebars.version} - - com.facebook.jcommon - collections - ${facebook.collections.version} - - - log4j - log4j - - - com.google.code.findbugs - annotations - - - commons-lang - commons-lang - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - commons-codec diff --git a/java/sketches/src/main/java/sleeper/sketches/Sketches.java b/java/sketches/src/main/java/sleeper/sketches/Sketches.java index bea844993f0..51faca5cefa 100644 --- a/java/sketches/src/main/java/sleeper/sketches/Sketches.java +++ b/java/sketches/src/main/java/sleeper/sketches/Sketches.java @@ -15,13 +15,13 @@ */ package sleeper.sketches; -import com.facebook.collections.ByteArray; import org.apache.datasketches.quantiles.ItemsSketch; import org.apache.datasketches.quantiles.ItemsUnion; import sleeper.core.row.Row; import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.PrimitiveType; diff --git a/java/sketches/src/main/java/sleeper/sketches/SketchesSerDe.java b/java/sketches/src/main/java/sleeper/sketches/SketchesSerDe.java index 90ec97e0327..dfcbcf423b8 100644 --- a/java/sketches/src/main/java/sleeper/sketches/SketchesSerDe.java +++ b/java/sketches/src/main/java/sleeper/sketches/SketchesSerDe.java @@ -15,7 +15,6 @@ */ package sleeper.sketches; -import com.facebook.collections.ByteArray; import org.apache.datasketches.ArrayOfItemsSerDe; import org.apache.datasketches.ArrayOfLongsSerDe; import org.apache.datasketches.ArrayOfStringsSerDe; @@ -26,6 +25,7 @@ import sleeper.core.schema.Field; import sleeper.core.schema.Schema; +import sleeper.core.schema.type.ByteArray; import sleeper.core.schema.type.ByteArrayType; import sleeper.core.schema.type.IntType; import sleeper.core.schema.type.LongType; diff --git a/java/sketches/src/test/java/sleeper/sketches/testutils/SketchDeciles.java b/java/sketches/src/test/java/sleeper/sketches/testutils/SketchDeciles.java index fc0f2d13668..fe0b2dfca41 100644 --- a/java/sketches/src/test/java/sleeper/sketches/testutils/SketchDeciles.java +++ b/java/sketches/src/test/java/sleeper/sketches/testutils/SketchDeciles.java @@ -15,10 +15,10 @@ */ package sleeper.sketches.testutils; -import com.facebook.collections.ByteArray; import org.apache.datasketches.quantiles.ItemsSketch; import sleeper.core.schema.Field; +import sleeper.core.schema.type.ByteArray; import sleeper.sketches.Sketches; import java.util.ArrayList;