Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 285AF200C14 for ; Tue, 7 Feb 2017 22:44:51 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 26D08160B3E; Tue, 7 Feb 2017 21:44:51 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id CBDDD160B32 for ; Tue, 7 Feb 2017 22:44:49 +0100 (CET) Received: (qmail 68483 invoked by uid 500); 7 Feb 2017 21:44:49 -0000 Mailing-List: contact commits-help@arrow.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@arrow.apache.org Delivered-To: mailing list commits@arrow.apache.org Received: (qmail 68467 invoked by uid 99); 7 Feb 2017 21:44:49 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 07 Feb 2017 21:44:49 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CE5A7DFBDB; Tue, 7 Feb 2017 21:44:48 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wesm@apache.org To: commits@arrow.apache.org Message-Id: <29f0ac2bed7749868d616bcc67fa9c7d@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: arrow git commit: ARROW-366 Java Dictionary Vector Date: Tue, 7 Feb 2017 21:44:48 +0000 (UTC) archived-at: Tue, 07 Feb 2017 21:44:51 -0000 Repository: arrow Updated Branches: refs/heads/master e97fbe640 -> c322cbf22 ARROW-366 Java Dictionary Vector I've added a dictionary type, and a partial implementation of a dictionary vector that just wraps an index vector and has a reference to a lookup vector. The spec seems to indicate that any array can be dictionary encoded, but the C++ implementation created a new type, so I went that way. Feedback would be appreciated - I want to make sure I'm on the right path. Author: Emilio Lahr-Vivaz Closes #309 from elahrvivaz/ARROW-366 and squashes the following commits: 60836ea [Emilio Lahr-Vivaz] removing dictionary ID from encoded vector 0871e13 [Emilio Lahr-Vivaz] ARROW-366 Adding Java dictionary vector Project: http://git-wip-us.apache.org/repos/asf/arrow/repo Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/c322cbf2 Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/c322cbf2 Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/c322cbf2 Branch: refs/heads/master Commit: c322cbf225b5da5e17ceec0e9e7373852bcba85c Parents: e97fbe6 Author: Emilio Lahr-Vivaz Authored: Tue Feb 7 16:44:35 2017 -0500 Committer: Wes McKinney Committed: Tue Feb 7 16:44:35 2017 -0500 ---------------------------------------------------------------------- .../arrow/vector/complex/DictionaryVector.java | 229 +++++++++++++++++++ .../apache/arrow/vector/types/Dictionary.java | 40 ++++ .../apache/arrow/vector/types/pojo/Field.java | 35 ++- .../java/org/apache/arrow/vector/util/Text.java | 31 ++- .../arrow/vector/TestDictionaryVector.java | 154 +++++++++++++ 5 files changed, 482 insertions(+), 7 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/arrow/blob/c322cbf2/java/vector/src/main/java/org/apache/arrow/vector/complex/DictionaryVector.java ---------------------------------------------------------------------- diff --git a/java/vector/src/main/java/org/apache/arrow/vector/complex/DictionaryVector.java b/java/vector/src/main/java/org/apache/arrow/vector/complex/DictionaryVector.java new file mode 100644 index 0000000..84760ea --- /dev/null +++ b/java/vector/src/main/java/org/apache/arrow/vector/complex/DictionaryVector.java @@ -0,0 +1,229 @@ +/******************************************************************************* + + * 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.arrow.vector.complex; + +import io.netty.buffer.ArrowBuf; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.OutOfMemoryException; +import org.apache.arrow.vector.NullableIntVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.complex.reader.FieldReader; +import org.apache.arrow.vector.types.Dictionary; +import org.apache.arrow.vector.types.Types.MinorType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.util.TransferPair; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +public class DictionaryVector implements ValueVector { + + private ValueVector indices; + private Dictionary dictionary; + + public DictionaryVector(ValueVector indices, Dictionary dictionary) { + this.indices = indices; + this.dictionary = dictionary; + } + + /** + * Dictionary encodes a vector. The dictionary will be built using the values from the vector. + * + * @param vector vector to encode + * @return dictionary encoded vector + */ + public static DictionaryVector encode(ValueVector vector) { + validateType(vector.getMinorType()); + Map lookUps = new HashMap<>(); + Map transfers = new HashMap<>(); + + ValueVector.Accessor accessor = vector.getAccessor(); + int count = accessor.getValueCount(); + + NullableIntVector indices = new NullableIntVector(vector.getField().getName(), vector.getAllocator()); + indices.allocateNew(count); + NullableIntVector.Mutator mutator = indices.getMutator(); + + int nextIndex = 0; + for (int i = 0; i < count; i++) { + Object value = accessor.getObject(i); + if (value != null) { // if it's null leave it null + Integer index = lookUps.get(value); + if (index == null) { + index = nextIndex++; + lookUps.put(value, index); + transfers.put(i, index); + } + mutator.set(i, index); + } + } + mutator.setValueCount(count); + + // copy the dictionary values into the dictionary vector + TransferPair dictionaryTransfer = vector.getTransferPair(vector.getAllocator()); + ValueVector dictionaryVector = dictionaryTransfer.getTo(); + dictionaryVector.allocateNewSafe(); + for (Map.Entry entry: transfers.entrySet()) { + dictionaryTransfer.copyValueSafe(entry.getKey(), entry.getValue()); + } + dictionaryVector.getMutator().setValueCount(transfers.size()); + Dictionary dictionary = new Dictionary(dictionaryVector, false); + + return new DictionaryVector(indices, dictionary); + } + + /** + * Dictionary encodes a vector with a provided dictionary. The dictionary must contain all values in the vector. + * + * @param vector vector to encode + * @param dictionary dictionary used for encoding + * @return dictionary encoded vector + */ + public static DictionaryVector encode(ValueVector vector, Dictionary dictionary) { + validateType(vector.getMinorType()); + // load dictionary values into a hashmap for lookup + ValueVector.Accessor dictionaryAccessor = dictionary.getDictionary().getAccessor(); + Map lookUps = new HashMap<>(dictionaryAccessor.getValueCount()); + for (int i = 0; i < dictionaryAccessor.getValueCount(); i++) { + // for primitive array types we need a wrapper that implements equals and hashcode appropriately + lookUps.put(dictionaryAccessor.getObject(i), i); + } + + // vector to hold our indices (dictionary encoded values) + NullableIntVector indices = new NullableIntVector(vector.getField().getName(), vector.getAllocator()); + NullableIntVector.Mutator mutator = indices.getMutator(); + + ValueVector.Accessor accessor = vector.getAccessor(); + int count = accessor.getValueCount(); + + indices.allocateNew(count); + + for (int i = 0; i < count; i++) { + Object value = accessor.getObject(i); + if (value != null) { // if it's null leave it null + // note: this may fail if value was not included in the dictionary + mutator.set(i, lookUps.get(value)); + } + } + mutator.setValueCount(count); + + return new DictionaryVector(indices, dictionary); + } + + /** + * Decodes a dictionary encoded array using the provided dictionary. + * + * @param indices dictionary encoded values, must be int type + * @param dictionary dictionary used to decode the values + * @return vector with values restored from dictionary + */ + public static ValueVector decode(ValueVector indices, Dictionary dictionary) { + ValueVector.Accessor accessor = indices.getAccessor(); + int count = accessor.getValueCount(); + ValueVector dictionaryVector = dictionary.getDictionary(); + // copy the dictionary values into the decoded vector + TransferPair transfer = dictionaryVector.getTransferPair(indices.getAllocator()); + transfer.getTo().allocateNewSafe(); + for (int i = 0; i < count; i++) { + Object index = accessor.getObject(i); + if (index != null) { + transfer.copyValueSafe(((Number) index).intValue(), i); + } + } + + ValueVector decoded = transfer.getTo(); + decoded.getMutator().setValueCount(count); + return decoded; + } + + private static void validateType(MinorType type) { + // byte arrays don't work as keys in our dictionary map - we could wrap them with something to + // implement equals and hashcode if we want that functionality + if (type == MinorType.VARBINARY || type == MinorType.LIST || type == MinorType.MAP || type == MinorType.UNION) { + throw new IllegalArgumentException("Dictionary encoding for complex types not implemented"); + } + } + + public ValueVector getIndexVector() { return indices; } + + public ValueVector getDictionaryVector() { return dictionary.getDictionary(); } + + public Dictionary getDictionary() { return dictionary; } + + @Override + public MinorType getMinorType() { return indices.getMinorType(); } + + @Override + public Field getField() { return indices.getField(); } + + // note: dictionary vector is not closed, as it may be shared + @Override + public void close() { indices.close(); } + + @Override + public void allocateNew() throws OutOfMemoryException { indices.allocateNew(); } + + @Override + public boolean allocateNewSafe() { return indices.allocateNewSafe(); } + + @Override + public BufferAllocator getAllocator() { return indices.getAllocator(); } + + @Override + public void setInitialCapacity(int numRecords) { indices.setInitialCapacity(numRecords); } + + @Override + public int getValueCapacity() { return indices.getValueCapacity(); } + + @Override + public int getBufferSize() { return indices.getBufferSize(); } + + @Override + public int getBufferSizeFor(int valueCount) { return indices.getBufferSizeFor(valueCount); } + + @Override + public Iterator iterator() { + return indices.iterator(); + } + + @Override + public void clear() { indices.clear(); } + + @Override + public TransferPair getTransferPair(BufferAllocator allocator) { return indices.getTransferPair(allocator); } + + @Override + public TransferPair getTransferPair(String ref, BufferAllocator allocator) { return indices.getTransferPair(ref, allocator); } + + @Override + public TransferPair makeTransferPair(ValueVector target) { return indices.makeTransferPair(target); } + + @Override + public Accessor getAccessor() { return indices.getAccessor(); } + + @Override + public Mutator getMutator() { return indices.getMutator(); } + + @Override + public FieldReader getReader() { return indices.getReader(); } + + @Override + public ArrowBuf[] getBuffers(boolean clear) { return indices.getBuffers(clear); } +} http://git-wip-us.apache.org/repos/asf/arrow/blob/c322cbf2/java/vector/src/main/java/org/apache/arrow/vector/types/Dictionary.java ---------------------------------------------------------------------- diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/Dictionary.java b/java/vector/src/main/java/org/apache/arrow/vector/types/Dictionary.java new file mode 100644 index 0000000..fbe1345 --- /dev/null +++ b/java/vector/src/main/java/org/apache/arrow/vector/types/Dictionary.java @@ -0,0 +1,40 @@ +/******************************************************************************* + + * 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.arrow.vector.types; + +import org.apache.arrow.vector.ValueVector; + +public class Dictionary { + + private ValueVector dictionary; + private boolean ordered; + + public Dictionary(ValueVector dictionary, boolean ordered) { + this.dictionary = dictionary; + this.ordered = ordered; + } + + public ValueVector getDictionary() { + return dictionary; + } + + public boolean isOrdered() { + return ordered; + } +} http://git-wip-us.apache.org/repos/asf/arrow/blob/c322cbf2/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java ---------------------------------------------------------------------- diff --git a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java index 412fc54..2d528e4 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/types/pojo/Field.java @@ -24,6 +24,9 @@ import static org.apache.arrow.vector.types.pojo.ArrowType.getTypeForField; import java.util.List; import java.util.Objects; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonInclude.Include; +import org.apache.arrow.flatbuf.DictionaryEncoding; import org.apache.arrow.vector.schema.TypeLayout; import org.apache.arrow.vector.schema.VectorLayout; @@ -37,6 +40,7 @@ public class Field { private final String name; private final boolean nullable; private final ArrowType type; + private final Long dictionary; private final List children; private final TypeLayout typeLayout; @@ -45,11 +49,13 @@ public class Field { @JsonProperty("name") String name, @JsonProperty("nullable") boolean nullable, @JsonProperty("type") ArrowType type, + @JsonProperty("dictionary") Long dictionary, @JsonProperty("children") List children, @JsonProperty("typeLayout") TypeLayout typeLayout) { this.name = name; this.nullable = nullable; this.type = checkNotNull(type); + this.dictionary = dictionary; if (children == null) { this.children = ImmutableList.of(); } else { @@ -59,13 +65,22 @@ public class Field { } public Field(String name, boolean nullable, ArrowType type, List children) { - this(name, nullable, type, children, TypeLayout.getTypeLayout(checkNotNull(type))); + this(name, nullable, type, null, children, TypeLayout.getTypeLayout(checkNotNull(type))); + } + + public Field(String name, boolean nullable, ArrowType type, Long dictionary, List children) { + this(name, nullable, type, dictionary, children, TypeLayout.getTypeLayout(checkNotNull(type))); } public static Field convertField(org.apache.arrow.flatbuf.Field field) { String name = field.name(); boolean nullable = field.nullable(); ArrowType type = getTypeForField(field); + DictionaryEncoding dictionaryEncoding = field.dictionary(); + Long dictionary = null; + if (dictionaryEncoding != null) { + dictionary = dictionaryEncoding.id(); + } ImmutableList.Builder layout = ImmutableList.builder(); for (int i = 0; i < field.layoutLength(); ++i) { layout.add(new org.apache.arrow.vector.schema.VectorLayout(field.layout(i))); @@ -75,8 +90,7 @@ public class Field { childrenBuilder.add(convertField(field.children(i))); } List children = childrenBuilder.build(); - Field result = new Field(name, nullable, type, children, new TypeLayout(layout.build())); - return result; + return new Field(name, nullable, type, dictionary, children, new TypeLayout(layout.build())); } public void validate() { @@ -89,6 +103,11 @@ public class Field { public int getField(FlatBufferBuilder builder) { int nameOffset = name == null ? -1 : builder.createString(name); int typeOffset = type.getType(builder); + int dictionaryOffset = -1; + if (dictionary != null) { + builder.addLong(dictionary); + dictionaryOffset = builder.offset(); + } int[] childrenData = new int[children.size()]; for (int i = 0; i < children.size(); i++) { childrenData[i] = children.get(i).getField(builder); @@ -107,6 +126,9 @@ public class Field { org.apache.arrow.flatbuf.Field.addNullable(builder, nullable); org.apache.arrow.flatbuf.Field.addTypeType(builder, type.getTypeID().getFlatbufID()); org.apache.arrow.flatbuf.Field.addType(builder, typeOffset); + if (dictionary != null) { + org.apache.arrow.flatbuf.Field.addDictionary(builder, dictionaryOffset); + } org.apache.arrow.flatbuf.Field.addChildren(builder, childrenOffset); org.apache.arrow.flatbuf.Field.addLayout(builder, layoutOffset); return org.apache.arrow.flatbuf.Field.endField(builder); @@ -124,6 +146,9 @@ public class Field { return type; } + @JsonInclude(Include.NON_NULL) + public Long getDictionary() { return dictionary; } + public List getChildren() { return children; } @@ -141,6 +166,7 @@ public class Field { return Objects.equals(this.name, that.name) && Objects.equals(this.nullable, that.nullable) && Objects.equals(this.type, that.type) && + Objects.equals(this.dictionary, that.dictionary) && (Objects.equals(this.children, that.children) || (this.children == null && that.children.size() == 0) || (this.children.size() == 0 && that.children == null)); @@ -153,6 +179,9 @@ public class Field { sb.append(name).append(": "); } sb.append(type); + if (dictionary != null) { + sb.append("[dictionary: ").append(dictionary).append("]"); + } if (!children.isEmpty()) { sb.append("<").append(Joiner.on(", ").join(children)).append(">"); } http://git-wip-us.apache.org/repos/asf/arrow/blob/c322cbf2/java/vector/src/main/java/org/apache/arrow/vector/util/Text.java ---------------------------------------------------------------------- diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/Text.java b/java/vector/src/main/java/org/apache/arrow/vector/util/Text.java index 3919f06..3db4358 100644 --- a/java/vector/src/main/java/org/apache/arrow/vector/util/Text.java +++ b/java/vector/src/main/java/org/apache/arrow/vector/util/Text.java @@ -299,6 +299,11 @@ public class Text { /** Returns true iff o is a Text with the same contents. */ @Override public boolean equals(Object o) { + if (o == this) { + return true; + } else if (o == null) { + return false; + } if (!(o instanceof Text)) { return false; } @@ -308,15 +313,33 @@ public class Text { return false; } - byte[] thisBytes = Arrays.copyOf(this.getBytes(), getLength()); - byte[] thatBytes = Arrays.copyOf(that.getBytes(), getLength()); - return Arrays.equals(thisBytes, thatBytes); + // copied from Arrays.equals so we don'thave to copy the byte arrays + for (int i = 0; i < length; i++) { + if (bytes[i] != that.bytes[i]) { + return false; + } + } + return true; } + /** + * Copied from Arrays.hashCode so we don't have to copy the byte array + * + * @return + */ @Override public int hashCode() { - return super.hashCode(); + if (bytes == null) { + return 0; + } + + int result = 1; + for (int i = 0; i < length; i++) { + result = 31 * result + bytes[i]; + } + + return result; } // / STATIC UTILITIES FROM HERE DOWN http://git-wip-us.apache.org/repos/asf/arrow/blob/c322cbf2/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java ---------------------------------------------------------------------- diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java new file mode 100644 index 0000000..962950a --- /dev/null +++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDictionaryVector.java @@ -0,0 +1,154 @@ +/** + * 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.arrow.vector; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.complex.DictionaryVector; +import org.apache.arrow.vector.types.Dictionary; +import org.apache.arrow.vector.types.Types.MinorType; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.nio.charset.StandardCharsets; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class TestDictionaryVector { + + private BufferAllocator allocator; + + byte[] zero = "foo".getBytes(StandardCharsets.UTF_8); + byte[] one = "bar".getBytes(StandardCharsets.UTF_8); + byte[] two = "baz".getBytes(StandardCharsets.UTF_8); + + @Before + public void init() { + allocator = new DirtyRootAllocator(Long.MAX_VALUE, (byte) 100); + } + + @After + public void terminate() throws Exception { + allocator.close(); + } + + @Test + public void testEncodeStringsWithGeneratedDictionary() { + // Create a new value vector + try (final NullableVarCharVector vector = (NullableVarCharVector) MinorType.VARCHAR.getNewVector("foo", allocator, null)) { + final NullableVarCharVector.Mutator m = vector.getMutator(); + vector.allocateNew(512, 5); + + // set some values + m.setSafe(0, zero, 0, zero.length); + m.setSafe(1, one, 0, one.length); + m.setSafe(2, one, 0, one.length); + m.setSafe(3, two, 0, two.length); + m.setSafe(4, zero, 0, zero.length); + m.setValueCount(5); + + DictionaryVector encoded = DictionaryVector.encode(vector); + + try { + // verify values in the dictionary + ValueVector dictionary = encoded.getDictionaryVector(); + assertEquals(vector.getClass(), dictionary.getClass()); + + NullableVarCharVector.Accessor dictionaryAccessor = ((NullableVarCharVector) dictionary).getAccessor(); + assertEquals(3, dictionaryAccessor.getValueCount()); + assertArrayEquals(zero, dictionaryAccessor.get(0)); + assertArrayEquals(one, dictionaryAccessor.get(1)); + assertArrayEquals(two, dictionaryAccessor.get(2)); + + // verify indices + ValueVector indices = encoded.getIndexVector(); + assertEquals(NullableIntVector.class, indices.getClass()); + + NullableIntVector.Accessor indexAccessor = ((NullableIntVector) indices).getAccessor(); + assertEquals(5, indexAccessor.getValueCount()); + assertEquals(0, indexAccessor.get(0)); + assertEquals(1, indexAccessor.get(1)); + assertEquals(1, indexAccessor.get(2)); + assertEquals(2, indexAccessor.get(3)); + assertEquals(0, indexAccessor.get(4)); + + // now run through the decoder and verify we get the original back + try (ValueVector decoded = DictionaryVector.decode(indices, encoded.getDictionary())) { + assertEquals(vector.getClass(), decoded.getClass()); + assertEquals(vector.getAccessor().getValueCount(), decoded.getAccessor().getValueCount()); + for (int i = 0; i < 5; i++) { + assertEquals(vector.getAccessor().getObject(i), decoded.getAccessor().getObject(i)); + } + } + } finally { + encoded.getDictionaryVector().close(); + encoded.getIndexVector().close(); + } + } + } + + @Test + public void testEncodeStringsWithProvidedDictionary() { + // Create a new value vector + try (final NullableVarCharVector vector = (NullableVarCharVector) MinorType.VARCHAR.getNewVector("foo", allocator, null); + final NullableVarCharVector dictionary = (NullableVarCharVector) MinorType.VARCHAR.getNewVector("dict", allocator, null)) { + final NullableVarCharVector.Mutator m = vector.getMutator(); + vector.allocateNew(512, 5); + + // set some values + m.setSafe(0, zero, 0, zero.length); + m.setSafe(1, one, 0, one.length); + m.setSafe(2, one, 0, one.length); + m.setSafe(3, two, 0, two.length); + m.setSafe(4, zero, 0, zero.length); + m.setValueCount(5); + + // set some dictionary values + final NullableVarCharVector.Mutator m2 = dictionary.getMutator(); + dictionary.allocateNew(512, 3); + m2.setSafe(0, zero, 0, zero.length); + m2.setSafe(1, one, 0, one.length); + m2.setSafe(2, two, 0, two.length); + m2.setValueCount(3); + + try(final DictionaryVector encoded = DictionaryVector.encode(vector, new Dictionary(dictionary, false))) { + // verify indices + ValueVector indices = encoded.getIndexVector(); + assertEquals(NullableIntVector.class, indices.getClass()); + + NullableIntVector.Accessor indexAccessor = ((NullableIntVector) indices).getAccessor(); + assertEquals(5, indexAccessor.getValueCount()); + assertEquals(0, indexAccessor.get(0)); + assertEquals(1, indexAccessor.get(1)); + assertEquals(1, indexAccessor.get(2)); + assertEquals(2, indexAccessor.get(3)); + assertEquals(0, indexAccessor.get(4)); + + // now run through the decoder and verify we get the original back + try (ValueVector decoded = DictionaryVector.decode(indices, encoded.getDictionary())) { + assertEquals(vector.getClass(), decoded.getClass()); + assertEquals(vector.getAccessor().getValueCount(), decoded.getAccessor().getValueCount()); + for (int i = 0; i < 5; i++) { + assertEquals(vector.getAccessor().getObject(i), decoded.getAccessor().getObject(i)); + } + } + } + } + } +}