From bb50d2a2b2d946b36bd29aae214d3259869e9971 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Tue, 22 Dec 2015 12:15:29 +0800 Subject: [PATCH 1/3] add some streaming writers --- .../CompressedVSizeIndexedV3Supplier.java | 138 +++++++++++ .../data/CompressedIntsIndexedWriter.java | 105 +++++++++ .../data/CompressedVSizeIndexedV3Writer.java | 72 ++++++ .../CompressedVSizeIntsIndexedSupplier.java | 2 +- .../CompressedVSizeIntsIndexedWriter.java | 119 ++++++++++ .../CompressedVSizeIndexedV3SupplierTest.java | 77 ++++++ .../data/CompressedIntsIndexedWriterTest.java | 166 +++++++++++++ .../CompressedVSizeIndexedSupplierTest.java | 35 ++- .../CompressedVSizeIndexedV3WriterTest.java | 219 ++++++++++++++++++ .../CompressedVSizeIntsIndexedWriterTest.java | 162 +++++++++++++ 10 files changed, 1082 insertions(+), 13 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java create mode 100644 processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java new file mode 100644 index 00000000000..657064bf462 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java @@ -0,0 +1,138 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.metamx.common.IAE; +import io.druid.segment.data.CompressedIntsIndexedSupplier; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedMultivalue; +import io.druid.segment.data.WritableSupplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * The format is mostly the same with CompressedVSizeIndexedSupplier(which has version 0x2, so we call it V2), + * the only difference is V3's offsets is not VSize encoded, it's just compressed. + * The reason we provide this is we can streams the data out in the binary format with CompressedVSizeIndexedV3Writer. + * If we want to streams VSizeInts, we must know the max value in the value sets. It's easy to know the max id of + * values(like dimension cardinality while encoding dimension), but difficult to known the max id of offsets. + */ +public class CompressedVSizeIndexedV3Supplier implements WritableSupplier> +{ + public static final byte version = 0x3; + + private final CompressedIntsIndexedSupplier offsetSupplier; + private final CompressedVSizeIntsIndexedSupplier valueSupplier; + + CompressedVSizeIndexedV3Supplier( + CompressedIntsIndexedSupplier offsetSupplier, + CompressedVSizeIntsIndexedSupplier valueSupplier + ) + { + this.offsetSupplier = offsetSupplier; + this.valueSupplier = valueSupplier; + } + + public static CompressedVSizeIndexedV3Supplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) + { + byte versionFromBuffer = buffer.get(); + + if (versionFromBuffer == version) { + CompressedIntsIndexedSupplier offsetSupplier = CompressedIntsIndexedSupplier.fromByteBuffer( + buffer, + order + ); + CompressedVSizeIntsIndexedSupplier valueSupplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer( + buffer, + order + ); + return new CompressedVSizeIndexedV3Supplier(offsetSupplier, valueSupplier); + } + throw new IAE("Unknown version[%s]", versionFromBuffer); + } + + // for test + public static CompressedVSizeIndexedV3Supplier fromIterable( + Iterable objectsIterable, + int offsetChunkFactor, + int maxValue, + final ByteOrder byteOrder, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + Iterator objects = objectsIterable.iterator(); + List offsetList = new ArrayList<>(); + List values = new ArrayList<>(); + + int offset = 0; + while (objects.hasNext()) { + IndexedInts next = objects.next(); + offsetList.add(offset); + for (int i = 0; i < next.size(); i++) { + values.add(next.get(i)); + } + offset += next.size(); + } + offsetList.add(offset); + CompressedIntsIndexedSupplier headerSupplier = CompressedIntsIndexedSupplier.fromList( + offsetList, + offsetChunkFactor, + byteOrder, + compression + ); + CompressedVSizeIntsIndexedSupplier valuesSupplier = CompressedVSizeIntsIndexedSupplier.fromList( + values, + maxValue, + CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), + byteOrder, + compression + ); + return new CompressedVSizeIndexedV3Supplier(headerSupplier, valuesSupplier); + } + + @Override + public long getSerializedSize() + { + return 1 + offsetSupplier.getSerializedSize() + valueSupplier.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version})); + offsetSupplier.writeToChannel(channel); + valueSupplier.writeToChannel(channel); + } + + @Override + public IndexedMultivalue get() + { + return new CompressedVSizeIndexedSupplier.CompressedVSizeIndexed(offsetSupplier.get(), valueSupplier.get()); + } + +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java new file mode 100644 index 00000000000..d12a2c8434d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -0,0 +1,105 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteStreams; +import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidResourceHolder; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.IntBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +/** + * Streams array of integers out in the binary format described by CompressedIntsIndexedSupplier + */ +public class CompressedIntsIndexedWriter +{ + public static final byte version = CompressedIntsIndexedSupplier.version; + + private final int chunkFactor; + private final ByteOrder byteOrder; + private final CompressedObjectStrategy.CompressionStrategy compression; + private final GenericIndexedWriter> flattener; + private IntBuffer endBuffer; + private int numInserted; + + public CompressedIntsIndexedWriter( + final IOPeon ioPeon, + final String filenameBase, + final int chunkFactor, + final ByteOrder byteOrder, + final CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.chunkFactor = chunkFactor; + this.byteOrder = byteOrder; + this.compression = compression; + this.flattener = new GenericIndexedWriter<>( + ioPeon, filenameBase, CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) + ); + this.endBuffer = IntBuffer.allocate(chunkFactor); + this.numInserted = 0; + } + + public void open() throws IOException + { + flattener.open(); + } + + public void add(int val) throws IOException + { + if (!endBuffer.hasRemaining()) { + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer.rewind(); + } + endBuffer.put(val); + numInserted++; + } + + public long closeAndWriteToChannel(WritableByteChannel channel) throws IOException + { + if (numInserted > 0) { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = null; + flattener.close(); + + channel.write(ByteBuffer.wrap(new byte[]{version})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + long dataLen = ByteStreams.copy(from, channel); + return 1 + // version + Ints.BYTES + // numInserted + Ints.BYTES + // chunkFactor + 1 + // compression id + dataLen; // data + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java new file mode 100644 index 00000000000..8b39d44eebb --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java @@ -0,0 +1,72 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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. + */ + +/** + * Streams array of integers out in the binary format described by CompressedVSizeIndexedV3Supplier + */ +package io.druid.segment.data; + +import io.druid.segment.CompressedVSizeIndexedV3Supplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class CompressedVSizeIndexedV3Writer +{ + public static final byte version = CompressedVSizeIndexedV3Supplier.version; + + private final CompressedIntsIndexedWriter offsetWriter; + private final CompressedVSizeIntsIndexedWriter valueWriter; + private int offset; + + public CompressedVSizeIndexedV3Writer( + CompressedIntsIndexedWriter offsetWriter, + CompressedVSizeIntsIndexedWriter valueWriter + ) + { + this.offsetWriter = offsetWriter; + this.valueWriter = valueWriter; + this.offset = 0; + } + + public void open() throws IOException + { + offsetWriter.open(); + valueWriter.open(); + } + + public void add(int[] vals) throws IOException + { + offsetWriter.add(offset); + for (int val : vals) { + valueWriter.add(val); + } + offset += vals.length; + } + + public long closeAndWriteToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version})); + offsetWriter.add(offset); + long offsetLen = offsetWriter.closeAndWriteToChannel(channel); + long dataLen = valueWriter.closeAndWriteToChannel(channel); + return 1 + offsetLen + dataLen; + } +} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java index 20aed86992f..0b1a34b70d1 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -79,7 +79,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier> flattener; + private final ByteBuffer intBuffer; + private ByteBuffer endBuffer; + private int numInserted; + + public CompressedVSizeIntsIndexedWriter( + final IOPeon ioPeon, + final String filenameBase, + final int maxValue, + final int chunkFactor, + final ByteOrder byteOrder, + final CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); + this.chunkFactor = chunkFactor; + this.chunkBytes = chunkFactor * numBytes + CompressedVSizeIntsIndexedSupplier.bufferPadding(numBytes); + this.byteOrder = byteOrder; + this.compression = compression; + this.flattener = new GenericIndexedWriter<>( + ioPeon, filenameBase, CompressedByteBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkBytes) + ); + this.intBuffer = ByteBuffer.allocate(Ints.BYTES).order(byteOrder); + this.endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder); + this.endBuffer.limit(numBytes * chunkFactor); + this.numInserted = 0; + } + + public void open() throws IOException + { + flattener.open(); + } + + public void add(int val) throws IOException + { + if (!endBuffer.hasRemaining()) { + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer.rewind(); + endBuffer.limit(numBytes * chunkFactor); + } + intBuffer.putInt(0, val); + if (byteOrder.equals(ByteOrder.BIG_ENDIAN)) { + endBuffer.put(intBuffer.array(), Ints.BYTES - numBytes, numBytes); + } else { + endBuffer.put(intBuffer.array(), 0, numBytes); + } + numInserted++; + } + + public long closeAndWriteToChannel(WritableByteChannel channel) throws IOException + { + if (numInserted > 0) { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = null; + flattener.close(); + + channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + long dataLen = ByteStreams.copy(from, channel); + return 1 + // version + 1 + // numBytes + Ints.BYTES + // numInserted + Ints.BYTES + // chunkFactor + 1 + // compression id + dataLen; // data + } +} diff --git a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java new file mode 100644 index 00000000000..557d01c4c1c --- /dev/null +++ b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java @@ -0,0 +1,77 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressedVSizeIndexedSupplierTest; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedMultivalue; +import io.druid.segment.data.VSizeIndexedInts; +import io.druid.segment.data.WritableSupplier; +import org.junit.After; +import org.junit.Before; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; + +public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexedSupplierTest +{ + @Before + public void setUpSimple(){ + vals = Arrays.asList( + new int[1], + new int[]{1, 2, 3, 4, 5}, + new int[]{6, 7, 8, 9, 10}, + new int[]{11, 12, 13, 14, 15, 16, 17, 18, 19, 20} + ); + + indexedSupplier = CompressedVSizeIndexedV3Supplier.fromIterable( + Iterables.transform( + vals, + new Function() + { + @Override + public IndexedInts apply(int[] input) + { + return VSizeIndexedInts.fromArray(input, 20); + } + } + ), 2, 20, ByteOrder.nativeOrder(), + CompressedObjectStrategy.CompressionStrategy.LZ4 + ); + } + + @After + public void teardown(){ + indexedSupplier = null; + vals = null; + } + + @Override + protected WritableSupplier> fromByteBuffer(ByteBuffer buffer, ByteOrder order) + { + return CompressedVSizeIndexedV3Supplier.fromByteBuffer( + buffer, ByteOrder.nativeOrder() + ); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java new file mode 100644 index 00000000000..6cbfdb0e6e6 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java @@ -0,0 +1,166 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.metamx.common.guava.CloseQuietly; +import org.apache.commons.io.IOUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.List; +import java.util.Random; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class CompressedIntsIndexedWriterTest +{ + @Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}") + public static Iterable compressionStrategiesAndByteOrders() + { + Set> combinations = Sets.cartesianProduct( + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) + ); + + return Iterables.transform( + combinations, new Function() + { + @Override + public Object[] apply(List input) + { + return new Object[]{input.get(0), input.get(1)}; + } + } + ); + } + + private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; + private static final int[] CHUNK_FACTORS = new int[]{1, 2, 100, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER}; + + private final IOPeon ioPeon = new TmpFileIOPeon(); + private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final ByteOrder byteOrder; + private final Random rand = new Random(0); + private int[] vals; + + public CompressedIntsIndexedWriterTest( + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + ByteOrder byteOrder + ) + { + this.compressionStrategy = compressionStrategy; + this.byteOrder = byteOrder; + } + + @Before + public void setUp() throws Exception + { + vals = null; + } + + @After + public void tearDown() throws Exception + { + ioPeon.cleanup(); + } + + private void generateVals(final int totalSize, final int maxValue) throws IOException + { + vals = new int[totalSize]; + for (int i = 0; i < vals.length; ++i) { + vals[i] = rand.nextInt(maxValue); + } + } + + private void checkSerializedSizeAndData(int chunkFactor) throws Exception + { + CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( + ioPeon, "test", chunkFactor, byteOrder, compressionStrategy + ); + CompressedIntsIndexedSupplier supplierFromList = CompressedIntsIndexedSupplier.fromList( + Ints.asList(vals), chunkFactor, byteOrder, compressionStrategy + ); + writer.open(); + for (int val : vals) { + writer.add(val); + } + final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); + long writtenLength = writer.closeAndWriteToChannel(outputChannel); + outputChannel.close(); + + assertEquals(writtenLength, supplierFromList.getSerializedSize()); + + // read from ByteBuffer and check values + CompressedIntsIndexedSupplier supplierFromByteBuffer = CompressedIntsIndexedSupplier.fromByteBuffer( + ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder + ); + IndexedInts indexedInts = supplierFromByteBuffer.get(); + assertEquals(vals.length, indexedInts.size()); + for (int i = 0; i < vals.length; ++i) { + assertEquals(vals[i], indexedInts.get(i)); + } + CloseQuietly.close(indexedInts); + } + + @Test + public void testSmallData() throws Exception + { + // less than one chunk + for (int maxValue : MAX_VALUES) { + for (int chunkFactor : CHUNK_FACTORS) { + generateVals(rand.nextInt(chunkFactor), maxValue); + checkSerializedSizeAndData(chunkFactor); + } + } + } + + @Test + public void testLargeData() throws Exception + { + // more than one chunk + for (int maxValue : MAX_VALUES) { + for (int chunkFactor : CHUNK_FACTORS) { + generateVals((rand.nextInt(5) + 5) * chunkFactor + rand.nextInt(chunkFactor), maxValue); + checkSerializedSizeAndData(chunkFactor); + } + } + } + + @Test + public void testWriteEmpty() throws Exception + { + vals = new int[0]; + checkSerializedSizeAndData(2); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java index eaf98217281..1b52dd75707 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java @@ -40,12 +40,13 @@ import java.util.List; */ public class CompressedVSizeIndexedSupplierTest { - private List vals; + protected List vals; - private CompressedVSizeIndexedSupplier indexedSupplier; + protected WritableSupplier> indexedSupplier; @Before - public void setUpSimple(){ + public void setUpSimple() + { vals = Arrays.asList( new int[1], new int[]{1, 2, 3, 4, 5}, @@ -70,7 +71,8 @@ public class CompressedVSizeIndexedSupplierTest } @After - public void teardown(){ + public void teardown() + { indexedSupplier = null; vals = null; } @@ -89,7 +91,7 @@ public class CompressedVSizeIndexedSupplierTest final byte[] bytes = baos.toByteArray(); Assert.assertEquals(indexedSupplier.getSerializedSize(), bytes.length); - CompressedVSizeIndexedSupplier deserializedIndexed = CompressedVSizeIndexedSupplier.fromByteBuffer( + WritableSupplier> deserializedIndexed = fromByteBuffer( ByteBuffer.wrap(bytes), ByteOrder.nativeOrder() ); @@ -98,26 +100,28 @@ public class CompressedVSizeIndexedSupplierTest } @Test(expected = IllegalArgumentException.class) - public void testGetInvalidElementInRow(){ + public void testGetInvalidElementInRow() + { indexedSupplier.get().get(3).get(15); } @Test - public void testIterators(){ + public void testIterators() + { Iterator iterator = indexedSupplier.get().iterator(); int row = 0; - while(iterator.hasNext()){ + while (iterator.hasNext()) { final int[] ints = vals.get(row); final IndexedInts vSizeIndexedInts = iterator.next(); Assert.assertEquals(ints.length, vSizeIndexedInts.size()); Iterator valsIterator = vSizeIndexedInts.iterator(); - int j=0; - while(valsIterator.hasNext()){ - Assert.assertEquals((Integer)ints[j], valsIterator.next()); + int j = 0; + while (valsIterator.hasNext()) { + Assert.assertEquals((Integer) ints[j], valsIterator.next()); j++; } - row ++; + row++; } } @@ -134,4 +138,11 @@ public class CompressedVSizeIndexedSupplierTest } } } + + protected WritableSupplier> fromByteBuffer(ByteBuffer buffer, ByteOrder order) + { + return CompressedVSizeIndexedSupplier.fromByteBuffer( + buffer, ByteOrder.nativeOrder() + ); + } } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java new file mode 100644 index 00000000000..414468d510e --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -0,0 +1,219 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.metamx.common.guava.CloseQuietly; +import io.druid.segment.CompressedVSizeIndexedV3Supplier; +import org.apache.commons.io.IOUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class CompressedVSizeIndexedV3WriterTest +{ + @Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}") + public static Iterable compressionStrategiesAndByteOrders() + { + Set> combinations = Sets.cartesianProduct( + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) + ); + + return Iterables.transform( + combinations, new Function() + { + @Override + public Object[] apply(List input) + { + return new Object[]{input.get(0), input.get(1)}; + } + } + ); + } + + private static final int[] OFFSET_CHUNK_FACTORS = new int[]{ + 1, + 2, + 100, + CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER + }; + private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; + + private final IOPeon ioPeon = new TmpFileIOPeon(); + private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final ByteOrder byteOrder; + private final Random rand = new Random(0); + private List vals; + + public CompressedVSizeIndexedV3WriterTest( + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + ByteOrder byteOrder + ) + { + this.compressionStrategy = compressionStrategy; + this.byteOrder = byteOrder; + } + + private void generateVals(final int totalSize, final int maxValue) throws IOException + { + vals = new ArrayList<>(totalSize); + for (int i = 0; i < totalSize; ++i) { + int len = rand.nextInt(2) + 1; + int[] subVals = new int[len]; + for (int j = 0; j < len; ++j) { + subVals[j] = rand.nextInt(maxValue); + } + vals.add(subVals); + } + } + + private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception + { + int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; + CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( + ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy + ); + CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( + ioPeon, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy + ); + CompressedVSizeIndexedV3Writer writer = new CompressedVSizeIndexedV3Writer(offsetWriter, valueWriter); + CompressedVSizeIndexedV3Supplier supplierFromIterable = CompressedVSizeIndexedV3Supplier.fromIterable( + Iterables.transform( + vals, new Function() + { + @Nullable + @Override + public IndexedInts apply(@Nullable final int[] input) + { + return new ArrayBasedIndexedInts(input); + } + } + ), offsetChunkFactor, maxValue, byteOrder, compressionStrategy + ); + writer.open(); + for (int[] val : vals) { + writer.add(val); + } + + final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); + long writtenLength = writer.closeAndWriteToChannel(outputChannel); + outputChannel.close(); + + assertEquals(writtenLength, supplierFromIterable.getSerializedSize()); + + // read from ByteBuffer and check values + CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( + ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder + ); + IndexedMultivalue indexedMultivalue = supplierFromByteBuffer.get(); + assertEquals(indexedMultivalue.size(), vals.size()); + for (int i = 0; i < vals.size(); ++i) { + IndexedInts subVals = indexedMultivalue.get(i); + assertEquals(subVals.size(), vals.get(i).length); + for (int j = 0; j < subVals.size(); ++j) { + assertEquals(subVals.get(j), vals.get(i)[j]); + } + } + CloseQuietly.close(indexedMultivalue); + } + + int getMaxValue(final List vals) + { + return Ordering.natural().max( + Iterables.transform( + vals, new Function() + { + @Nullable + @Override + public Integer apply(int[] input) + { + return input.length > 0 ? Ints.max(input) : 0; + } + } + ) + ); + } + + @Before + public void setUp() throws Exception + { + vals = null; + } + + @After + public void tearDown() throws Exception + { + ioPeon.cleanup(); + } + + @Test + public void testSmallData() throws Exception + { + // less than one chunk + for (int offsetChunk : OFFSET_CHUNK_FACTORS) { + for (int maxValue : MAX_VALUES) { + final int valueChunk = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue); + generateVals(rand.nextInt(valueChunk), maxValue); + checkSerializedSizeAndData(offsetChunk, valueChunk); + } + } + } + + @Test + public void testLargeData() throws Exception + { + // more than one chunk + for (int offsetChunk : OFFSET_CHUNK_FACTORS) { + for (int maxValue : MAX_VALUES) { + final int valueChunk = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue); + generateVals((rand.nextInt(2) + 1) * valueChunk + rand.nextInt(valueChunk), maxValue); + checkSerializedSizeAndData(offsetChunk, valueChunk); + } + } + } + + @Test + public void testEmpty() throws Exception + { + vals = new ArrayList<>(); + checkSerializedSizeAndData(1, 2); + } +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java new file mode 100644 index 00000000000..5863428ef4b --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java @@ -0,0 +1,162 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import com.google.common.primitives.Ints; +import com.metamx.common.guava.CloseQuietly; +import org.apache.commons.io.IOUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.List; +import java.util.Random; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +@RunWith(Parameterized.class) +public class CompressedVSizeIntsIndexedWriterTest +{ + @Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}") + public static Iterable compressionStrategiesAndByteOrders() + { + Set> combinations = Sets.cartesianProduct( + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) + ); + + return Iterables.transform( + combinations, new Function() + { + @Override + public Object[] apply(List input) + { + return new Object[]{input.get(0), input.get(1)}; + } + } + ); + } + + private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; + + private final IOPeon ioPeon = new TmpFileIOPeon(); + private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final ByteOrder byteOrder; + private final Random rand = new Random(0); + private int[] vals; + + public CompressedVSizeIntsIndexedWriterTest( + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + ByteOrder byteOrder + ) + { + this.compressionStrategy = compressionStrategy; + this.byteOrder = byteOrder; + } + + @Before + public void setUp() throws Exception + { + vals = null; + } + + @After + public void tearDown() throws Exception + { + ioPeon.cleanup(); + } + + private void generateVals(final int totalSize, final int maxValue) throws IOException + { + vals = new int[totalSize]; + for (int i = 0; i < vals.length; ++i) { + vals[i] = rand.nextInt(maxValue); + } + } + + private void checkSerializedSizeAndData(int chunkSize) throws Exception + { + CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( + ioPeon, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy + ); + CompressedVSizeIntsIndexedSupplier supplierFromList = CompressedVSizeIntsIndexedSupplier.fromList( + Ints.asList(vals), vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy + ); + writer.open(); + for (int val : vals) { + writer.add(val); + } + final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); + long writtenLength = writer.closeAndWriteToChannel(outputChannel); + outputChannel.close(); + + assertEquals(writtenLength, supplierFromList.getSerializedSize()); + + // read from ByteBuffer and check values + CompressedVSizeIntsIndexedSupplier supplierFromByteBuffer = CompressedVSizeIntsIndexedSupplier.fromByteBuffer( + ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), byteOrder + ); + IndexedInts indexedInts = supplierFromByteBuffer.get(); + for (int i = 0; i < vals.length; ++i) { + assertEquals(vals[i], indexedInts.get(i)); + } + CloseQuietly.close(indexedInts); + } + + @Test + public void testSmallData() throws Exception + { + // less than one chunk + for (int maxValue : MAX_VALUES) { + final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue); + generateVals(rand.nextInt(maxChunkSize), maxValue); + checkSerializedSizeAndData(maxChunkSize); + } + } + + @Test + public void testLargeData() throws Exception + { + // more than one chunk + for (int maxValue : MAX_VALUES) { + final int maxChunkSize = CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue); + generateVals((rand.nextInt(5) + 5) * maxChunkSize + rand.nextInt(maxChunkSize), maxValue); + checkSerializedSizeAndData(maxChunkSize); + } + } + + @Test + public void testEmpty() throws Exception + { + vals = new int[0]; + checkSerializedSizeAndData(2); + } +} \ No newline at end of file From 1f2168fae5ce590efab0621069f92f99615a16f0 Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Thu, 24 Dec 2015 13:56:41 +0800 Subject: [PATCH 2/3] add IndexMergerV9 add unit tests for IndexMergerV9 and fix some bugs add more unit tests and fix bugs handle null values and add more tests minor changes & use LoggingProgressIndicator in IndexGeneratorReducer make some static class public from IndexMerger minor changes and add some comments changes for comments --- .../theta/SketchMergeComplexMetricSerde.java | 5 +- .../ApproximateHistogramFoldingSerde.java | 7 +- .../io/druid/indexer/IndexGeneratorJob.java | 1 + .../hyperloglog/HyperUniquesSerde.java | 7 +- .../CompressedVSizeIndexedV3Supplier.java | 6 +- .../druid/segment/FloatColumnSerializer.java | 97 ++ .../segment/GenericColumnSerializer.java | 35 + .../main/java/io/druid/segment/IndexIO.java | 36 +- .../java/io/druid/segment/IndexMerger.java | 37 +- .../java/io/druid/segment/IndexMergerV9.java | 1304 +++++++++++++++++ .../druid/segment/LongColumnSerializer.java | 97 ++ .../segment/column/ColumnDescriptor.java | 6 +- .../druid/segment/data/ByteBufferWriter.java | 14 + .../CompressedFloatsSupplierSerializer.java | 53 +- .../data/CompressedIntsIndexedSupplier.java | 6 +- .../data/CompressedIntsIndexedWriter.java | 70 +- .../CompressedLongsSupplierSerializer.java | 54 +- .../data/CompressedVSizeIndexedV3Writer.java | 77 +- .../CompressedVSizeIntsIndexedSupplier.java | 6 +- .../CompressedVSizeIntsIndexedWriter.java | 67 +- .../segment/data/GenericIndexedWriter.java | 18 + .../druid/segment/data/IndexedIntsWriter.java | 35 + .../data/MultiValueIndexedIntsWriter.java | 45 + .../data/SingleValueIndexedIntsWriter.java | 48 + .../io/druid/segment/data/TmpFileIOPeon.java | 22 +- .../druid/segment/data/VSizeIndexedInts.java | 6 +- .../segment/data/VSizeIndexedIntsWriter.java | 94 ++ .../segment/data/VSizeIndexedWriter.java | 56 +- .../incremental/IncrementalIndexAdapter.java | 52 +- .../druid/segment/serde/ColumnPartSerde.java | 18 +- .../segment/serde/ComplexColumnPartSerde.java | 116 +- .../serde/ComplexColumnSerializer.java | 90 ++ .../segment/serde/ComplexMetricSerde.java | 6 +- .../DictionaryEncodedColumnPartSerde.java | 580 ++++---- .../serde/FloatGenericColumnPartSerde.java | 124 +- .../serde/LongGenericColumnPartSerde.java | 126 +- .../io/druid/segment/IndexMergerTest.java | 347 ++++- .../IndexMergerV9CompatibilityTest.java | 310 ++++ .../io/druid/segment/IndexMergerV9Test.java | 1025 +++++++++++++ .../IndexMergerV9WithSpatialIndexTest.java | 707 +++++++++ .../java/io/druid/segment/TestHelper.java | 7 + ...ompressedFloatsSupplierSerializerTest.java | 43 + .../data/CompressedIntsIndexedWriterTest.java | 4 +- ...CompressedLongsSupplierSerializerTest.java | 38 + .../CompressedVSizeIndexedV3WriterTest.java | 5 +- .../CompressedVSizeIntsIndexedWriterTest.java | 4 +- .../data/VSizeIndexedIntsWriterTest.java | 111 ++ 47 files changed, 5491 insertions(+), 531 deletions(-) create mode 100644 processing/src/main/java/io/druid/segment/FloatColumnSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/GenericColumnSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/IndexMergerV9.java create mode 100644 processing/src/main/java/io/druid/segment/LongColumnSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/SingleValueIndexedIntsWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java create mode 100644 processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java create mode 100644 processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java create mode 100644 processing/src/test/java/io/druid/segment/IndexMergerV9Test.java create mode 100644 processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java diff --git a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java index 1a535f6a399..cbb8935fad5 100644 --- a/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java +++ b/extensions/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java @@ -25,8 +25,6 @@ import io.druid.data.input.InputRow; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.ObjectStrategy; -import io.druid.segment.serde.ColumnPartSerde; -import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -67,11 +65,10 @@ public class SketchMergeComplexMetricSerde extends ComplexMetricSerde } @Override - public ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder) + public void deserializeColumn(ByteBuffer buffer, ColumnBuilder builder) { GenericIndexed ge = GenericIndexed.read(buffer, strategy); builder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), ge)); - return new ComplexColumnPartSerde(ge, getTypeName()); } @Override diff --git a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java index 48b7b48d84c..20c9ea691eb 100644 --- a/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java +++ b/extensions/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -24,8 +24,6 @@ import io.druid.data.input.InputRow; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.ObjectStrategy; -import io.druid.segment.serde.ColumnPartSerde; -import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -92,15 +90,12 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde } @Override - public ColumnPartSerde deserializeColumn( + public void deserializeColumn( ByteBuffer byteBuffer, ColumnBuilder columnBuilder ) { final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy()); - columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); - - return new ComplexColumnPartSerde(column, getTypeName()); } public ObjectStrategy getObjectStrategy() diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 28b4abb83e2..56059c866e7 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -454,6 +454,7 @@ public class IndexGeneratorJob implements Jobby @Override public void progress() { + super.progress(); context.progress(); } }; diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java index c92cf26670c..640c1ed94d6 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -26,8 +26,6 @@ import io.druid.data.input.InputRow; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.ObjectStrategy; -import io.druid.segment.serde.ColumnPartSerde; -import io.druid.segment.serde.ComplexColumnPartSerde; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; import io.druid.segment.serde.ComplexMetricSerde; @@ -103,15 +101,12 @@ public class HyperUniquesSerde extends ComplexMetricSerde } @Override - public ColumnPartSerde deserializeColumn( + public void deserializeColumn( ByteBuffer byteBuffer, ColumnBuilder columnBuilder ) { final GenericIndexed column = GenericIndexed.read(byteBuffer, getObjectStrategy()); - columnBuilder.setComplexColumn(new ComplexColumnPartSupplier(getTypeName(), column)); - - return new ComplexColumnPartSerde(column, getTypeName()); } @Override diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java index 657064bf462..5de5604c970 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java @@ -44,7 +44,7 @@ import java.util.List; */ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier> { - public static final byte version = 0x3; + public static final byte VERSION = 0x3; private final CompressedIntsIndexedSupplier offsetSupplier; private final CompressedVSizeIntsIndexedSupplier valueSupplier; @@ -62,7 +62,7 @@ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier getLexicographicMergedDimensions(List indexes) + private static List getLexicographicMergedDimensions(List indexes) { return mergeIndexed( Lists.transform( @@ -272,7 +272,7 @@ public class IndexMerger ); } - private List getMergedDimensions(List indexes) + public static List getMergedDimensions(List indexes) { if (indexes.size() == 0) { return ImmutableList.of(); @@ -280,7 +280,7 @@ public class IndexMerger Indexed dimOrder = indexes.get(0).getDimensionNames(); for (IndexableAdapter index : indexes) { Indexed dimOrder2 = index.getDimensionNames(); - if(!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) { + if (!Iterators.elementsEqual(dimOrder.iterator(), dimOrder2.iterator())) { return getLexicographicMergedDimensions(indexes); } } @@ -964,7 +964,7 @@ public class IndexMerger return outDir; } - private ArrayList mergeIndexed(final List> indexedLists) + public static ArrayList mergeIndexed(final List> indexedLists) { Set retVal = Sets.newTreeSet(Ordering.natural().nullsFirst()); @@ -1003,7 +1003,7 @@ public class IndexMerger IndexIO.checkFileSize(indexFile); } - private static class DimValueConverter + public static class DimValueConverter { private final Indexed dimSet; private final IntBuffer conversionBuf; @@ -1074,7 +1074,7 @@ public class IndexMerger } } - private static class ConvertingIndexedInts implements Iterable + public static class ConvertingIndexedInts implements Iterable { private final IndexedInts baseIndex; private final IntBuffer conversionBuffer; @@ -1115,7 +1115,7 @@ public class IndexMerger } } - private static class MMappedIndexRowIterable implements Iterable + public static class MMappedIndexRowIterable implements Iterable { private final Iterable index; private final List convertedDims; @@ -1140,21 +1140,6 @@ public class IndexMerger return index; } - public List getConvertedDims() - { - return convertedDims; - } - - public Map getConverters() - { - return converters; - } - - public int getIndexNumber() - { - return indexNumber; - } - @Override public Iterator iterator() { @@ -1174,8 +1159,6 @@ public class IndexMerger index.iterator(), new Function() { - int rowCount = 0; - @Override public Rowboat apply(@Nullable Rowboat input) { @@ -1218,7 +1201,7 @@ public class IndexMerger } } - private static class AggFactoryStringIndexed implements Indexed + public static class AggFactoryStringIndexed implements Indexed { private final AggregatorFactory[] metricAggs; @@ -1255,7 +1238,7 @@ public class IndexMerger } } - private static class RowboatMergeFunction implements BinaryFn + public static class RowboatMergeFunction implements BinaryFn { private final AggregatorFactory[] metricAggs; @@ -1301,7 +1284,7 @@ public class IndexMerger } } - static boolean isNullColumn(Iterable dimValues) + public static boolean isNullColumn(Iterable dimValues) { if (dimValues == null) { return true; diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java new file mode 100644 index 00000000000..e83b76a60dc --- /dev/null +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -0,0 +1,1304 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.io.ByteStreams; +import com.google.common.io.Files; +import com.google.common.primitives.Ints; +import com.google.inject.Inject; +import com.metamx.collections.bitmap.BitmapFactory; +import com.metamx.collections.bitmap.ImmutableBitmap; +import com.metamx.collections.bitmap.MutableBitmap; +import com.metamx.collections.spatial.ImmutableRTree; +import com.metamx.collections.spatial.RTree; +import com.metamx.collections.spatial.split.LinearGutmanSplitStrategy; +import com.metamx.common.IAE; +import com.metamx.common.ISE; +import com.metamx.common.guava.FunctionalIterable; +import com.metamx.common.guava.MergeIterable; +import com.metamx.common.io.smoosh.FileSmoosher; +import com.metamx.common.io.smoosh.SmooshedWriter; +import com.metamx.common.logger.Logger; +import io.druid.collections.CombiningIterable; +import io.druid.common.utils.JodaUtils; +import io.druid.common.utils.SerializerUtils; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.segment.column.BitmapIndexSeeker; +import io.druid.segment.column.Column; +import io.druid.segment.column.ColumnCapabilities; +import io.druid.segment.column.ColumnCapabilitiesImpl; +import io.druid.segment.column.ColumnDescriptor; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.ArrayIndexed; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.ByteBufferWriter; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressedVSizeIndexedV3Writer; +import io.druid.segment.data.CompressedVSizeIntsIndexedWriter; +import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; +import io.druid.segment.data.Indexed; +import io.druid.segment.data.IndexedIntsWriter; +import io.druid.segment.data.IndexedIterable; +import io.druid.segment.data.IndexedRTree; +import io.druid.segment.data.TmpFileIOPeon; +import io.druid.segment.data.VSizeIndexedIntsWriter; +import io.druid.segment.data.VSizeIndexedWriter; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; +import io.druid.segment.serde.ComplexColumnPartSerde; +import io.druid.segment.serde.ComplexColumnSerializer; +import io.druid.segment.serde.ComplexMetricSerde; +import io.druid.segment.serde.ComplexMetrics; +import io.druid.segment.serde.DictionaryEncodedColumnPartSerde; +import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.serde.LongGenericColumnPartSerde; +import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.IntBuffer; +import java.nio.MappedByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + +public class IndexMergerV9 +{ + private static final Logger log = new Logger(IndexMergerV9.class); + + private static final SerializerUtils serializerUtils = new SerializerUtils(); + private static final int INVALID_ROW = -1; + private static final Splitter SPLITTER = Splitter.on(","); + + private final ObjectMapper mapper; + private final IndexIO indexIO; + + @Inject + public IndexMergerV9( + ObjectMapper mapper, + IndexIO indexIO + ) + { + this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); + this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); + } + + public File persist( + final IncrementalIndex index, + File outDir, + Map segmentMetadata, + IndexSpec indexSpec + ) throws IOException + { + return persist(index, index.getInterval(), outDir, segmentMetadata, indexSpec); + } + + public File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + Map segmentMetadata, + IndexSpec indexSpec + ) throws IOException + { + return persist( + index, + dataInterval, + outDir, + segmentMetadata, + indexSpec, + new BaseProgressIndicator() + ); + } + + public File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + Map segmentMetadata, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + if (index.isEmpty()) { + throw new IAE("Trying to persist an empty index!"); + } + + final long firstTimestamp = index.getMinTime().getMillis(); + final long lastTimestamp = index.getMaxTime().getMillis(); + if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { + throw new IAE( + "interval[%s] does not encapsulate the full range of timestamps[%s, %s]", + dataInterval, + new DateTime(firstTimestamp), + new DateTime(lastTimestamp) + ); + } + + if (!outDir.exists()) { + outDir.mkdirs(); + } + if (!outDir.isDirectory()) { + throw new ISE("Can only persist to directories, [%s] wasn't a directory", outDir); + } + + log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + return merge( + Arrays.asList( + new IncrementalIndexAdapter( + dataInterval, + index, + indexSpec.getBitmapSerdeFactory().getBitmapFactory() + ) + ), + index.getMetricAggs(), + outDir, + segmentMetadata, + indexSpec, + progress + ); + } + + public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException + { + return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); + } + + public File convert( + final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress + ) throws IOException + { + try (QueryableIndex index = indexIO.loadIndex(inDir)) { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); + return makeIndexFiles( + ImmutableList.of(adapter), + outDir, + progress, + Lists.newArrayList(adapter.getDimensionNames()), + Lists.newArrayList(adapter.getMetricNames()), + null, + new Function>, Iterable>() + { + @Nullable + @Override + public Iterable apply(ArrayList> input) + { + return input.get(0); + } + }, + indexSpec + ); + } + } + + public File mergeQueryableIndex( + List indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec + ) throws IOException + { + return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + } + + public File mergeQueryableIndex( + List indexes, + final AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + // We are materializing the list for performance reasons. Lists.transform + // only creates a "view" of the original list, meaning the function gets + // applied every time you access an element. + List indexAdapteres = Lists.newArrayList( + Iterables.transform( + indexes, + new Function() + { + @Override + public IndexableAdapter apply(final QueryableIndex input) + { + return new QueryableIndexIndexableAdapter(input); + } + } + ) + ); + return merge( + indexAdapteres, + metricAggs, + outDir, + null, + indexSpec, + progress + ); + } + + public File merge( + List indexes, + final AggregatorFactory[] metricAggs, + File outDir, + Map segmentMetadata, + IndexSpec indexSpec + ) throws IOException + { + return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator()); + } + + public File merge( + List adapters, + final AggregatorFactory[] metricAggs, + File outDir, + Map segmentMetadata, + IndexSpec indexSpec, + ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + if (!outDir.mkdirs()) { + throw new ISE("Couldn't make outdir[%s].", outDir); + } + + final List mergedDimensions = IndexMerger.getMergedDimensions(adapters); + + final List mergedMetrics = Lists.transform( + IndexMerger.mergeIndexed( + Lists.newArrayList( + FunctionalIterable + .create(adapters) + .transform( + new Function>() + { + @Override + public Iterable apply(@Nullable IndexableAdapter input) + { + return input.getMetricNames(); + } + } + ) + .concat(Arrays.>asList(new IndexMerger.AggFactoryStringIndexed(metricAggs))) + ) + ), + new Function() + { + @Override + public String apply(@Nullable String input) + { + return input; + } + } + ); + if (mergedMetrics.size() != metricAggs.length) { + throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), metricAggs.length); + } + + final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; + for (int i = 0; i < metricAggs.length; i++) { + AggregatorFactory metricAgg = metricAggs[i]; + sortedMetricAggs[mergedMetrics.indexOf(metricAgg.getName())] = metricAgg; + } + + for (int i = 0; i < mergedMetrics.size(); i++) { + if (!sortedMetricAggs[i].getName().equals(mergedMetrics.get(i))) { + throw new IAE( + "Metric mismatch, index[%d] [%s] != [%s]", + i, + metricAggs[i].getName(), + mergedMetrics.get(i) + ); + } + } + + Function>, Iterable> rowMergerFn = + new Function>, Iterable>() + { + @Override + public Iterable apply( + @Nullable ArrayList> boats + ) + { + return CombiningIterable.create( + new MergeIterable<>( + Ordering.natural().nullsFirst(), + boats + ), + Ordering.natural().nullsFirst(), + new IndexMerger.RowboatMergeFunction(sortedMetricAggs) + ); + } + }; + + return makeIndexFiles( + adapters, outDir, progress, mergedDimensions, mergedMetrics, segmentMetadata, rowMergerFn, indexSpec + ); + } + + public File append( + List indexes, File outDir, IndexSpec indexSpec + ) throws IOException + { + return append(indexes, outDir, indexSpec, new BaseProgressIndicator()); + } + + public File append( + List indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress + ) throws IOException + { + FileUtils.deleteDirectory(outDir); + if (!outDir.mkdirs()) { + throw new ISE("Couldn't make outdir[%s].", outDir); + } + + final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); + + final List mergedMetrics = IndexMerger.mergeIndexed( + Lists.transform( + indexes, + new Function>() + { + @Override + public Iterable apply(@Nullable IndexableAdapter input) + { + return Iterables.transform( + input.getMetricNames(), + new Function() + { + @Override + public String apply(@Nullable String input) + { + return input; + } + } + ); + } + } + ) + ); + + Function>, Iterable> rowMergerFn = new Function>, Iterable>() + { + @Override + public Iterable apply( + @Nullable final ArrayList> boats + ) + { + return new MergeIterable( + Ordering.natural().nullsFirst(), + boats + ); + } + }; + + return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec); + } + + private File makeIndexFiles( + final List adapters, + final File outDir, + final ProgressIndicator progress, + final List mergedDimensions, + final List mergedMetrics, + final Map segmentMetadata, + final Function>, Iterable> rowMergerFn, + final IndexSpec indexSpec + ) throws IOException + { + progress.start(); + progress.progress(); + + final IOPeon ioPeon = new TmpFileIOPeon(false); + final FileSmoosher v9Smoosher = new FileSmoosher(outDir); + final File v9TmpDir = new File(outDir, "v9-tmp"); + v9TmpDir.mkdirs(); + log.info("Start making v9 index files, outDir:%s", outDir); + + long startTime = System.currentTimeMillis(); + ByteStreams.write( + Ints.toByteArray(IndexIO.V9_VERSION), + Files.newOutputStreamSupplier(new File(outDir, "version.bin")) + ); + log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime); + + progress.progress(); + final Map metricsValueTypes = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final Map metricTypeNames = Maps.newTreeMap(Ordering.natural().nullsFirst()); + final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); + mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities); + + /************* Setup Dim Conversions **************/ + progress.progress(); + startTime = System.currentTimeMillis(); + final Map dimCardinalities = Maps.newHashMap(); + final ArrayList> dimValueWriters = setupDimValueWriters(ioPeon, mergedDimensions); + final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); + final ArrayList dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size()); + writeDimValueAndSetupDimConversion( + adapters, progress, mergedDimensions, dimCardinalities, dimValueWriters, dimensionSkipFlag, dimConversions + ); + log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); + + /************* Walk through data sets, merge them, and write merged columns *************/ + progress.progress(); + final Iterable theRows = makeRowIterable( + adapters, mergedDimensions, mergedMetrics, dimConversions, rowMergerFn + ); + final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon); + final ArrayList dimWriters = setupDimensionWriters( + ioPeon, mergedDimensions, dimCapabilities, dimCardinalities, indexSpec + ); + final ArrayList metWriters = setupMetricsWriters( + ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec + ); + final List rowNumConversions = Lists.newArrayListWithCapacity(adapters.size()); + final ArrayList nullRowsList = Lists.newArrayListWithCapacity(mergedDimensions.size()); + for (int i = 0; i < mergedDimensions.size(); ++i) { + nullRowsList.add(indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap()); + } + mergeIndexesAndWriteColumns( + adapters, progress, theRows, timeWriter, dimWriters, metWriters, + dimensionSkipFlag, rowNumConversions, nullRowsList + ); + + /************ Create Inverted Indexes *************/ + progress.progress(); + final ArrayList> bitmapIndexWriters = setupBitmapIndexWriters( + ioPeon, mergedDimensions, indexSpec + ); + final ArrayList> spatialIndexWriters = setupSpatialIndexWriters( + ioPeon, mergedDimensions, indexSpec, dimCapabilities + ); + makeInvertedIndexes( + adapters, progress, mergedDimensions, indexSpec, v9TmpDir, rowNumConversions, + nullRowsList, dimValueWriters, bitmapIndexWriters, spatialIndexWriters + ); + + /************ Finalize Build Columns *************/ + progress.progress(); + makeTimeColumn(v9Smoosher, progress, timeWriter); + makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsValueTypes, metricTypeNames, metWriters); + makeDimensionColumns( + v9Smoosher, progress, indexSpec, mergedDimensions, dimensionSkipFlag, dimCapabilities, + dimValueWriters, dimWriters, bitmapIndexWriters, spatialIndexWriters + ); + + /************* Make index.drd & metadata.drd files **************/ + progress.progress(); + makeIndexBinary( + v9Smoosher, adapters, outDir, mergedDimensions, dimensionSkipFlag, mergedMetrics, progress, indexSpec + ); + makeMetadataBinary(v9Smoosher, progress, segmentMetadata); + + v9Smoosher.close(); + ioPeon.cleanup(); + FileUtils.deleteDirectory(v9TmpDir); + progress.stop(); + + return outDir; + } + + private void makeMetadataBinary( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final Map segmentMetadata + ) throws IOException + { + if (segmentMetadata != null && !segmentMetadata.isEmpty()) { + progress.startSection("make metadata.drd"); + v9Smoosher.add("metadata.drd", ByteBuffer.wrap(mapper.writeValueAsBytes(segmentMetadata))); + progress.stopSection("make metadata.drd"); + } + } + + private void makeIndexBinary( + final FileSmoosher v9Smoosher, + final List adapters, + final File outDir, + final List mergedDimensions, + final ArrayList dimensionSkipFlag, + final List mergedMetrics, + final ProgressIndicator progress, + final IndexSpec indexSpec + ) throws IOException + { + final String section = "make index.drd"; + progress.startSection(section); + + long startTime = System.currentTimeMillis(); + final Set finalColumns = Sets.newTreeSet(); + final Set finalDimensions = Sets.newTreeSet(); + finalColumns.addAll(mergedMetrics); + for (int i = 0; i < mergedDimensions.size(); ++i) { + if (dimensionSkipFlag.get(i)) { + continue; + } + finalColumns.add(mergedDimensions.get(i)); + finalDimensions.add(mergedDimensions.get(i)); + } + + GenericIndexed cols = GenericIndexed.fromIterable(finalColumns, GenericIndexed.STRING_STRATEGY); + GenericIndexed dims = GenericIndexed.fromIterable(finalDimensions, GenericIndexed.STRING_STRATEGY); + + final String bitmapSerdeFactoryType = mapper.writeValueAsString(indexSpec.getBitmapSerdeFactory()); + final long numBytes = cols.getSerializedSize() + + dims.getSerializedSize() + + 16 + + serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType); + + final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); + cols.writeToChannel(writer); + dims.writeToChannel(writer); + + DateTime minTime = new DateTime(JodaUtils.MAX_INSTANT); + DateTime maxTime = new DateTime(JodaUtils.MIN_INSTANT); + + for (IndexableAdapter index : adapters) { + minTime = JodaUtils.minDateTime(minTime, index.getDataInterval().getStart()); + maxTime = JodaUtils.maxDateTime(maxTime, index.getDataInterval().getEnd()); + } + final Interval dataInterval = new Interval(minTime, maxTime); + + serializerUtils.writeLong(writer, dataInterval.getStartMillis()); + serializerUtils.writeLong(writer, dataInterval.getEndMillis()); + + serializerUtils.writeString( + writer, bitmapSerdeFactoryType + ); + writer.close(); + + IndexIO.checkFileSize(new File(outDir, "index.drd")); + log.info("Completed index.drd in %,d millis.", System.currentTimeMillis() - startTime); + + progress.stopSection(section); + } + + private void makeDimensionColumns( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final IndexSpec indexSpec, + final List mergedDimensions, + final ArrayList dimensionSkipFlag, + final List dimCapabilities, + final ArrayList> dimValueWriters, + final ArrayList dimWriters, + final ArrayList> bitmapIndexWriters, + final ArrayList> spatialIndexWriters + ) throws IOException + { + final String section = "make dimension columns"; + progress.startSection(section); + + long startTime = System.currentTimeMillis(); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy(); + for (int i = 0; i < mergedDimensions.size(); ++i) { + long dimStartTime = System.currentTimeMillis(); + final String dim = mergedDimensions.get(i); + final IndexedIntsWriter dimWriter = dimWriters.get(i); + final GenericIndexedWriter bitmapIndexWriter = bitmapIndexWriters.get(i); + final ByteBufferWriter spatialIndexWriter = spatialIndexWriters.get(i); + + dimWriter.close(); + bitmapIndexWriter.close(); + if (spatialIndexWriter != null) { + spatialIndexWriter.close(); + } + if (dimensionSkipFlag.get(i)) { + continue; + } + + boolean hasMultiValue = dimCapabilities.get(i).hasMultipleValues(); + + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + builder.setValueType(ValueType.STRING); + builder.setHasMultipleValues(hasMultiValue); + final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde + .serializerBuilder() + .withDictionary(dimValueWriters.get(i)) + .withValue(dimWriters.get(i), hasMultiValue, compressionStrategy != null) + .withBitmapSerdeFactory(bitmapSerdeFactory) + .withBitmapIndex(bitmapIndexWriters.get(i)) + .withSpatialIndex(spatialIndexWriters.get(i)) + .withByteOrder(IndexIO.BYTE_ORDER); + final ColumnDescriptor serdeficator = builder + .addSerde(partBuilder.build()) + .build(); + makeColumn(v9Smoosher, dim, serdeficator); + log.info("Completed dimension column[%s] in %,d millis.", dim, System.currentTimeMillis() - dimStartTime); + } + log.info("Completed dimension columns in %,d millis.", System.currentTimeMillis() - startTime); + progress.stopSection(section); + } + + private void makeMetricsColumns( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final List mergedMetrics, + final Map metricsValueTypes, + final Map metricTypeNames, + final List metWriters + ) throws IOException + { + final String section = "make metric columns"; + progress.startSection(section); + long startTime = System.currentTimeMillis(); + + for (int i = 0; i < mergedMetrics.size(); ++i) { + String metric = mergedMetrics.get(i); + long metricStartTime = System.currentTimeMillis(); + GenericColumnSerializer writer = metWriters.get(i); + writer.close(); + + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + ValueType type = metricsValueTypes.get(metric); + switch (type) { + case LONG: + builder.setValueType(ValueType.LONG); + builder.addSerde( + LongGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate((LongColumnSerializer) writer) + .build() + ); + break; + case FLOAT: + builder.setValueType(ValueType.FLOAT); + builder.addSerde( + FloatGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate((FloatColumnSerializer) writer) + .build() + ); + break; + case COMPLEX: + final String typeName = metricTypeNames.get(metric); + builder.setValueType(ValueType.COMPLEX); + builder.addSerde( + ComplexColumnPartSerde.serializerBuilder().withTypeName(typeName) + .withDelegate((ComplexColumnSerializer) writer) + .build() + ); + break; + default: + throw new ISE("Unknown type[%s]", type); + } + makeColumn(v9Smoosher, metric, builder.build()); + log.info("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); + } + log.info("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime); + progress.stopSection(section); + } + + + private void makeTimeColumn( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final LongColumnSerializer timeWriter + ) throws IOException + { + final String section = "make time column"; + progress.startSection(section); + long startTime = System.currentTimeMillis(); + + timeWriter.close(); + + final ColumnDescriptor serdeficator = ColumnDescriptor + .builder() + .setValueType(ValueType.LONG) + .addSerde( + LongGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate(timeWriter) + .build() + ) + .build(); + makeColumn(v9Smoosher, Column.TIME_COLUMN_NAME, serdeficator); + log.info("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); + progress.stopSection(section); + } + + private void makeColumn( + final FileSmoosher v9Smoosher, + final String columnName, + final ColumnDescriptor serdeficator + ) throws IOException + { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializerUtils.writeString(baos, mapper.writeValueAsString(serdeficator)); + byte[] specBytes = baos.toByteArray(); + + final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( + columnName, serdeficator.numBytes() + specBytes.length + ); + try { + channel.write(ByteBuffer.wrap(specBytes)); + serdeficator.write(channel); + } + finally { + channel.close(); + } + } + + private void makeInvertedIndexes( + final List adapters, + final ProgressIndicator progress, + final List mergedDimensions, + final IndexSpec indexSpec, + final File v9OutDir, + final List rowNumConversions, + final ArrayList nullRowsList, + final ArrayList> dimValueWriters, + final ArrayList> bitmapIndexWriters, + final ArrayList> spatialIndexWriters + ) throws IOException + { + final String section = "build inverted index"; + progress.startSection(section); + + long startTime = System.currentTimeMillis(); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { + String dimension = mergedDimensions.get(dimIndex); + long dimStartTime = System.currentTimeMillis(); + + // write dim values to one single file because we need to read it + File dimValueFile = IndexIO.makeDimFile(v9OutDir, dimension); + FileOutputStream fos = new FileOutputStream(dimValueFile); + ByteStreams.copy(dimValueWriters.get(dimIndex).combineStreams(), fos); + fos.close(); + + final MappedByteBuffer dimValsMapped = Files.map(dimValueFile); + Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); + + ByteBufferWriter spatialIndexWriter = spatialIndexWriters.get(dimIndex); + RTree tree = null; + if (spatialIndexWriter != null) { + BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); + } + + BitmapIndexSeeker[] bitmapIndexSeeker = new BitmapIndexSeeker[adapters.size()]; + for (int j = 0; j < adapters.size(); j++) { + bitmapIndexSeeker[j] = adapters.get(j).getBitmapIndexSeeker(dimension); + } + + ImmutableBitmap nullRowBitmap = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap( + nullRowsList.get(dimIndex) + ); + if (Iterables.getFirst(dimVals, "") != null && !nullRowsList.get(dimIndex).isEmpty()) { + bitmapIndexWriters.get(dimIndex).write(nullRowBitmap); + } + + for (String dimVal : IndexedIterable.create(dimVals)) { + progress.progress(); + List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); + for (int j = 0; j < adapters.size(); ++j) { + convertedInverteds.add( + new IndexMerger.ConvertingIndexedInts( + bitmapIndexSeeker[j].seek(dimVal), rowNumConversions.get(j) + ) + ); + } + + MutableBitmap bitset = bitmapSerdeFactory.getBitmapFactory().makeEmptyMutableBitmap(); + for (Integer row : CombiningIterable.createSplatted( + convertedInverteds, + Ordering.natural().nullsFirst() + )) { + if (row != INVALID_ROW) { + bitset.add(row); + } + } + + ImmutableBitmap bitmapToWrite = bitmapSerdeFactory.getBitmapFactory().makeImmutableBitmap(bitset); + if (dimVal == null) { + bitmapIndexWriters.get(dimIndex).write(nullRowBitmap.union(bitmapToWrite)); + } else { + bitmapIndexWriters.get(dimIndex).write(bitmapToWrite); + } + + if (spatialIndexWriter != null && dimVal != null) { + List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); + float[] coords = new float[stringCoords.size()]; + for (int j = 0; j < coords.length; j++) { + coords[j] = Float.valueOf(stringCoords.get(j)); + } + tree.insert(coords, bitset); + } + } + if (spatialIndexWriter != null) { + spatialIndexWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); + } + log.info( + "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", + dimension, + dimVals.size(), + System.currentTimeMillis() - dimStartTime + ); + } + log.info("Completed inverted index in %,d millis.", System.currentTimeMillis() - startTime); + progress.stopSection(section); + } + + + private ArrayList> setupBitmapIndexWriters( + final IOPeon ioPeon, + final List mergedDimensions, + final IndexSpec indexSpec + ) throws IOException + { + ArrayList> writers = Lists.newArrayListWithCapacity(mergedDimensions.size()); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + for (String dimension : mergedDimensions) { + GenericIndexedWriter writer = new GenericIndexedWriter<>( + ioPeon, String.format("%s.inverted", dimension), bitmapSerdeFactory.getObjectStrategy() + ); + writer.open(); + writers.add(writer); + } + return writers; + } + + private ArrayList> setupSpatialIndexWriters( + final IOPeon ioPeon, + final List mergedDimensions, + final IndexSpec indexSpec, + final List dimCapabilities + ) throws IOException + { + ArrayList> writers = Lists.newArrayListWithCapacity(mergedDimensions.size()); + final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { + if (dimCapabilities.get(dimIndex).hasSpatialIndexes()) { + BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); + ByteBufferWriter writer = new ByteBufferWriter<>( + ioPeon, + String.format("%s.spatial", mergedDimensions.get(dimIndex)), + new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) + ); + writer.open(); + writers.add(writer); + } else { + writers.add(null); + } + } + return writers; + } + + private void mergeIndexesAndWriteColumns( + final List adapters, + final ProgressIndicator progress, + final Iterable theRows, + final LongColumnSerializer timeWriter, + final ArrayList dimWriters, + final ArrayList metWriters, + final ArrayList dimensionSkipFlag, + final List rowNumConversions, + final ArrayList nullRowsList + ) throws IOException + { + final String section = "walk through and merge rows"; + progress.startSection(section); + long startTime = System.currentTimeMillis(); + + int rowCount = 0; + for (IndexableAdapter adapter : adapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); + } + + long time = System.currentTimeMillis(); + for (Rowboat theRow : theRows) { + progress.progress(); + timeWriter.serialize(theRow.getTimestamp()); + + final Object[] metrics = theRow.getMetrics(); + for (int i = 0; i < metrics.length; ++i) { + metWriters.get(i).serialize(metrics[i]); + } + + int[][] dims = theRow.getDims(); + for (int i = 0; i < dims.length; ++i) { + if (dimensionSkipFlag.get(i)) { + continue; + } + if (dims[i] == null || dims[i].length == 0) { + nullRowsList.get(i).add(rowCount); + } + dimWriters.get(i).add(dims[i]); + } + + for (Map.Entry> comprisedRow : theRow.getComprisedRows().entrySet()) { + final IntBuffer conversionBuffer = rowNumConversions.get(comprisedRow.getKey()); + + for (Integer rowNum : comprisedRow.getValue()) { + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); + } + } + if ((++rowCount % 500000) == 0) { + log.info("walked 500,000/%d rows in %,d millis.", rowCount, System.currentTimeMillis() - time); + time = System.currentTimeMillis(); + } + } + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); + } + log.info("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); + progress.stopSection(section); + } + + private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon) throws IOException + { + LongColumnSerializer timeWriter = LongColumnSerializer.create( + ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ); + // we will close this writer after we added all the timestamps + timeWriter.open(); + return timeWriter; + } + + private ArrayList setupMetricsWriters( + final IOPeon ioPeon, + final List mergedMetrics, + final Map metricsValueTypes, + final Map metricTypeNames, + final IndexSpec indexSpec + ) throws IOException + { + ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); + final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompressionStrategy(); + for (String metric : mergedMetrics) { + ValueType type = metricsValueTypes.get(metric); + GenericColumnSerializer writer; + switch (type) { + case LONG: + writer = LongColumnSerializer.create(ioPeon, metric, metCompression); + break; + case FLOAT: + writer = FloatColumnSerializer.create(ioPeon, metric, metCompression); + break; + case COMPLEX: + final String typeName = metricTypeNames.get(metric); + ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + throw new ISE("Unknown type[%s]", typeName); + } + writer = ComplexColumnSerializer.create(ioPeon, metric, serde); + break; + default: + throw new ISE("Unknown type[%s]", type); + } + writer.open(); + // we will close these writers in another method after we added all the metrics + metWriters.add(writer); + } + return metWriters; + } + + private ArrayList setupDimensionWriters( + final IOPeon ioPeon, + final List mergedDimensions, + final List dimCapabilities, + final Map dimCardinalities, + final IndexSpec indexSpec + ) throws IOException + { + ArrayList dimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); + final CompressedObjectStrategy.CompressionStrategy dimCompression = indexSpec.getDimensionCompressionStrategy(); + for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { + String dim = mergedDimensions.get(dimIndex); + int cardinality = dimCardinalities.get(dim); + ColumnCapabilitiesImpl capabilities = dimCapabilities.get(dimIndex); + String filenameBase = String.format("%s.forward_dim", dim); + IndexedIntsWriter writer; + if (capabilities.hasMultipleValues()) { + writer = (dimCompression != null) + ? CompressedVSizeIndexedV3Writer.create(ioPeon, filenameBase, cardinality, dimCompression) + : new VSizeIndexedWriter(ioPeon, filenameBase, cardinality); + } else { + writer = (dimCompression != null) + ? CompressedVSizeIntsIndexedWriter.create(ioPeon, filenameBase, cardinality, dimCompression) + : new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality); + } + writer.open(); + // we will close these writers in another method after we added all the values + dimWriters.add(writer); + } + return dimWriters; + } + + private Iterable makeRowIterable( + final List adapters, + final List mergedDimensions, + final List mergedMetrics, + final ArrayList> dimConversions, + final Function>, Iterable> rowMergerFn + ) + { + ArrayList> boats = Lists.newArrayListWithCapacity(adapters.size()); + + for (int i = 0; i < adapters.size(); ++i) { + final IndexableAdapter adapter = adapters.get(i); + + final int[] dimLookup = new int[mergedDimensions.size()]; + int count = 0; + for (String dim : adapter.getDimensionNames()) { + dimLookup[count] = mergedDimensions.indexOf(dim); + count++; + } + + final int[] metricLookup = new int[mergedMetrics.size()]; + count = 0; + for (String metric : adapter.getMetricNames()) { + metricLookup[count] = mergedMetrics.indexOf(metric); + count++; + } + + boats.add( + new IndexMerger.MMappedIndexRowIterable( + Iterables.transform( + adapters.get(i).getRows(), + new Function() + { + @Override + public Rowboat apply(Rowboat input) + { + int[][] newDims = new int[mergedDimensions.size()][]; + int j = 0; + for (int[] dim : input.getDims()) { + newDims[dimLookup[j]] = dim; + j++; + } + + Object[] newMetrics = new Object[mergedMetrics.size()]; + j = 0; + for (Object met : input.getMetrics()) { + newMetrics[metricLookup[j]] = met; + j++; + } + + return new Rowboat( + input.getTimestamp(), + newDims, + newMetrics, + input.getRowNum() + ); + } + } + ), + mergedDimensions, + dimConversions.get(i), + i + ) + ); + } + + return rowMergerFn.apply(boats); + } + + private ArrayList> setupDimValueWriters( + final IOPeon ioPeon, + final List mergedDimensions + ) + throws IOException + { + ArrayList> dimValueWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); + for (String dimension : mergedDimensions) { + final GenericIndexedWriter writer = new GenericIndexedWriter<>( + ioPeon, String.format("%s.dim_values", dimension), GenericIndexed.STRING_STRATEGY + ); + writer.open(); + dimValueWriters.add(writer); + } + return dimValueWriters; + } + + private void writeDimValueAndSetupDimConversion( + final List adapters, + final ProgressIndicator progress, + final List mergedDimensions, + final Map dimensionCardinalities, + final ArrayList> dimValueWriters, + final ArrayList dimensionSkipFlag, + final List> dimConversions + ) throws IOException + { + final String section = "setup dimension conversions"; + progress.startSection(section); + + for (int i = 0; i < adapters.size(); ++i) { + dimConversions.add(Maps.newHashMap()); + } + + for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { + long dimStartTime = System.currentTimeMillis(); + String dimension = mergedDimensions.get(dimIndex); + + // lookups for all dimension values of this dimension + List> dimValueLookups = Lists.newArrayListWithCapacity(adapters.size()); + + // each converter converts dim values of this dimension to global dictionary + IndexMerger.DimValueConverter[] converters = new IndexMerger.DimValueConverter[adapters.size()]; + + boolean existNullColumn = false; + for (int i = 0; i < adapters.size(); i++) { + Indexed dimValues = adapters.get(i).getDimValueLookup(dimension); + if (!IndexMerger.isNullColumn(dimValues)) { + dimValueLookups.add(dimValues); + converters[i] = new IndexMerger.DimValueConverter(dimValues); + } else { + existNullColumn = true; + } + } + + Iterable> bumpedDimValueLookups; + if (!dimValueLookups.isEmpty() && existNullColumn) { + log.info("dim[%s] are null in some indexes, append null value to dim values", dimension); + bumpedDimValueLookups = Iterables.concat( + Arrays.asList(new ArrayIndexed<>(new String[]{null}, String.class)), + dimValueLookups + ); + } else { + bumpedDimValueLookups = dimValueLookups; + } + + // sort all dimension values and treat all null values as empty strings + Iterable dimensionValues = CombiningIterable.createSplatted( + Iterables.transform( + bumpedDimValueLookups, + new Function, Iterable>() + { + @Override + public Iterable apply(@Nullable Indexed indexed) + { + return Iterables.transform( + indexed, + new Function() + { + @Override + public String apply(@Nullable String input) + { + return (input == null) ? "" : input; + } + } + ); + } + } + ), Ordering.natural().nullsFirst() + ); + + GenericIndexedWriter writer = dimValueWriters.get(dimIndex); + int cardinality = 0; + for (String value : dimensionValues) { + value = value == null ? "" : value; + writer.write(value); + + for (int i = 0; i < adapters.size(); i++) { + IndexMerger.DimValueConverter converter = converters[i]; + if (converter != null) { + converter.convert(value, cardinality); + } + } + ++cardinality; + } + + log.info( + "Completed dim[%s] conversions with cardinality[%,d] in %,d millis.", + dimension, + cardinality, + System.currentTimeMillis() - dimStartTime + ); + dimensionCardinalities.put(dimension, cardinality); + writer.close(); + + if (cardinality == 0) { + log.info(String.format("Skipping [%s], it is empty!", dimension)); + dimensionSkipFlag.add(true); + continue; + } + dimensionSkipFlag.add(false); + + // make the conversion + for (int i = 0; i < adapters.size(); ++i) { + IndexMerger.DimValueConverter converter = converters[i]; + if (converter != null) { + dimConversions.get(i).put(dimension, converters[i].getConversionBuffer()); + } + } + } + progress.stopSection(section); + } + + private void mergeCapabilities( + final List adapters, + final List mergedDimensions, + final Map metricsValueTypes, + final Map metricTypeNames, + final List dimCapabilities + ) + { + final Map capabilitiesMap = Maps.newHashMap(); + for (IndexableAdapter adapter : adapters) { + for (String dimension : adapter.getDimensionNames()) { + ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(dimension); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + mergedCapabilities.setType(ValueType.STRING); + } + capabilitiesMap.put(dimension, mergedCapabilities.merge(adapter.getCapabilities(dimension))); + } + for (String metric : adapter.getMetricNames()) { + ColumnCapabilitiesImpl mergedCapabilities = capabilitiesMap.get(metric); + ColumnCapabilities capabilities = adapter.getCapabilities(metric); + if (mergedCapabilities == null) { + mergedCapabilities = new ColumnCapabilitiesImpl(); + } + capabilitiesMap.put(metric, mergedCapabilities.merge(capabilities)); + metricsValueTypes.put(metric, capabilities.getType()); + metricTypeNames.put(metric, adapter.getMetricType(metric)); + } + } + for (String dim : mergedDimensions) { + dimCapabilities.add(capabilitiesMap.get(dim)); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java new file mode 100644 index 00000000000..fa03c86d125 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java @@ -0,0 +1,97 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import io.druid.segment.data.CompressedLongsSupplierSerializer; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.IOPeon; + +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class LongColumnSerializer implements GenericColumnSerializer +{ + public static LongColumnSerializer create( + IOPeon ioPeon, + String filenameBase, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + } + + private final IOPeon ioPeon; + private final String filenameBase; + private final ByteOrder byteOrder; + private final CompressedObjectStrategy.CompressionStrategy compression; + private CompressedLongsSupplierSerializer writer; + + public LongColumnSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder byteOrder, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.ioPeon = ioPeon; + this.filenameBase = filenameBase; + this.byteOrder = byteOrder; + this.compression = compression; + } + + @Override + public void open() throws IOException + { + writer = CompressedLongsSupplierSerializer.create( + ioPeon, + String.format("%s.long_column", filenameBase), + byteOrder, + compression + ); + writer.open(); + } + + @Override + public void serialize(Object obj) throws IOException + { + long val = (obj == null) ? 0 : ((Number) obj).longValue(); + writer.add(val); + } + + @Override + public void close() throws IOException + { + writer.close(); + } + + @Override + public long getSerializedSize() + { + return writer.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + writer.writeToChannel(channel); + } + +} diff --git a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java index 3f5c48152c2..b2c0fb9a0d5 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java @@ -79,7 +79,7 @@ public class ColumnDescriptor long retVal = 0; for (ColumnPartSerde part : parts) { - retVal += part.numBytes(); + retVal += part.getSerializer().numBytes(); } return retVal; @@ -88,7 +88,7 @@ public class ColumnDescriptor public void write(WritableByteChannel channel) throws IOException { for (ColumnPartSerde part : parts) { - part.write(channel); + part.getSerializer().write(channel); } } @@ -99,7 +99,7 @@ public class ColumnDescriptor .setHasMultipleValues(hasMultipleValues); for (ColumnPartSerde part : parts) { - part.read(buffer, builder, columnConfig); + part.getDeserializer().read(buffer, builder, columnConfig); } return builder.build(); diff --git a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java index 6df1e244cc6..d8b227e5f0f 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java @@ -30,6 +30,9 @@ import com.google.common.primitives.Ints; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; import java.util.Arrays; /** @@ -85,6 +88,11 @@ public class ByteBufferWriter implements Closeable ); } + public long getSerializedSize() + { + return headerOut.getCount() + valueOut.getCount(); + } + public InputSupplier combineStreams() { return ByteStreams.join( @@ -108,4 +116,10 @@ public class ByteBufferWriter implements Closeable ) ); } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput()); + ByteStreams.copy(from, channel); + } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java index 98ed0ff27d0..31d045ea484 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java @@ -27,22 +27,33 @@ import io.druid.collections.StupidResourceHolder; import java.io.IOException; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.FloatBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; /** */ public class CompressedFloatsSupplierSerializer { public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + IOPeon ioPeon, + final String filenameBase, + final ByteOrder order, + final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order, compression); } public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final int sizePer, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + IOPeon ioPeon, + final String filenameBase, + final int sizePer, + final ByteOrder order, + final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer( @@ -89,7 +100,7 @@ public class CompressedFloatsSupplierSerializer public void add(float value) throws IOException { - if (! endBuffer.hasRemaining()) { + if (!endBuffer.hasRemaining()) { endBuffer.rewind(); flattener.write(StupidResourceHolder.create(endBuffer)); endBuffer = FloatBuffer.allocate(sizePer); @@ -102,13 +113,7 @@ public class CompressedFloatsSupplierSerializer public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = null; - - flattener.close(); - + close(); try (OutputStream out = consolidatedOut.getOutput()) { out.write(CompressedFloatsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); @@ -117,4 +122,32 @@ public class CompressedFloatsSupplierSerializer ByteStreams.copy(flattener.combineStreams(), out); } } + + public void close() throws IOException + { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = null; + flattener.close(); + } + + public long getSerializedSize() + { + return 1 + // version + Ints.BYTES + // elements num + Ints.BYTES + // sizePer + 1 + // compression id + flattener.getSerializedSize(); + } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + ByteStreams.copy(from, channel); + } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java index f33e7ddcfb2..a2c9296f830 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java @@ -38,7 +38,7 @@ import java.util.List; public class CompressedIntsIndexedSupplier implements WritableSupplier { - public static final byte version = 0x2; + public static final byte VERSION = 0x2; public static final int MAX_INTS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Ints.BYTES; @@ -103,7 +103,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier> flattener; private IntBuffer endBuffer; @@ -55,7 +70,6 @@ public class CompressedIntsIndexedWriter ) { this.chunkFactor = chunkFactor; - this.byteOrder = byteOrder; this.compression = compression; this.flattener = new GenericIndexedWriter<>( ioPeon, filenameBase, CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) @@ -64,42 +78,58 @@ public class CompressedIntsIndexedWriter this.numInserted = 0; } + @Override public void open() throws IOException { flattener.open(); } - public void add(int val) throws IOException + @Override + protected void addValue(int val) throws IOException { if (!endBuffer.hasRemaining()) { endBuffer.rewind(); flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer.rewind(); + endBuffer = IntBuffer.allocate(chunkFactor); } endBuffer.put(val); numInserted++; } - public long closeAndWriteToChannel(WritableByteChannel channel) throws IOException + @Override + public void close() throws IOException { - if (numInserted > 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); + try { + if (numInserted > 0) { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = null; } - endBuffer = null; - flattener.close(); + finally { + flattener.close(); + } + } - channel.write(ByteBuffer.wrap(new byte[]{version})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); - long dataLen = ByteStreams.copy(from, channel); + @Override + public long getSerializedSize() + { return 1 + // version Ints.BYTES + // numInserted Ints.BYTES + // chunkFactor 1 + // compression id - dataLen; // data + flattener.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{VERSION})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + ByteStreams.copy(from, channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java index dc963135417..5ea2b20bcc3 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java @@ -27,21 +27,34 @@ import io.druid.collections.StupidResourceHolder; import java.io.IOException; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.LongBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; /** */ public class CompressedLongsSupplierSerializer { public static CompressedLongsSupplierSerializer create( - IOPeon ioPeon, final String filenameBase, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression + final IOPeon ioPeon, + final String filenameBase, + final ByteOrder order, + final CompressedObjectStrategy.CompressionStrategy compression ) throws IOException { final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer( CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER, new GenericIndexedWriter>( - ioPeon, filenameBase, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER) + ioPeon, + filenameBase, + CompressedLongBufferObjectStrategy.getBufferForOrder( + order, + compression, + CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER + ) ), compression ); @@ -82,7 +95,7 @@ public class CompressedLongsSupplierSerializer public void add(long value) throws IOException { - if (! endBuffer.hasRemaining()) { + if (!endBuffer.hasRemaining()) { endBuffer.rewind(); flattener.write(StupidResourceHolder.create(endBuffer)); endBuffer = LongBuffer.allocate(sizePer); @@ -95,13 +108,7 @@ public class CompressedLongsSupplierSerializer public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = null; - - flattener.close(); - + close(); try (OutputStream out = consolidatedOut.getOutput()) { out.write(CompressedLongsIndexedSupplier.version); out.write(Ints.toByteArray(numInserted)); @@ -110,4 +117,31 @@ public class CompressedLongsSupplierSerializer ByteStreams.copy(flattener.combineStreams(), out); } } + + public void close() throws IOException { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer = null; + flattener.close(); + } + + public long getSerializedSize() + { + return 1 + // version + Ints.BYTES + // elements num + Ints.BYTES + // sizePer + 1 + // compression id + flattener.getSerializedSize(); + } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + ByteStreams.copy(from, channel); + } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java index 8b39d44eebb..efcc1cbe064 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java @@ -23,14 +23,45 @@ package io.druid.segment.data; import io.druid.segment.CompressedVSizeIndexedV3Supplier; +import io.druid.segment.IndexIO; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.List; -public class CompressedVSizeIndexedV3Writer +public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter { - public static final byte version = CompressedVSizeIndexedV3Supplier.version; + private static final byte VERSION = CompressedVSizeIndexedV3Supplier.VERSION; + + private static final List EMPTY_LIST = new ArrayList<>(); + + public static CompressedVSizeIndexedV3Writer create( + final IOPeon ioPeon, + final String filenameBase, + final int maxValue, + final CompressedObjectStrategy.CompressionStrategy compression + ) + { + return new CompressedVSizeIndexedV3Writer( + new CompressedIntsIndexedWriter( + ioPeon, + String.format("%s.offsets", filenameBase), + CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, + IndexIO.BYTE_ORDER, + compression + ), + new CompressedVSizeIntsIndexedWriter( + ioPeon, + String.format("%s.values", filenameBase), + maxValue, + CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), + IndexIO.BYTE_ORDER, + compression + ) + ); + } private final CompressedIntsIndexedWriter offsetWriter; private final CompressedVSizeIntsIndexedWriter valueWriter; @@ -46,27 +77,51 @@ public class CompressedVSizeIndexedV3Writer this.offset = 0; } + @Override public void open() throws IOException { offsetWriter.open(); valueWriter.open(); } - public void add(int[] vals) throws IOException + @Override + protected void addValues(List vals) throws IOException { + if (vals == null) { + vals = EMPTY_LIST; + } offsetWriter.add(offset); - for (int val : vals) { + for (Integer val : vals) { valueWriter.add(val); } - offset += vals.length; + offset += vals.size(); } - public long closeAndWriteToChannel(WritableByteChannel channel) throws IOException + @Override + public void close() throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{version})); - offsetWriter.add(offset); - long offsetLen = offsetWriter.closeAndWriteToChannel(channel); - long dataLen = valueWriter.closeAndWriteToChannel(channel); - return 1 + offsetLen + dataLen; + try { + offsetWriter.add(offset); + } + finally { + offsetWriter.close(); + valueWriter.close(); + } + } + + @Override + public long getSerializedSize() + { + return 1 + // version + offsetWriter.getSerializedSize() + + valueWriter.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{VERSION})); + offsetWriter.writeToChannel(channel); + valueWriter.writeToChannel(channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java index 0b1a34b70d1..52488cbbdad 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -40,7 +40,7 @@ import java.util.List; public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier { - public static final byte version = 0x2; + public static final byte VERSION = 0x2; private final int totalSize; private final int sizePer; @@ -132,7 +132,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); + try { + if (numInserted > 0) { + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = null; } - endBuffer = null; - flattener.close(); + finally { + flattener.close(); + } + } - channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); - long dataLen = ByteStreams.copy(from, channel); + @Override + public long getSerializedSize() + { return 1 + // version 1 + // numBytes Ints.BYTES + // numInserted Ints.BYTES + // chunkFactor 1 + // compression id - dataLen; // data + flattener.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); + channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); + channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); + ByteStreams.copy(from, channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index 3dd6ee4cb40..f8c42d7bfbd 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -31,6 +31,9 @@ import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; import java.util.Arrays; /** @@ -120,6 +123,15 @@ public class GenericIndexedWriter implements Closeable } } + public long getSerializedSize() + { + return 2 + // version and sorted flag + Ints.BYTES + // numBytesWritten + Ints.BYTES + // numElements + headerOut.getCount() + // header length + valuesOut.getCount(); // value length + } + public InputSupplier combineStreams() { return ByteStreams.join( @@ -143,4 +155,10 @@ public class GenericIndexedWriter implements Closeable ) ); } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput()); + ByteStreams.copy(from, channel); + } } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java new file mode 100644 index 00000000000..4898351af2c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java @@ -0,0 +1,35 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public interface IndexedIntsWriter extends Closeable +{ + public void open() throws IOException; + + public void add(Object obj) throws IOException; + + public long getSerializedSize(); + + public void writeToChannel(WritableByteChannel channel) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java new file mode 100644 index 00000000000..3aebdc5a058 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.primitives.Ints; +import com.metamx.common.IAE; + +import java.io.IOException; +import java.util.List; + +public abstract class MultiValueIndexedIntsWriter implements IndexedIntsWriter +{ + @Override + public void add(Object obj) throws IOException + { + if (obj == null) { + addValues(null); + } else if (obj instanceof int[]) { + addValues(Ints.asList((int[]) obj)); + } else if (obj instanceof List) { + addValues((List) obj); + } else { + throw new IAE("unsupported multi-value type: " + obj.getClass()); + } + } + + protected abstract void addValues(List vals) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/SingleValueIndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/SingleValueIndexedIntsWriter.java new file mode 100644 index 00000000000..c6caf733039 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/SingleValueIndexedIntsWriter.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.metamx.common.IAE; + +import java.io.IOException; + +public abstract class SingleValueIndexedIntsWriter implements IndexedIntsWriter +{ + @Override + public void add(Object obj) throws IOException + { + if (obj == null) { + addValue(0); + } else if (obj instanceof Integer) { + addValue(((Number) obj).intValue()); + } else if (obj instanceof int[]) { + int[] vals = (int[]) obj; + if (vals.length == 0) { + addValue(0); + } else { + addValue(vals[0]); + } + } else { + throw new IAE("Unsupported single value type: " + obj.getClass()); + } + } + + protected abstract void addValue(int val) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java b/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java index 5682a32a2ae..d8e6c5d39a3 100644 --- a/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java +++ b/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java @@ -34,8 +34,19 @@ import java.util.Map; */ public class TmpFileIOPeon implements IOPeon { + private final boolean allowOverwrite; Map createdFiles = Maps.newLinkedHashMap(); + public TmpFileIOPeon() + { + this(true); + } + + public TmpFileIOPeon(boolean allowOverwrite) + { + this.allowOverwrite = allowOverwrite; + } + @Override public OutputStream makeOutputStream(String filename) throws IOException { @@ -44,8 +55,12 @@ public class TmpFileIOPeon implements IOPeon retFile = File.createTempFile("filePeon", filename); retFile.deleteOnExit(); createdFiles.put(filename, retFile); + return new BufferedOutputStream(new FileOutputStream(retFile)); + } else if (allowOverwrite) { + return new BufferedOutputStream(new FileOutputStream(retFile)); + } else { + throw new IOException("tmp file conflicts, file[" + filename + "] already exist!"); } - return new BufferedOutputStream(new FileOutputStream(retFile)); } @Override @@ -64,4 +79,9 @@ public class TmpFileIOPeon implements IOPeon } createdFiles.clear(); } + + public boolean isOverwriteAllowed() + { + return allowOverwrite; + } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index 079c3b00005..3148b2ac73e 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -33,7 +33,7 @@ import java.util.List; */ public class VSizeIndexedInts implements IndexedInts, Comparable { - private static final byte version = 0x0; + public static final byte VERSION = 0x0; public static VSizeIndexedInts fromArray(int[] array) { @@ -185,7 +185,7 @@ public class VSizeIndexedInts implements IndexedInts, Comparable val) throws IOException + { + write(val); } public void write(List ints) throws IOException @@ -79,11 +92,6 @@ public class VSizeIndexedWriter implements Closeable ++numWritten; } - private String makeFilename(String suffix) - { - return String.format("%s.%s", filenameBase, suffix); - } - @Override public void close() throws IOException { @@ -107,10 +115,10 @@ public class VSizeIndexedWriter implements Closeable numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten ); - OutputStream metaOut = ioPeon.makeOutputStream(makeFilename("meta")); + OutputStream metaOut = ioPeon.makeOutputStream(metaFileName); try { - metaOut.write(new byte[]{version, numBytesForMax}); + metaOut.write(new byte[]{VERSION, numBytesForMax}); metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); metaOut.write(Ints.toByteArray(numWritten)); } @@ -123,7 +131,7 @@ public class VSizeIndexedWriter implements Closeable { return ByteStreams.join( Iterables.transform( - Arrays.asList("meta", "header", "values"), + Arrays.asList(metaFileName, headerFileName, valuesFileName), new Function>() { @Override @@ -134,7 +142,7 @@ public class VSizeIndexedWriter implements Closeable @Override public InputStream getInput() throws IOException { - return ioPeon.makeInputStream(makeFilename(input)); + return ioPeon.makeInputStream(input); } }; } @@ -142,4 +150,22 @@ public class VSizeIndexedWriter implements Closeable ) ); } + + @Override + public long getSerializedSize() + { + return 1 + // version + 1 + // numBytes + 4 + // numBytesWritten + 4 + // numElements + headerOut.getCount() + + valuesOut.getCount(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput()); + ByteStreams.copy(from, channel); + } } diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java index c6c1e33a4ae..3b124f9f0d7 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexAdapter.java @@ -20,13 +20,14 @@ package io.druid.segment.incremental; import com.google.common.base.Function; +import com.google.common.base.Strings; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.metamx.collections.bitmap.BitmapFactory; import com.metamx.collections.bitmap.MutableBitmap; import com.metamx.common.ISE; import com.metamx.common.logger.Logger; - import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; import io.druid.segment.column.BitmapIndexSeeker; @@ -38,16 +39,15 @@ import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.ListIndexed; - import org.joda.time.Interval; import org.roaringbitmap.IntIterator; import javax.annotation.Nullable; - import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; /** */ @@ -57,6 +57,7 @@ public class IncrementalIndexAdapter implements IndexableAdapter private final Interval dataInterval; private final IncrementalIndex index; private final Map> invertedIndexes; + private final Set hasNullValueDimensions; public IncrementalIndexAdapter( Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory @@ -64,8 +65,17 @@ public class IncrementalIndexAdapter implements IndexableAdapter { this.dataInterval = dataInterval; this.index = index; - this.invertedIndexes = Maps.newHashMap(); + /* Sometimes it's hard to tell whether one dimension contains a null value or not. + * If one dimension had show a null or empty value explicitly, then yes, it contains + * null value. But if one dimension's values are all non-null, it still early to say + * this dimension does not contain null value. Consider a two row case, first row had + * "dimA=1" and "dimB=2", the second row only had "dimA=3". To dimB, its value are "2" and + * never showed a null or empty value. But when we combines these two rows, dimB is null + * in row 2. So we should iterate all rows to determine whether one dimension contains + * a null value. + */ + this.hasNullValueDimensions = Sets.newHashSet(); final List dimensions = index.getDimensions(); @@ -86,8 +96,12 @@ public class IncrementalIndexAdapter implements IndexableAdapter continue; } if (dimIndex >= dims.length || dims[dimIndex] == null) { + hasNullValueDimensions.add(dimension.getName()); continue; } + if (hasNullValue(dims[dimIndex])) { + hasNullValueDimensions.add(dimension.getName()); + } for (String dimValue : dims[dimIndex]) { MutableBitmap mutableBitmap = bitmapIndexes.get(dimValue); @@ -138,6 +152,11 @@ public class IncrementalIndexAdapter implements IndexableAdapter public Indexed getDimValueLookup(String dimension) { final IncrementalIndex.DimDim dimDim = index.getDimensionValues(dimension); + if (hasNullValueDimensions.contains(dimension) + && !dimDim.contains(null)) + { + dimDim.add(null); + } dimDim.sort(); return new Indexed() @@ -287,10 +306,13 @@ public class IncrementalIndexAdapter implements IndexableAdapter @Override public IndexedInts seek(String value) { - if (value != null && GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) { - throw new ISE("Value[%s] is less than the last value[%s] I have, cannot be.", - value, lastVal); + if (value != null && GenericIndexed.STRING_STRATEGY.compare(value, lastVal) <= 0) { + throw new ISE( + "Value[%s] is less than the last value[%s] I have, cannot be.", + value, lastVal + ); } + value = Strings.nullToEmpty(value); lastVal = value; final MutableBitmap bitmapIndex = dimInverted.get(value); if (bitmapIndex == null) { @@ -301,7 +323,21 @@ public class IncrementalIndexAdapter implements IndexableAdapter }; } - static class BitmapIndexedInts implements IndexedInts { + private boolean hasNullValue(String[] dimValues) + { + if (dimValues == null || dimValues.length == 0) { + return true; + } + for (String dimVal : dimValues) { + if (Strings.isNullOrEmpty(dimVal)) { + return true; + } + } + return false; + } + + static class BitmapIndexedInts implements IndexedInts + { private final MutableBitmap bitmapIndex; diff --git a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java index 11d78ec840b..09b8ffc54af 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -39,7 +39,19 @@ import java.nio.channels.WritableByteChannel; }) public interface ColumnPartSerde { - public long numBytes(); - public void write(WritableByteChannel channel) throws IOException; - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); + public Serializer getSerializer(); + + public Deserializer getDeserializer(); + + public interface Serializer + { + public long numBytes(); + + public void write(WritableByteChannel channel) throws IOException; + } + + public interface Deserializer + { + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); + } } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index a7fd07e7127..f81d91b4e19 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -30,7 +30,7 @@ import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; /** -*/ + */ public class ComplexColumnPartSerde implements ColumnPartSerde { @JsonCreator @@ -38,19 +38,18 @@ public class ComplexColumnPartSerde implements ColumnPartSerde @JsonProperty("typeName") String complexType ) { - return new ComplexColumnPartSerde(null, complexType); + return new ComplexColumnPartSerde(complexType, null); } - private final GenericIndexed column; private final String typeName; - private final ComplexMetricSerde serde; + private final Serializer serializer; - public ComplexColumnPartSerde(GenericIndexed column, String typeName) + private ComplexColumnPartSerde(String typeName, Serializer serializer) { - this.column = column; this.typeName = typeName; - serde = ComplexMetrics.getSerdeForType(typeName); + this.serde = ComplexMetrics.getSerdeForType(typeName); + this.serializer = serializer; } @JsonProperty @@ -59,21 +58,110 @@ public class ComplexColumnPartSerde implements ColumnPartSerde return typeName; } - @Override - public long numBytes() + public static SerializerBuilder serializerBuilder() { - return column.getSerializedSize(); + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private String typeName = null; + private ComplexColumnSerializer delegate = null; + + public SerializerBuilder withTypeName(final String typeName) + { + this.typeName = typeName; + return this; + } + + public SerializerBuilder withDelegate(final ComplexColumnSerializer delegate) + { + this.delegate = delegate; + return this; + } + + public ComplexColumnPartSerde build() + { + return new ComplexColumnPartSerde( + typeName, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } + } + + public static LegacySerializerBuilder legacySerializerBuilder() + { + return new LegacySerializerBuilder(); + } + + public static class LegacySerializerBuilder + { + private String typeName = null; + private GenericIndexed delegate = null; + + public LegacySerializerBuilder withTypeName(final String typeName) + { + this.typeName = typeName; + return this; + } + + public LegacySerializerBuilder withDelegate(final GenericIndexed delegate) + { + this.delegate = delegate; + return this; + } + + public ComplexColumnPartSerde build() + { + return new ComplexColumnPartSerde( + typeName, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } } @Override - public void write(WritableByteChannel channel) throws IOException + public Serializer getSerializer() { - column.writeToChannel(channel); + return serializer; } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public Deserializer getDeserializer() { - return serde == null ? this : serde.deserializeColumn(buffer, builder); + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + if (serde != null) { + serde.deserializeColumn(buffer, builder); + } + } + }; } } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java new file mode 100644 index 00000000000..3498a979f24 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java @@ -0,0 +1,90 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.serde; + +import io.druid.segment.GenericColumnSerializer; +import io.druid.segment.data.GenericIndexedWriter; +import io.druid.segment.data.IOPeon; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public class ComplexColumnSerializer implements GenericColumnSerializer +{ + public static ComplexColumnSerializer create( + IOPeon ioPeon, + String filenameBase, + ComplexMetricSerde serde + ) + { + return new ComplexColumnSerializer(ioPeon, filenameBase, serde); + } + + private final IOPeon ioPeon; + private final String filenameBase; + private final ComplexMetricSerde serde; + private GenericIndexedWriter writer; + + public ComplexColumnSerializer( + IOPeon ioPeon, + String filenameBase, + ComplexMetricSerde serde + ) + { + this.ioPeon = ioPeon; + this.filenameBase = filenameBase; + this.serde = serde; + } + + @SuppressWarnings(value = "unchecked") + @Override + public void open() throws IOException + { + writer = new GenericIndexedWriter( + ioPeon, String.format("%s.complex_column", filenameBase), serde.getObjectStrategy() + ); + writer.open(); + } + + @SuppressWarnings(value = "unchecked") + @Override + public void serialize(Object obj) throws IOException + { + writer.write(obj); + } + + @Override + public void close() throws IOException + { + writer.close(); + } + + @Override + public long getSerializedSize() + { + return writer.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + writer.writeToChannel(channel); + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java index 75466c87bda..a55b7d78a4e 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java @@ -36,14 +36,10 @@ public abstract class ComplexMetricSerde * Deserializes a ByteBuffer and adds it to the ColumnBuilder. This method allows for the ComplexMetricSerde * to implement it's own versioning scheme to allow for changes of binary format in a forward-compatible manner. * - * The method is also in charge of returning a ColumnPartSerde that knows how to serialize out the object it - * added to the builder. - * * @param buffer the buffer to deserialize * @param builder ColumnBuilder to add the column to - * @return a ColumnPartSerde that can serialize out the object that was read from the buffer to the builder */ - public abstract ColumnPartSerde deserializeColumn(ByteBuffer buffer, ColumnBuilder builder); + public abstract void deserializeColumn(ByteBuffer buffer, ColumnBuilder builder); /** * This is deprecated because its usage is going to be removed from the code. diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 60315f4744c..43cb2fc9753 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -27,15 +27,18 @@ import com.metamx.collections.bitmap.ImmutableBitmap; import com.metamx.collections.spatial.ImmutableRTree; import com.metamx.common.IAE; import io.druid.segment.CompressedVSizeIndexedSupplier; +import io.druid.segment.CompressedVSizeIndexedV3Supplier; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; -import io.druid.segment.data.BitmapSerde; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferSerializer; +import io.druid.segment.data.ByteBufferWriter; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; import io.druid.segment.data.GenericIndexed; +import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.IndexedIntsWriter; import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.IndexedRTree; import io.druid.segment.data.VSizeIndexed; @@ -56,7 +59,8 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde enum Feature { - MULTI_VALUE; + MULTI_VALUE, + MULTI_VALUE_V3; public boolean isSet(int flags) { return (getMask() & flags) != 0; } @@ -83,7 +87,169 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde } } - public static class Builder + @JsonCreator + public static DictionaryEncodedColumnPartSerde createDeserializer( + @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory, + @NotNull @JsonProperty("byteOrder") ByteOrder byteOrder + ) + { + return new DictionaryEncodedColumnPartSerde(byteOrder, bitmapSerdeFactory, null); + } + + private final ByteOrder byteOrder; + private final BitmapSerdeFactory bitmapSerdeFactory; + private final Serializer serializer; + + private DictionaryEncodedColumnPartSerde( + ByteOrder byteOrder, + BitmapSerdeFactory bitmapSerdeFactory, + Serializer serializer + ) + { + this.byteOrder = byteOrder; + this.bitmapSerdeFactory = bitmapSerdeFactory; + this.serializer = serializer; + } + + @JsonProperty + public BitmapSerdeFactory getBitmapSerdeFactory() + { + return bitmapSerdeFactory; + } + + @JsonProperty + public ByteOrder getByteOrder() + { + return byteOrder; + } + + public static SerializerBuilder serializerBuilder() + { + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private VERSION version = null; + private int flags = NO_FLAGS; + private GenericIndexedWriter dictionaryWriter = null; + private IndexedIntsWriter valueWriter = null; + private BitmapSerdeFactory bitmapSerdeFactory = null; + private GenericIndexedWriter bitmapIndexWriter = null; + private ByteBufferWriter spatialIndexWriter = null; + private ByteOrder byteOrder = null; + + public SerializerBuilder withDictionary(GenericIndexedWriter dictionaryWriter) + { + this.dictionaryWriter = dictionaryWriter; + return this; + } + + public SerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + { + this.bitmapSerdeFactory = bitmapSerdeFactory; + return this; + } + + public SerializerBuilder withBitmapIndex(GenericIndexedWriter bitmapIndexWriter) + { + this.bitmapIndexWriter = bitmapIndexWriter; + return this; + } + + public SerializerBuilder withSpatialIndex(ByteBufferWriter spatialIndexWriter) + { + this.spatialIndexWriter = spatialIndexWriter; + return this; + } + + public SerializerBuilder withByteOrder(ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withValue(IndexedIntsWriter valueWriter, boolean hasMultiValue, boolean compressed) + { + this.valueWriter = valueWriter; + if (hasMultiValue) { + if (compressed) { + this.version = VERSION.COMPRESSED; + this.flags |= Feature.MULTI_VALUE_V3.getMask(); + } else { + this.version = VERSION.UNCOMPRESSED_MULTI_VALUE; + this.flags |= Feature.MULTI_VALUE.getMask(); + } + } else { + if (compressed) { + this.version = VERSION.COMPRESSED; + } else { + this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE; + } + } + return this; + } + + public DictionaryEncodedColumnPartSerde build() + { + return new DictionaryEncodedColumnPartSerde( + byteOrder, + bitmapSerdeFactory, + new Serializer() + { + @Override + public long numBytes() + { + long size = 1 + // version + (version.compareTo(VERSION.COMPRESSED) >= 0 + ? Ints.BYTES + : 0); // flag if version >= compressed + if (dictionaryWriter != null) { + size += dictionaryWriter.getSerializedSize(); + } + if (valueWriter != null) { + size += valueWriter.getSerializedSize(); + } + if (bitmapIndexWriter != null) { + size += bitmapIndexWriter.getSerializedSize(); + } + if (spatialIndexWriter != null) { + size += spatialIndexWriter.getSerializedSize(); + } + return size; + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version.asByte()})); + if (version.compareTo(VERSION.COMPRESSED) >= 0) { + channel.write(ByteBuffer.wrap(Ints.toByteArray(flags))); + } + if (dictionaryWriter != null) { + dictionaryWriter.writeToChannel(channel); + } + if (valueWriter != null) { + valueWriter.writeToChannel(channel); + } + if (bitmapIndexWriter != null) { + bitmapIndexWriter.writeToChannel(channel); + } + if (spatialIndexWriter != null) { + spatialIndexWriter.writeToChannel(channel); + } + } + } + ); + } + } + + public static LegacySerializerBuilder legacySerializerBuilder() + { + return new LegacySerializerBuilder(); + } + + public static class LegacySerializerBuilder { private VERSION version = null; private int flags = NO_FLAGS; @@ -95,41 +261,41 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde private ImmutableRTree spatialIndex = null; private ByteOrder byteOrder = null; - private Builder() + private LegacySerializerBuilder() { } - public Builder withDictionary(GenericIndexed dictionary) + public LegacySerializerBuilder withDictionary(GenericIndexed dictionary) { this.dictionary = dictionary; return this; } - public Builder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) + public LegacySerializerBuilder withBitmapSerdeFactory(BitmapSerdeFactory bitmapSerdeFactory) { this.bitmapSerdeFactory = bitmapSerdeFactory; return this; } - public Builder withBitmaps(GenericIndexed bitmaps) + public LegacySerializerBuilder withBitmaps(GenericIndexed bitmaps) { this.bitmaps = bitmaps; return this; } - public Builder withSpatialIndex(ImmutableRTree spatialIndex) + public LegacySerializerBuilder withSpatialIndex(ImmutableRTree spatialIndex) { this.spatialIndex = spatialIndex; return this; } - public Builder withByteOrder(ByteOrder byteOrder) + public LegacySerializerBuilder withByteOrder(ByteOrder byteOrder) { this.byteOrder = byteOrder; return this; } - public Builder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn) + public LegacySerializerBuilder withSingleValuedColumn(VSizeIndexedInts singleValuedColumn) { Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE; @@ -137,7 +303,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde return this; } - public Builder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn) + public LegacySerializerBuilder withSingleValuedColumn(CompressedVSizeIntsIndexedSupplier singleValuedColumn) { Preconditions.checkState(multiValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); this.version = VERSION.COMPRESSED; @@ -145,7 +311,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde return this; } - public Builder withMultiValuedColumn(VSizeIndexed multiValuedColumn) + public LegacySerializerBuilder withMultiValuedColumn(VSizeIndexed multiValuedColumn) { Preconditions.checkState(singleValuedColumn == null, "Cannot set both multiValuedColumn and singleValuedColumn"); this.version = VERSION.UNCOMPRESSED_MULTI_VALUE; @@ -154,7 +320,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde return this; } - public Builder withMultiValuedColumn(CompressedVSizeIndexedSupplier multiValuedColumn) + public LegacySerializerBuilder withMultiValuedColumn(CompressedVSizeIndexedSupplier multiValuedColumn) { Preconditions.checkState(singleValuedColumn == null, "Cannot set both singleValuedColumn and multiValuedColumn"); this.version = VERSION.COMPRESSED; @@ -172,251 +338,171 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde ); return new DictionaryEncodedColumnPartSerde( - version, - flags, - dictionary, - singleValuedColumn, - multiValuedColumn, + byteOrder, bitmapSerdeFactory, - bitmaps, - spatialIndex, - byteOrder + new Serializer() + { + @Override + public long numBytes() + { + long size = 1 + // version + (version.compareTo(VERSION.COMPRESSED) >= 0 ? Ints.BYTES : 0);// flag if version >= compressed + + size += dictionary.getSerializedSize(); + + if (Feature.MULTI_VALUE.isSet(flags)) { + size += multiValuedColumn.getSerializedSize(); + } else { + size += singleValuedColumn.getSerializedSize(); + } + + size += bitmaps.getSerializedSize(); + if (spatialIndex != null) { + size += spatialIndex.size() + Ints.BYTES; + } + return size; + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + channel.write(ByteBuffer.wrap(new byte[]{version.asByte()})); + if (version.compareTo(VERSION.COMPRESSED) >= 0) { + channel.write(ByteBuffer.wrap(Ints.toByteArray(flags))); + } + + if (dictionary != null) { + dictionary.writeToChannel(channel); + } + + if (Feature.MULTI_VALUE.isSet(flags)) { + if (multiValuedColumn != null) { + multiValuedColumn.writeToChannel(channel); + } + } else { + if (singleValuedColumn != null) { + singleValuedColumn.writeToChannel(channel); + } + } + + if (bitmaps != null) { + bitmaps.writeToChannel(channel); + } + + if (spatialIndex != null) { + ByteBufferSerializer.writeToChannel( + spatialIndex, + new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()), + channel + ); + } + } + } ); } - - - } - - public static Builder builder() - { - return new Builder(); - } - - private final BitmapSerdeFactory bitmapSerdeFactory; - private final ByteOrder byteOrder; - - private final GenericIndexed dictionary; - private final WritableSupplier singleValuedColumn; - private final WritableSupplier> multiValuedColumn; - private final GenericIndexed bitmaps; - private final ImmutableRTree spatialIndex; - private final int flags; - private final VERSION version; - private final long size; - - - @JsonCreator - public DictionaryEncodedColumnPartSerde( - @Nullable @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory, - @NotNull @JsonProperty("byteOrder") ByteOrder byteOrder - ) - { - this.bitmapSerdeFactory = bitmapSerdeFactory == null - ? new BitmapSerde.LegacyBitmapSerdeFactory() - : bitmapSerdeFactory; - this.byteOrder = byteOrder; - - // dummy values - this.dictionary = null; - this.singleValuedColumn = null; - this.multiValuedColumn = null; - this.bitmaps = null; - this.spatialIndex = null; - this.size = -1; - this.flags = 0; - this.version = VERSION.COMPRESSED; - } - - private DictionaryEncodedColumnPartSerde( - VERSION version, - int flags, - GenericIndexed dictionary, - WritableSupplier singleValuedColumn, - WritableSupplier> multiValuedColumn, - BitmapSerdeFactory bitmapSerdeFactory, - GenericIndexed bitmaps, - ImmutableRTree spatialIndex, - ByteOrder byteOrder - ) - { - Preconditions.checkArgument(version.compareTo(VERSION.COMPRESSED) <= 0, "Unsupported version[%s]", version); - - this.bitmapSerdeFactory = bitmapSerdeFactory; - this.byteOrder = byteOrder; - - this.version = version; - this.flags = flags; - - this.dictionary = dictionary; - this.singleValuedColumn = singleValuedColumn; - this.multiValuedColumn = multiValuedColumn; - this.bitmaps = bitmaps; - this.spatialIndex = spatialIndex; - - long size = dictionary.getSerializedSize(); - - if (Feature.MULTI_VALUE.isSet(flags)) { - size += multiValuedColumn.getSerializedSize(); - } else { - size += singleValuedColumn.getSerializedSize(); - } - - size += bitmaps.getSerializedSize(); - if (spatialIndex != null) { - size += spatialIndex.size() + Ints.BYTES; - } - - this.size = size; - } - - @JsonProperty - public BitmapSerdeFactory getBitmapSerdeFactory() - { - return bitmapSerdeFactory; - } - - @JsonProperty - public ByteOrder getByteOrder() - { - return byteOrder; } @Override - public void write(WritableByteChannel channel) throws IOException + public Serializer getSerializer() { - channel.write(ByteBuffer.wrap(new byte[]{version.asByte()})); - if (version.compareTo(VERSION.COMPRESSED) >= 0) { - channel.write(ByteBuffer.wrap(Ints.toByteArray(flags))); - } + return serializer; + } - if (dictionary != null) { - dictionary.writeToChannel(channel); - } + @Override + public Deserializer getDeserializer() + { + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final VERSION rVersion = VERSION.fromByte(buffer.get()); + final int rFlags; - if (Feature.MULTI_VALUE.isSet(flags)) { - if (multiValuedColumn != null) { - multiValuedColumn.writeToChannel(channel); + if (rVersion.compareTo(VERSION.COMPRESSED) >= 0) { + rFlags = buffer.getInt(); + } else { + rFlags = rVersion.equals(VERSION.UNCOMPRESSED_MULTI_VALUE) + ? Feature.MULTI_VALUE.getMask() + : NO_FLAGS; + } + + final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags) || Feature.MULTI_VALUE_V3.isSet(rFlags); + + final GenericIndexed rDictionary = GenericIndexed.read(buffer, GenericIndexed.STRING_STRATEGY); + builder.setType(ValueType.STRING); + + final WritableSupplier rSingleValuedColumn; + final WritableSupplier> rMultiValuedColumn; + + if (hasMultipleValues) { + rMultiValuedColumn = readMultiValuedColum(rVersion, buffer, rFlags); + rSingleValuedColumn = null; + } else { + rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer); + rMultiValuedColumn = null; + } + + builder.setHasMultipleValues(hasMultipleValues) + .setDictionaryEncodedColumn( + new DictionaryEncodedColumnSupplier( + rDictionary, + rSingleValuedColumn, + rMultiValuedColumn, + columnConfig.columnCacheSizeBytes() + ) + ); + + GenericIndexed rBitmaps = GenericIndexed.read( + buffer, bitmapSerdeFactory.getObjectStrategy() + ); + builder.setBitmapIndex( + new BitmapIndexColumnPartSupplier( + bitmapSerdeFactory.getBitmapFactory(), + rBitmaps, + rDictionary + ) + ); + + ImmutableRTree rSpatialIndex = null; + if (buffer.hasRemaining()) { + rSpatialIndex = ByteBufferSerializer.read( + buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) + ); + builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex)); + } } - } else { - if (singleValuedColumn != null) { - singleValuedColumn.writeToChannel(channel); + + + private WritableSupplier readSingleValuedColumn(VERSION version, ByteBuffer buffer) + { + switch (version) { + case UNCOMPRESSED_SINGLE_VALUE: + return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier(); + case COMPRESSED: + return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder); + } + throw new IAE("Unsupported single-value version[%s]", version); } - } - if (bitmaps != null) { - bitmaps.writeToChannel(channel); - } - - if (spatialIndex != null) { - ByteBufferSerializer.writeToChannel( - spatialIndex, - new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()), - channel - ); - } - } - - @Override - public ColumnPartSerde read( - ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig - ) - { - final VERSION rVersion = VERSION.fromByte(buffer.get()); - final int rFlags; - - if (rVersion.compareTo(VERSION.COMPRESSED) >= 0) { - rFlags = buffer.getInt(); - } else { - rFlags = rVersion.equals(VERSION.UNCOMPRESSED_MULTI_VALUE) ? - Feature.MULTI_VALUE.getMask() : - NO_FLAGS; - } - - final boolean hasMultipleValues = Feature.MULTI_VALUE.isSet(rFlags); - - final GenericIndexed rDictionary = GenericIndexed.read(buffer, GenericIndexed.STRING_STRATEGY); - builder.setType(ValueType.STRING); - - final WritableSupplier rSingleValuedColumn; - final WritableSupplier> rMultiValuedColumn; - - if (hasMultipleValues) { - rMultiValuedColumn = readMultiValuedColum(rVersion, buffer); - rSingleValuedColumn = null; - } else { - rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer); - rMultiValuedColumn = null; - } - - builder.setHasMultipleValues(hasMultipleValues) - .setDictionaryEncodedColumn( - new DictionaryEncodedColumnSupplier( - rDictionary, - rSingleValuedColumn, - rMultiValuedColumn, - columnConfig.columnCacheSizeBytes() - ) - ); - - GenericIndexed rBitmaps = GenericIndexed.read( - buffer, bitmapSerdeFactory.getObjectStrategy() - ); - builder.setBitmapIndex( - new BitmapIndexColumnPartSupplier( - bitmapSerdeFactory.getBitmapFactory(), - rBitmaps, - rDictionary - ) - ); - - ImmutableRTree rSpatialIndex = null; - if (buffer.hasRemaining()) { - rSpatialIndex = ByteBufferSerializer.read( - buffer, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) - ); - builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex)); - } - - return new DictionaryEncodedColumnPartSerde( - rVersion, - rFlags, - rDictionary, - rSingleValuedColumn, - rMultiValuedColumn, - bitmapSerdeFactory, - rBitmaps, - rSpatialIndex, - byteOrder - ); - } - - private WritableSupplier readSingleValuedColumn(VERSION version, ByteBuffer buffer) - { - switch (version) { - case UNCOMPRESSED_SINGLE_VALUE: - return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier(); - case COMPRESSED: - return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder); - } - throw new IAE("Unsupported single-value version[%s]", version); - } - - private WritableSupplier> readMultiValuedColum(VERSION version, ByteBuffer buffer) - { - switch (version) { - case UNCOMPRESSED_MULTI_VALUE: - return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier(); - case COMPRESSED: - return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder); - } - throw new IAE("Unsupported multi-value version[%s]", version); - } - - @Override - public long numBytes() - { - return 1 + // version - (version.compareTo(VERSION.COMPRESSED) >= 0 ? Ints.BYTES : 0) + // flag if version >= compressed - size; // size of everything else (dictionary, bitmaps, column, spatialIndex) + private WritableSupplier> readMultiValuedColum( + VERSION version, ByteBuffer buffer, int flags + ) + { + switch (version) { + case UNCOMPRESSED_MULTI_VALUE: + return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier(); + case COMPRESSED: + if (Feature.MULTI_VALUE.isSet(flags)) { + return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder); + } else if (Feature.MULTI_VALUE_V3.isSet(flags)) { + return CompressedVSizeIndexedV3Supplier.fromByteBuffer(buffer, byteOrder); + } else { + throw new IAE("Unrecognized multi-value flag[%d]", flags); + } + } + throw new IAE("Unsupported multi-value version[%s]", version); + } + }; } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index 063dc921f62..503b460027e 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -21,6 +21,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.FloatColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; @@ -32,7 +33,7 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; /** -*/ + */ public class FloatGenericColumnPartSerde implements ColumnPartSerde { @JsonCreator @@ -40,16 +41,16 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde @JsonProperty("byteOrder") ByteOrder byteOrder ) { - return new FloatGenericColumnPartSerde(null, byteOrder); + return new FloatGenericColumnPartSerde(byteOrder, null); } - private final CompressedFloatsIndexedSupplier compressedFloats; private final ByteOrder byteOrder; + private Serializer serializer; - public FloatGenericColumnPartSerde(CompressedFloatsIndexedSupplier compressedFloats, ByteOrder byteOrder) + private FloatGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { - this.compressedFloats = compressedFloats; this.byteOrder = byteOrder; + this.serializer = serializer; } @JsonProperty @@ -58,27 +59,114 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde return byteOrder; } - @Override - public long numBytes() + public static SerializerBuilder serializerBuilder() { - return compressedFloats.getSerializedSize(); + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private FloatColumnSerializer delegate = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final FloatColumnSerializer delegate) + { + this.delegate = delegate; + return this; + } + + public FloatGenericColumnPartSerde build() + { + return new FloatGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } + } + + public static LegacySerializerBuilder legacySerializerBuilder() + { + return new LegacySerializerBuilder(); + } + + public static class LegacySerializerBuilder + { + private ByteOrder byteOrder = null; + private CompressedFloatsIndexedSupplier delegate = null; + + public LegacySerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public LegacySerializerBuilder withDelegate(final CompressedFloatsIndexedSupplier delegate) + { + this.delegate = delegate; + return this; + } + + public FloatGenericColumnPartSerde build() + { + return new FloatGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } } @Override - public void write(WritableByteChannel channel) throws IOException + public Serializer getSerializer() { - compressedFloats.writeToChannel(channel); + return serializer; } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public Deserializer getDeserializer() { - final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, byteOrder); - - builder.setType(ValueType.FLOAT) - .setHasMultipleValues(false) - .setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder)); - - return new FloatGenericColumnPartSerde(column, byteOrder); + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer( + buffer, + byteOrder + ); + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder)); + } + }; } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index db9afe61128..79091fca853 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -21,6 +21,7 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.LongColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; @@ -32,7 +33,7 @@ import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; /** -*/ + */ public class LongGenericColumnPartSerde implements ColumnPartSerde { @JsonCreator @@ -40,16 +41,16 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde @JsonProperty("byteOrder") ByteOrder byteOrder ) { - return new LongGenericColumnPartSerde(null, byteOrder); + return new LongGenericColumnPartSerde(byteOrder, null); } - private final CompressedLongsIndexedSupplier compressedLongs; private final ByteOrder byteOrder; + private Serializer serializer; - public LongGenericColumnPartSerde(CompressedLongsIndexedSupplier compressedLongs, ByteOrder byteOrder) + private LongGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { - this.compressedLongs = compressedLongs; this.byteOrder = byteOrder; + this.serializer = serializer; } @JsonProperty @@ -58,27 +59,114 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde return byteOrder; } - @Override - public long numBytes() + public static SerializerBuilder serializerBuilder() { - return compressedLongs.getSerializedSize(); + return new SerializerBuilder(); + } + + public static class SerializerBuilder + { + private ByteOrder byteOrder = null; + private LongColumnSerializer delegate = null; + + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public SerializerBuilder withDelegate(final LongColumnSerializer delegate) + { + this.delegate = delegate; + return this; + } + + public LongGenericColumnPartSerde build() + { + return new LongGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } + } + + public static LegacySerializerBuilder legacySerializerBuilder() + { + return new LegacySerializerBuilder(); + } + + public static class LegacySerializerBuilder + { + private ByteOrder byteOrder = null; + private CompressedLongsIndexedSupplier delegate = null; + + public LegacySerializerBuilder withByteOrder(final ByteOrder byteOrder) + { + this.byteOrder = byteOrder; + return this; + } + + public LegacySerializerBuilder withDelegate(final CompressedLongsIndexedSupplier delegate) + { + this.delegate = delegate; + return this; + } + + public LongGenericColumnPartSerde build() + { + return new LongGenericColumnPartSerde( + byteOrder, new Serializer() + { + @Override + public long numBytes() + { + return delegate.getSerializedSize(); + } + + @Override + public void write(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } + } + ); + } } @Override - public void write(WritableByteChannel channel) throws IOException + public Serializer getSerializer() { - compressedLongs.writeToChannel(channel); + return serializer; } @Override - public ColumnPartSerde read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public Deserializer getDeserializer() { - final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, byteOrder); - - builder.setType(ValueType.LONG) - .setHasMultipleValues(false) - .setGenericColumn(new LongGenericColumnSupplier(column)); - - return new LongGenericColumnPartSerde(column, byteOrder); + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer( + buffer, + byteOrder + ); + builder.setType(ValueType.LONG) + .setHasMultipleValues(false) + .setGenericColumn(new LongGenericColumnSupplier(column)); + } + }; } -} +} \ No newline at end of file diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index a160af199f9..53ed45453fe 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -37,6 +37,7 @@ import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; +import io.druid.segment.data.IndexedInts; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -54,6 +55,7 @@ import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Iterator; import java.util.List; import java.util.Map; @@ -63,7 +65,7 @@ public class IndexMergerTest @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + protected final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); @Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}") @@ -74,10 +76,12 @@ public class IndexMergerTest ImmutableList.of( ImmutableSet.of(new RoaringBitmapSerdeFactory(), new ConciseBitmapSerdeFactory()), ImmutableSet.of( + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, CompressedObjectStrategy.CompressionStrategy.LZ4, CompressedObjectStrategy.CompressionStrategy.LZF ), ImmutableSet.of( + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, CompressedObjectStrategy.CompressionStrategy.LZ4, CompressedObjectStrategy.CompressionStrategy.LZF ) @@ -151,6 +155,119 @@ public class IndexMergerTest assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); } + @Test + public void testPersistWithDifferentDims() throws Exception + { + IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null); + toPersist.add( + new MapBasedInputRow( + 1, + Arrays.asList("dim1", "dim2"), + ImmutableMap.of("dim1", "1", "dim2", "2") + ) + ); + toPersist.add( + new MapBasedInputRow( + 1, + Arrays.asList("dim1"), + ImmutableMap.of("dim1", "3") + ) + ); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist, + tempDir, + null, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(3, index.getColumnNames().size()); + assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); + Iterable boats = adapter.getRows(); + List boatList = new ArrayList<>(); + for (Rowboat boat : boats) { + boatList.add(boat); + } + + Assert.assertEquals(2, boatList.size()); + Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); + Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(1).getDims()); + + checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "1")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim1", "3")); + + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim2", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "2")); + } + + @Test + public void testMergeRetainsValues() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File mergedDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tempDir1, + null, + indexSpec + ) + ) + ); + + + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(3, index1.getColumnNames().size()); + + + QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + ImmutableList.of(index1), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + mergedDir, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); + Assert.assertEquals(3, merged.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + } + @Test public void testPersistWithSegmentMetadata() throws Exception { @@ -345,65 +462,6 @@ public class IndexMergerTest assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); } - @Test - public void testMergeRetainsValues() throws Exception - { - final long timestamp = System.currentTimeMillis(); - IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); - IncrementalIndexTest.populateIndex(timestamp, toPersist1); - - final File tempDir1 = temporaryFolder.newFolder(); - final File mergedDir = temporaryFolder.newFolder(); - final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( - toPersist1.getInterval(), - toPersist1, - indexSpec.getBitmapSerdeFactory() - .getBitmapFactory() - ); - - QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( - INDEX_MERGER.persist( - toPersist1, - tempDir1, - null, - indexSpec - ) - ) - ); - - - final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); - - Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); - Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); - Assert.assertEquals(3, index1.getColumnNames().size()); - - - QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex( - ImmutableList.of(index1), - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - mergedDir, - indexSpec - ) - ) - ); - - Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); - Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); - Assert.assertEquals(3, merged.getColumnNames().size()); - - INDEX_IO.validateTwoSegments(tempDir1, mergedDir); - - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); - } - - @Test public void testAppendRetainsValues() throws Exception { @@ -656,6 +714,9 @@ public class IndexMergerTest throws Exception { // Java voodoo + if (expectedStrategy == null || expectedStrategy == CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { + return; + } Object encodedColumn = index.getColumn("dim2").getDictionaryEncoding(); Field field = SimpleDictionaryEncodedColumn.class.getDeclaredField("column"); @@ -666,6 +727,7 @@ public class IndexMergerTest compressedSupplierField.setAccessible(true); Object supplier = compressedSupplierField.get(obj); + Field compressionField = supplier.getClass().getDeclaredField("compression"); compressionField.setAccessible(true); @@ -811,6 +873,171 @@ public class IndexMergerTest Assert.assertArrayEquals(new Object[]{1L}, boatList.get(3).getMetrics()); Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(4).getDims()); Assert.assertArrayEquals(new Object[]{1L}, boatList.get(4).getMetrics()); + + checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("dimA", "")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Lists.newArrayList(4), adapter.getBitmapIndex("dimA", "2")); + + checkBitmapIndex(Lists.newArrayList(3, 4), adapter.getBitmapIndex("dimB", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimB", "1")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimB", "2")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimB", "3")); + } + + @Test + public void testJointDimMerge() throws Exception + { + // (d1, d2, d3) from only one index, and their dim values are ('empty', 'has null', 'no null') + // (d4, d5, d6, d7, d8, d9) are from both indexes + // d4: 'empty' join 'empty' + // d5: 'empty' join 'has null' + // d6: 'empty' join 'no null' + // d7: 'has null' join 'has null' + // d8: 'has null' join 'no null' + // d9: 'no null' join 'no null' + + IncrementalIndex toPersistA = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + toPersistA.add( + new MapBasedInputRow( + 1, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" + ) + ) + ); + toPersistA.add( + new MapBasedInputRow( + 2, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911" + ) + ) + ); + + IncrementalIndex toPersistB = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + toPersistB.add( + new MapBasedInputRow( + 3, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920" + ) + ) + ); + toPersistB.add( + new MapBasedInputRow( + 4, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921" + ) + ) + ); + final File tmpDirA = temporaryFolder.newFolder(); + final File tmpDirB = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); + + QueryableIndex indexA = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistA, + tmpDirA, + null, + indexSpec + ) + ) + ); + + QueryableIndex indexB = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistB, + tmpDirB, + null, + indexSpec + ) + ) + ); + + final QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + Arrays.asList(indexA, indexB), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec + ) + ) + ); + + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + Iterable boats = adapter.getRows(); + List boatList = new ArrayList<>(); + for (Rowboat boat : boats) { + boatList.add(boat); + } + + Assert.assertEquals( + ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames()) + ); + Assert.assertEquals(4, boatList.size()); + Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}, {0}, {0}, {0}, {0}}, boatList.get(0).getDims()); + Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}, {0}, {1}, {1}, {1}}, boatList.get(1).getDims()); + Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}, {1}, {2}, {2}, {2}}, boatList.get(2).getDims()); + Assert.assertArrayEquals(new int[][]{{0}, {0}, {0}, {2}, {0}, {3}, {3}}, boatList.get(3).getDims()); + + checkBitmapIndex(Lists.newArrayList(0, 2, 3), adapter.getBitmapIndex("d2", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "210")); + + checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("d3", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "310")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "311")); + + checkBitmapIndex(Lists.newArrayList(0, 1, 3), adapter.getBitmapIndex("d5", "")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d5", "520")); + + checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("d6", "")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d6", "620")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621")); + + checkBitmapIndex(Lists.newArrayList(0, 3), adapter.getBitmapIndex("d7", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d7", "710")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d7", "720")); + + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d8", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d8", "810")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d8", "820")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821")); + + checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d9", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d9", "910")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d9", "911")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d9", "920")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921")); + } + + private void checkBitmapIndex(ArrayList expectIndex, IndexedInts index) + { + Assert.assertEquals(expectIndex.size(), index.size()); + int i = 0; + Iterator it = index.iterator(); + while (it.hasNext()) { + Assert.assertEquals(expectIndex.get(i), it.next()); + i++; + } } private IncrementalIndex getIndexD3() throws Exception diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java new file mode 100644 index 00000000000..db8cf9f8e17 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -0,0 +1,310 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.io.Files; +import io.druid.common.utils.JodaUtils; +import io.druid.data.input.InputRow; +import io.druid.data.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.ConciseBitmapSerdeFactory; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.commons.io.FileUtils; +import org.joda.time.DateTime; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class IndexMergerV9CompatibilityTest +{ + @Rule + public final CloserRule closer = new CloserRule(false); + private static final long TIMESTAMP = DateTime.parse("2014-01-01").getMillis(); + private static final AggregatorFactory[] DEFAULT_AGG_FACTORIES = new AggregatorFactory[]{ + new CountAggregatorFactory( + "count" + ) + }; + + private static final IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); + private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMerger(); + + private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( + new ConciseBitmapSerdeFactory(), + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZ4 + ); + private static final List DIMS = ImmutableList.of("dim0", "dim1"); + + private static final Function>, Object[]> OBJECT_MAKER = new Function>, Object[]>() + { + @Nullable + @Override + public Object[] apply(Collection> input) + { + final ArrayList list = new ArrayList<>(); + int i = 0; + for (final Map map : input) { + list.add(new MapBasedInputRow(TIMESTAMP + i++, DIMS, map)); + } + return new Object[]{list}; + } + }; + + @SafeVarargs + public static Collection permute(Map... maps) + { + if (maps == null) { + return ImmutableList.of(); + } + return Collections2.transform( + Collections2.permutations( + Arrays.asList(maps) + ), + OBJECT_MAKER + ); + } + + @Parameterized.Parameters + public static Iterable paramFeeder() + { + final Map map1 = ImmutableMap.of( + DIMS.get(0), ImmutableList.of("dim00", "dim01"), + DIMS.get(1), "dim10" + ); + + final List nullList = Collections.singletonList(null); + + final Map map2 = ImmutableMap.of( + DIMS.get(0), nullList, + DIMS.get(1), "dim10" + ); + + + final Map map3 = ImmutableMap.of( + DIMS.get(0), + ImmutableList.of("dim00", "dim01") + ); + + final Map map4 = ImmutableMap.of(); + + final Map map5 = ImmutableMap.of(DIMS.get(1), "dim10"); + + final Map map6 = new HashMap<>(); + map6.put(DIMS.get(1), null); // ImmutableMap cannot take null + + + return Iterables.concat( + permute(map1) + , permute(map1, map4) + , permute(map1, map5) + , permute(map5, map6) + , permute(map4, map5) + , Iterables.transform(ImmutableList.of(Arrays.asList(map1, map2, map3, map4, map5, map6)), OBJECT_MAKER) + ); + + } + + private final Collection events; + + public IndexMergerV9CompatibilityTest( + final Collection events + ) + { + this.events = events; + } + + IncrementalIndex toPersist; + File tmpDir; + File persistTmpDir; + + @Before + public void setUp() throws IOException + { + toPersist = new OnheapIncrementalIndex( + JodaUtils.MIN_INSTANT, + QueryGranularity.NONE, + DEFAULT_AGG_FACTORIES, + 1000000 + ); + for (InputRow event : events) { + toPersist.add(event); + } + tmpDir = Files.createTempDir(); + persistTmpDir = new File(tmpDir, "persistDir"); + INDEX_MERGER.persist(toPersist, persistTmpDir, null, INDEX_SPEC); + } + + @After + public void tearDown() throws IOException + { + FileUtils.deleteDirectory(tmpDir); + } + + @Test + public void testPersistWithSegmentMetadata() throws IOException + { + File outDir = Files.createTempDir(); + QueryableIndex index = null; + try { + outDir = Files.createTempDir(); + Map segmentMetadata = ImmutableMap.of("key", "value"); + index = INDEX_IO.loadIndex(INDEX_MERGER_V9.persist(toPersist, outDir, segmentMetadata, INDEX_SPEC)); + + Assert.assertEquals(segmentMetadata, index.getMetaData()); + } + finally { + if (index != null) { + index.close(); + ; + } + + if (outDir != null) { + FileUtils.deleteDirectory(outDir); + } + } + } + + @Test + public void testSimpleReprocess() throws IOException + { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( + closer.closeLater( + INDEX_IO.loadIndex( + persistTmpDir + ) + ) + ); + Assert.assertEquals(events.size(), adapter.getNumRows()); + reprocessAndValidate(persistTmpDir, new File(tmpDir, "reprocessed")); + } + + private File reprocessAndValidate(File inDir, File tmpDir) throws IOException + { + final File outDir = INDEX_MERGER_V9.convert( + inDir, + tmpDir, + INDEX_SPEC + ); + INDEX_IO.validateTwoSegments(persistTmpDir, outDir); + return outDir; + } + + private File appendAndValidate(File inDir, File tmpDir) throws IOException + { + final File outDir = INDEX_MERGER.append( + ImmutableList.of(new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(inDir)))), + tmpDir, + INDEX_SPEC + ); + INDEX_IO.validateTwoSegments(persistTmpDir, outDir); + return outDir; + } + + @Test + public void testIdempotentReprocess() throws IOException + { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( + closer.closeLater( + INDEX_IO.loadIndex( + persistTmpDir + ) + ) + ); + Assert.assertEquals(events.size(), adapter.getNumRows()); + final File tmpDir1 = new File(tmpDir, "reprocessed1"); + reprocessAndValidate(persistTmpDir, tmpDir1); + + final File tmpDir2 = new File(tmpDir, "reprocessed2"); + final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1))); + Assert.assertEquals(events.size(), adapter2.getNumRows()); + reprocessAndValidate(tmpDir1, tmpDir2); + + final File tmpDir3 = new File(tmpDir, "reprocessed3"); + final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2))); + Assert.assertEquals(events.size(), adapter3.getNumRows()); + reprocessAndValidate(tmpDir2, tmpDir3); + } + + @Test + public void testSimpleAppend() throws IOException + { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( + closer.closeLater( + INDEX_IO.loadIndex( + persistTmpDir + ) + ) + ); + Assert.assertEquals(events.size(), adapter.getNumRows()); + appendAndValidate(persistTmpDir, new File(tmpDir, "reprocessed")); + } + + @Test + public void testIdempotentAppend() throws IOException + { + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( + closer.closeLater( + INDEX_IO.loadIndex( + persistTmpDir + ) + ) + ); + Assert.assertEquals(events.size(), adapter.getNumRows()); + final File tmpDir1 = new File(tmpDir, "reprocessed1"); + appendAndValidate(persistTmpDir, tmpDir1); + + final File tmpDir2 = new File(tmpDir, "reprocessed2"); + final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1))); + Assert.assertEquals(events.size(), adapter2.getNumRows()); + appendAndValidate(tmpDir1, tmpDir2); + + final File tmpDir3 = new File(tmpDir, "reprocessed3"); + final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2))); + Assert.assertEquals(events.size(), adapter3.getNumRows()); + appendAndValidate(tmpDir2, tmpDir3); + } +} diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/IndexMergerV9Test.java new file mode 100644 index 00000000000..e75854e0ec8 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9Test.java @@ -0,0 +1,1025 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import io.druid.data.input.MapBasedInputRow; +import io.druid.granularity.QueryGranularity; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.segment.column.Column; +import io.druid.segment.column.SimpleDictionaryEncodedColumn; +import io.druid.segment.data.BitmapSerdeFactory; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.ConciseBitmapSerdeFactory; +import io.druid.segment.data.IncrementalIndexTest; +import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.RoaringBitmapSerdeFactory; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexAdapter; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.File; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +@RunWith(Parameterized.class) +public class IndexMergerV9Test +{ + @Rule + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + + protected final static IndexMergerV9 INDEX_MERGER = TestHelper.getTestIndexMergerV9(); + private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + + @Parameterized.Parameters(name = "{index}: bitmap={0}, metric compression={1}, dimension compression={2}") + public static Collection data() + { + return Collections2.transform( + Sets.cartesianProduct( + ImmutableList.of( + ImmutableSet.of(new RoaringBitmapSerdeFactory(), new ConciseBitmapSerdeFactory()), + ImmutableSet.of( + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZF + ), + ImmutableSet.of( + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZF + ) + ) + ), new Function, Object[]>() + { + @Nullable + @Override + public Object[] apply(List input) + { + return input.toArray(); + } + } + ); + } + + static IndexSpec makeIndexSpec( + BitmapSerdeFactory bitmapSerdeFactory, + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy + ) + { + if (bitmapSerdeFactory != null || compressionStrategy != null) { + return new IndexSpec( + bitmapSerdeFactory, + compressionStrategy.name().toLowerCase(), + dimCompressionStrategy.name().toLowerCase() + ); + } else { + return new IndexSpec(); + } + } + + private final IndexSpec indexSpec; + @Rule + public final CloserRule closer = new CloserRule(false); + + public IndexMergerV9Test( + BitmapSerdeFactory bitmapSerdeFactory, + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy + ) + { + this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy); + } + + @Test + public void testPersist() throws Exception + { + final long timestamp = System.currentTimeMillis(); + + IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist, + tempDir, + null, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(3, index.getColumnNames().size()); + + assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + } + + @Test + public void testPersistWithDifferentDims() throws Exception + { + IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null); + toPersist.add( + new MapBasedInputRow( + 1, + Arrays.asList("dim1", "dim2"), + ImmutableMap.of("dim1", "1", "dim2", "2") + ) + ); + toPersist.add( + new MapBasedInputRow( + 1, + Arrays.asList("dim1"), + ImmutableMap.of("dim1", "3") + ) + ); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist, + tempDir, + null, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(3, index.getColumnNames().size()); + assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); + Iterable boats = adapter.getRows(); + List boatList = new ArrayList<>(); + for (Rowboat boat : boats) { + boatList.add(boat); + } + + Assert.assertEquals(2, boatList.size()); + Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); + Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(1).getDims()); + + checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("dim1", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim1", "1")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim1", "3")); + + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dim2", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dim2", "2")); + } + + @Test + public void testPersistWithSegmentMetadata() throws Exception + { + final long timestamp = System.currentTimeMillis(); + + IncrementalIndex toPersist = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist); + + Map segmentMetadata = ImmutableMap.of("key", "value"); + + final File tempDir = temporaryFolder.newFolder(); + QueryableIndex index = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist, + tempDir, + segmentMetadata, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); + Assert.assertEquals(3, index.getColumnNames().size()); + + assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + + Assert.assertEquals(segmentMetadata, index.getMetaData()); + } + + @Test + public void testPersistMerge() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + IncrementalIndex toPersist2 = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + + toPersist2.add( + new MapBasedInputRow( + timestamp, + Arrays.asList("dim1", "dim2"), + ImmutableMap.of("dim1", "1", "dim2", "2") + ) + ); + + toPersist2.add( + new MapBasedInputRow( + timestamp, + Arrays.asList("dim1", "dim2"), + ImmutableMap.of("dim1", "5", "dim2", "6") + ) + ); + + final File tempDir1 = temporaryFolder.newFolder(); + final File tempDir2 = temporaryFolder.newFolder(); + final File mergedDir = temporaryFolder.newFolder(); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tempDir1, + null, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(3, index1.getColumnNames().size()); + + QueryableIndex index2 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist2, + tempDir2, + null, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index2.getAvailableDimensions())); + Assert.assertEquals(3, index2.getColumnNames().size()); + + QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + Arrays.asList(index1, index2), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + mergedDir, + indexSpec + ) + ) + ); + + Assert.assertEquals(3, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); + Assert.assertEquals(3, merged.getColumnNames().size()); + assertDimCompression(index2, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + } + + @Test + public void testPersistEmptyColumn() throws Exception + { + final IncrementalIndex toPersist1 = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{}, + 10 + ); + final IncrementalIndex toPersist2 = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{}, + 10 + ); + final File tmpDir1 = temporaryFolder.newFolder(); + final File tmpDir2 = temporaryFolder.newFolder(); + final File tmpDir3 = temporaryFolder.newFolder(); + + toPersist1.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "foo") + ) + ); + + toPersist2.add( + new MapBasedInputRow( + 1L, + ImmutableList.of("dim1", "dim2"), + ImmutableMap.of("dim1", ImmutableList.of(), "dim2", "bar") + ) + ); + + final QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tmpDir1, + null, + indexSpec + ) + ) + ); + final QueryableIndex index2 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tmpDir2, + null, + indexSpec + ) + ) + ); + final QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + Arrays.asList(index1, index2), + new AggregatorFactory[]{}, + tmpDir3, + indexSpec + ) + ) + ); + + Assert.assertEquals(1, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index1.getAvailableDimensions())); + + Assert.assertEquals(1, index2.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(index2.getAvailableDimensions())); + + Assert.assertEquals(1, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index2, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + } + + @Test + public void testMergeRetainsValues() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File mergedDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tempDir1, + null, + indexSpec + ) + ) + ); + + + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(3, index1.getColumnNames().size()); + + + QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + ImmutableList.of(index1), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + mergedDir, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); + Assert.assertEquals(3, merged.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + } + + + @Test + public void testAppendRetainsValues() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File mergedDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.append( + ImmutableList.of(incrementalAdapter), tempDir1, indexSpec + ) + ) + ); + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(3, index1.getColumnNames().size()); + + + QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + ImmutableList.of(index1), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + mergedDir, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); + Assert.assertEquals(3, merged.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + } + + @Test + public void testMergeSpecChange() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File mergedDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tempDir1, + null, + indexSpec + ) + ) + ); + + + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(3, index1.getColumnNames().size()); + + + IndexSpec newSpec = new IndexSpec( + indexSpec.getBitmapSerdeFactory(), + "lz4".equals(indexSpec.getDimensionCompression()) ? "lzf" : "lz4", + "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" + ); + + + QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + ImmutableList.of(index1), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + mergedDir, + newSpec + ) + ) + ); + + Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); + Assert.assertEquals(3, merged.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(merged, newSpec.getDimensionCompressionStrategy()); + } + + + @Test + public void testConvertSame() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex( + new AggregatorFactory[]{ + new LongSumAggregatorFactory( + "longSum1", + "dim1" + ), + new LongSumAggregatorFactory("longSum2", "dim2") + } + ); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File convertDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex(INDEX_MERGER.persist(toPersist1, tempDir1, null, indexSpec)) + ); + + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(4, index1.getColumnNames().size()); + + + QueryableIndex converted = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.convert( + tempDir1, + convertDir, + indexSpec + ) + ) + ); + + Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); + Assert.assertEquals(4, converted.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, convertDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy()); + } + + + @Test + public void testConvertDifferent() throws Exception + { + final long timestamp = System.currentTimeMillis(); + IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex( + new AggregatorFactory[]{ + new LongSumAggregatorFactory( + "longSum1", + "dim1" + ), + new LongSumAggregatorFactory("longSum2", "dim2") + } + ); + IncrementalIndexTest.populateIndex(timestamp, toPersist1); + + final File tempDir1 = temporaryFolder.newFolder(); + final File convertDir = temporaryFolder.newFolder(); + final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( + toPersist1.getInterval(), + toPersist1, + indexSpec.getBitmapSerdeFactory() + .getBitmapFactory() + ); + + QueryableIndex index1 = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersist1, + tempDir1, + null, + indexSpec + ) + ) + ); + + + final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); + + INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + + Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); + Assert.assertEquals(4, index1.getColumnNames().size()); + + + IndexSpec newSpec = new IndexSpec( + indexSpec.getBitmapSerdeFactory(), + "lz4".equals(indexSpec.getDimensionCompression()) ? "lzf" : "lz4", + "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" + ); + + QueryableIndex converted = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.convert( + tempDir1, + convertDir, + newSpec + ) + ) + ); + + Assert.assertEquals(2, converted.getColumn(Column.TIME_COLUMN_NAME).getLength()); + Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); + Assert.assertEquals(4, converted.getColumnNames().size()); + + INDEX_IO.validateTwoSegments(tempDir1, convertDir); + + assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(converted, newSpec.getDimensionCompressionStrategy()); + } + + private void assertDimCompression(QueryableIndex index, CompressedObjectStrategy.CompressionStrategy expectedStrategy) + throws Exception + { + // Java voodoo + if (expectedStrategy == null || expectedStrategy == CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { + return; + } + + Object encodedColumn = index.getColumn("dim2").getDictionaryEncoding(); + Field field = SimpleDictionaryEncodedColumn.class.getDeclaredField("column"); + field.setAccessible(true); + + Object obj = field.get(encodedColumn); + Field compressedSupplierField = obj.getClass().getDeclaredField("this$0"); + compressedSupplierField.setAccessible(true); + + Object supplier = compressedSupplierField.get(obj); + Field compressionField = supplier.getClass().getDeclaredField("compression"); + compressionField.setAccessible(true); + + Object strategy = compressionField.get(supplier); + + Assert.assertEquals(expectedStrategy, strategy); + } + + @Test + public void testDisjointDimMerge() throws Exception + { + IncrementalIndex toPersistA = getSingleDimIndex("dimA", Arrays.asList("1", "2")); + IncrementalIndex toPersistB = getSingleDimIndex("dimB", Arrays.asList("1", "2", "3")); + + final File tmpDirA = temporaryFolder.newFolder(); + final File tmpDirB = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); + + QueryableIndex indexA = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistA, + tmpDirA, + null, + indexSpec + ) + ) + ); + + QueryableIndex indexB = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistB, + tmpDirB, + null, + indexSpec + ) + ) + ); + + final QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + Arrays.asList(indexA, indexB), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec + ) + ) + ); + + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + Iterable boats = adapter.getRows(); + List boatList = new ArrayList<>(); + for (Rowboat boat : boats) { + boatList.add(boat); + } + + Assert.assertEquals(ImmutableList.of("dimA", "dimB"), ImmutableList.copyOf(adapter.getDimensionNames())); + Assert.assertEquals(5, boatList.size()); + Assert.assertArrayEquals(new int[][]{{0}, {1}}, boatList.get(0).getDims()); + Assert.assertArrayEquals(new Object[]{1L}, boatList.get(0).getMetrics()); + Assert.assertArrayEquals(new int[][]{{0}, {2}}, boatList.get(1).getDims()); + Assert.assertArrayEquals(new Object[]{1L}, boatList.get(1).getMetrics()); + Assert.assertArrayEquals(new int[][]{{0}, {3}}, boatList.get(2).getDims()); + Assert.assertArrayEquals(new Object[]{1L}, boatList.get(2).getMetrics()); + Assert.assertArrayEquals(new int[][]{{1}, {0}}, boatList.get(3).getDims()); + Assert.assertArrayEquals(new Object[]{1L}, boatList.get(3).getMetrics()); + Assert.assertArrayEquals(new int[][]{{2}, {0}}, boatList.get(4).getDims()); + Assert.assertArrayEquals(new Object[]{1L}, boatList.get(4).getMetrics()); + + checkBitmapIndex(Lists.newArrayList(0, 1, 2), adapter.getBitmapIndex("dimA", "")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("dimA", "1")); + checkBitmapIndex(Lists.newArrayList(4), adapter.getBitmapIndex("dimA", "2")); + + checkBitmapIndex(Lists.newArrayList(3, 4), adapter.getBitmapIndex("dimB", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("dimB", "1")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("dimB", "2")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("dimB", "3")); + } + + @Test + public void testJointDimMerge() throws Exception + { + // (d1, d2, d3) from only one index, and their dim values are ('empty', 'has null', 'no null') + // (d4, d5, d6, d7, d8, d9) are from both indexes + // d4: 'empty' join 'empty' + // d5: 'empty' join 'has null' + // d6: 'empty' join 'no null' + // d7: 'has null' join 'has null' + // d8: 'has null' join 'no null' + // d9: 'no null' join 'no null' + + IncrementalIndex toPersistA = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + toPersistA.add( + new MapBasedInputRow( + 1, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d1", "", "d2", "", "d3", "310", "d7", "", "d9", "910" + ) + ) + ); + toPersistA.add( + new MapBasedInputRow( + 2, + Arrays.asList("d1", "d2", "d3", "d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d2", "210", "d3", "311", "d7", "710", "d8", "810", "d9", "911" + ) + ) + ); + + IncrementalIndex toPersistB = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + toPersistB.add( + new MapBasedInputRow( + 3, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d5", "520", "d6", "620", "d7", "720", "d8", "820", "d9", "920" + ) + ) + ); + toPersistB.add( + new MapBasedInputRow( + 4, + Arrays.asList("d4", "d5", "d6", "d7", "d8", "d9"), + ImmutableMap.of( + "d5", "", "d6", "621", "d7", "", "d8", "821", "d9", "921" + ) + ) + ); + final File tmpDirA = temporaryFolder.newFolder(); + final File tmpDirB = temporaryFolder.newFolder(); + final File tmpDirMerged = temporaryFolder.newFolder(); + + QueryableIndex indexA = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistA, + tmpDirA, + null, + indexSpec + ) + ) + ); + + QueryableIndex indexB = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.persist( + toPersistB, + tmpDirB, + null, + indexSpec + ) + ) + ); + + final QueryableIndex merged = closer.closeLater( + INDEX_IO.loadIndex( + INDEX_MERGER.mergeQueryableIndex( + Arrays.asList(indexA, indexB), + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + tmpDirMerged, + indexSpec + ) + ) + ); + + final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(merged); + Iterable boats = adapter.getRows(); + List boatList = new ArrayList<>(); + for (Rowboat boat : boats) { + boatList.add(boat); + } + + Assert.assertEquals( + ImmutableList.of("d2", "d3", "d5", "d6", "d7", "d8", "d9"), + ImmutableList.copyOf(adapter.getDimensionNames()) + ); + Assert.assertEquals(4, boatList.size()); + Assert.assertArrayEquals(new int[][]{{0}, {1}, {0}, {0}, {0}, {0}, {0}}, boatList.get(0).getDims()); + Assert.assertArrayEquals(new int[][]{{1}, {2}, {0}, {0}, {1}, {1}, {1}}, boatList.get(1).getDims()); + Assert.assertArrayEquals(new int[][]{{0}, {0}, {1}, {1}, {2}, {2}, {2}}, boatList.get(2).getDims()); + Assert.assertArrayEquals(new int[][]{{0}, {0}, {0}, {2}, {0}, {3}, {3}}, boatList.get(3).getDims()); + + checkBitmapIndex(Lists.newArrayList(0, 2, 3), adapter.getBitmapIndex("d2", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d2", "210")); + + checkBitmapIndex(Lists.newArrayList(2, 3), adapter.getBitmapIndex("d3", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d3", "310")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d3", "311")); + + checkBitmapIndex(Lists.newArrayList(0, 1, 3), adapter.getBitmapIndex("d5", "")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d5", "520")); + + checkBitmapIndex(Lists.newArrayList(0, 1), adapter.getBitmapIndex("d6", "")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d6", "620")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d6", "621")); + + checkBitmapIndex(Lists.newArrayList(0, 3), adapter.getBitmapIndex("d7", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d7", "710")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d7", "720")); + + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d8", "")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d8", "810")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d8", "820")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d8", "821")); + + checkBitmapIndex(new ArrayList(), adapter.getBitmapIndex("d9", "")); + checkBitmapIndex(Lists.newArrayList(0), adapter.getBitmapIndex("d9", "910")); + checkBitmapIndex(Lists.newArrayList(1), adapter.getBitmapIndex("d9", "911")); + checkBitmapIndex(Lists.newArrayList(2), adapter.getBitmapIndex("d9", "920")); + checkBitmapIndex(Lists.newArrayList(3), adapter.getBitmapIndex("d9", "921")); + } + + private void checkBitmapIndex(ArrayList expectIndex, IndexedInts index) + { + Assert.assertEquals(expectIndex.size(), index.size()); + int i = 0; + Iterator it = index.iterator(); + while (it.hasNext()) { + Assert.assertEquals(expectIndex.get(i), it.next()); + i++; + } + } + + private IncrementalIndex getIndexD3() throws Exception + { + IncrementalIndex toPersist1 = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + + toPersist1.add( + new MapBasedInputRow( + 0, + Arrays.asList("d3", "d1", "d2"), + ImmutableMap.of("d1", "100", "d2", "4000", "d3", "30000") + ) + ); + + toPersist1.add( + new MapBasedInputRow( + 0, + Arrays.asList("d3", "d1", "d2"), + ImmutableMap.of("d1", "200", "d2", "3000", "d3", "50000") + ) + ); + + toPersist1.add( + new MapBasedInputRow( + 0, + Arrays.asList("d3", "d1", "d2"), + ImmutableMap.of("d1", "300", "d2", "2000", "d3", "40000") + ) + ); + + return toPersist1; + } + + private IncrementalIndex getSingleDimIndex(String dimName, List values) throws Exception + { + IncrementalIndex toPersist1 = new OnheapIncrementalIndex( + 0L, + QueryGranularity.NONE, + new AggregatorFactory[]{new CountAggregatorFactory("count")}, + 1000 + ); + + for (String val : values) { + toPersist1.add( + new MapBasedInputRow( + 1, + Arrays.asList(dimName), + ImmutableMap.of(dimName, val) + ) + ); + } + + return toPersist1; + } +} diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java new file mode 100644 index 00000000000..568f7d48023 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -0,0 +1,707 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.metamx.collections.spatial.search.RadiusBound; +import com.metamx.collections.spatial.search.RectangularBound; +import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.granularity.QueryGranularity; +import io.druid.query.Druids; +import io.druid.query.FinalizeResultsQueryRunner; +import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; +import io.druid.query.Result; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.SpatialDimFilter; +import io.druid.query.timeseries.TimeseriesQuery; +import io.druid.query.timeseries.TimeseriesQueryEngine; +import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.incremental.OnheapIncrementalIndex; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Random; + +/** + */ +@RunWith(Parameterized.class) +public class IndexMergerV9WithSpatialIndexTest +{ + private static IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); + private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + + public static final int NUM_POINTS = 5000; + private static Interval DATA_INTERVAL = new Interval("2013-01-01/2013-01-07"); + + private static AggregatorFactory[] METRIC_AGGS = new AggregatorFactory[]{ + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") + }; + + private static List DIMS = Lists.newArrayList("dim", "lat", "long", "lat2", "long2"); + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + final IndexSpec indexSpec = new IndexSpec(); + final IncrementalIndex rtIndex = makeIncrementalIndex(); + final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec); + final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec); + return Arrays.asList( + new Object[][]{ + { + new IncrementalIndexSegment(rtIndex, null) + }, + { + new QueryableIndexSegment(null, mMappedTestIndex) + }, + { + new QueryableIndexSegment(null, mergedRealtimeIndex) + } + } + ); + } + + private static IncrementalIndex makeIncrementalIndex() throws IOException + { + IncrementalIndex theIndex = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) + .withQueryGranularity(QueryGranularity.DAY) + .withMetrics(METRIC_AGGS) + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + + ) + ) + ).build(), + false, + NUM_POINTS + ); + + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-01").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-01").toString(), + "dim", "foo", + "lat", 0.0f, + "long", 0.0f, + "val", 17L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-02").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-02").toString(), + "dim", "foo", + "lat", 1.0f, + "long", 3.0f, + "val", 29L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-03").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-03").toString(), + "dim", "foo", + "lat", 4.0f, + "long", 2.0f, + "val", 13L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-04").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-04").toString(), + "dim", "foo", + "lat", 7.0f, + "long", 3.0f, + "val", 91L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "lat", 8.0f, + "long", 6.0f, + "val", 47L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "lat", "_mmx.unknown", + "long", "_mmx.unknown", + "val", 101L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "dim.geo", "_mmx.unknown", + "val", 501L + ) + ) + ); + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "lat2", 0.0f, + "long2", 0.0f, + "val", 13L + ) + ) + ); + + // Add a bunch of random points + Random rand = new Random(); + for (int i = 8; i < NUM_POINTS; i++) { + theIndex.add( + new MapBasedInputRow( + new DateTime("2013-01-01").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-01").toString(), + "dim", "boo", + "lat", (float) (rand.nextFloat() * 10 + 10.0), + "long", (float) (rand.nextFloat() * 10 + 10.0), + "val", i + ) + ) + ); + } + + return theIndex; + } + + private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException + { + IncrementalIndex theIndex = makeIncrementalIndex(); + File tmpFile = File.createTempFile("billy", "yay"); + tmpFile.delete(); + tmpFile.mkdirs(); + tmpFile.deleteOnExit(); + + INDEX_MERGER_V9.persist(theIndex, tmpFile, null, indexSpec); + return INDEX_IO.loadIndex(tmpFile); + } + + private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) + { + try { + IncrementalIndex first = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) + .withQueryGranularity(QueryGranularity.DAY) + .withMetrics(METRIC_AGGS) + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + + ) + ) + ).build(), + false, + 1000 + ); + IncrementalIndex second = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) + .withQueryGranularity(QueryGranularity.DAY) + .withMetrics(METRIC_AGGS) + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + + ) + ) + ).build(), + false, + 1000 + ); + IncrementalIndex third = new OnheapIncrementalIndex( + new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) + .withQueryGranularity(QueryGranularity.DAY) + .withMetrics(METRIC_AGGS) + .withDimensionsSpec( + new DimensionsSpec( + null, + null, + Arrays.asList( + new SpatialDimensionSchema( + "dim.geo", + Arrays.asList("lat", "long") + ), + new SpatialDimensionSchema( + "spatialIsRad", + Arrays.asList("lat2", "long2") + ) + + ) + ) + ).build(), + false, + NUM_POINTS + ); + + + first.add( + new MapBasedInputRow( + new DateTime("2013-01-01").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-01").toString(), + "dim", "foo", + "lat", 0.0f, + "long", 0.0f, + "val", 17L + ) + ) + ); + first.add( + new MapBasedInputRow( + new DateTime("2013-01-02").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-02").toString(), + "dim", "foo", + "lat", 1.0f, + "long", 3.0f, + "val", 29L + ) + ) + ); + first.add( + new MapBasedInputRow( + new DateTime("2013-01-03").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-03").toString(), + "dim", "foo", + "lat", 4.0f, + "long", 2.0f, + "val", 13L + ) + ) + ); + first.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "lat", "_mmx.unknown", + "long", "_mmx.unknown", + "val", 101L + ) + ) + ); + first.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "dim.geo", "_mmx.unknown", + "val", 501L + ) + ) + ); + second.add( + new MapBasedInputRow( + new DateTime("2013-01-04").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-04").toString(), + "dim", "foo", + "lat", 7.0f, + "long", 3.0f, + "val", 91L + ) + ) + ); + second.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "dim", "foo", + "lat", 8.0f, + "long", 6.0f, + "val", 47L + ) + ) + ); + second.add( + new MapBasedInputRow( + new DateTime("2013-01-05").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-05").toString(), + "lat2", 0.0f, + "long2", 0.0f, + "val", 13L + ) + ) + ); + + // Add a bunch of random points + Random rand = new Random(); + for (int i = 8; i < NUM_POINTS; i++) { + third.add( + new MapBasedInputRow( + new DateTime("2013-01-01").getMillis(), + DIMS, + ImmutableMap.of( + "timestamp", new DateTime("2013-01-01").toString(), + "dim", "boo", + "lat", (float) (rand.nextFloat() * 10 + 10.0), + "long", (float) (rand.nextFloat() * 10 + 10.0), + "val", i + ) + ) + ); + } + + + File tmpFile = File.createTempFile("yay", "who"); + tmpFile.delete(); + + File firstFile = new File(tmpFile, "first"); + File secondFile = new File(tmpFile, "second"); + File thirdFile = new File(tmpFile, "third"); + File mergedFile = new File(tmpFile, "merged"); + + firstFile.mkdirs(); + firstFile.deleteOnExit(); + secondFile.mkdirs(); + secondFile.deleteOnExit(); + thirdFile.mkdirs(); + thirdFile.deleteOnExit(); + mergedFile.mkdirs(); + mergedFile.deleteOnExit(); + + INDEX_MERGER_V9.persist(first, DATA_INTERVAL, firstFile, null, indexSpec); + INDEX_MERGER_V9.persist(second, DATA_INTERVAL, secondFile, null, indexSpec); + INDEX_MERGER_V9.persist(third, DATA_INTERVAL, thirdFile, null, indexSpec); + + QueryableIndex mergedRealtime = INDEX_IO.loadIndex( + INDEX_MERGER_V9.mergeQueryableIndex( + Arrays.asList( + INDEX_IO.loadIndex(firstFile), + INDEX_IO.loadIndex(secondFile), + INDEX_IO.loadIndex(thirdFile) + ), + METRIC_AGGS, + mergedFile, + indexSpec + ) + ); + + return mergedRealtime; + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private final Segment segment; + + public IndexMergerV9WithSpatialIndexTest(Segment segment) + { + this.segment = segment; + } + + @Test + public void testSpatialQuery() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .granularity(QueryGranularity.ALL) + .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) + .filters( + new SpatialDimFilter( + "dim.geo", + new RadiusBound(new float[]{0.0f, 0.0f}, 5) + ) + ) + .aggregators( + Arrays.asList( + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") + ) + ) + .build(); + + List> expectedResults = Arrays.asList( + new Result<>( + new DateTime("2013-01-01T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 3L) + .put("val", 59L) + .build() + ) + ) + ); + try { + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + + QueryRunner runner = new FinalizeResultsQueryRunner( + factory.createRunner(segment), + factory.getToolchest() + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + + @Test + public void testSpatialQueryWithOtherSpatialDim() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .granularity(QueryGranularity.ALL) + .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) + .filters( + new SpatialDimFilter( + "spatialIsRad", + new RadiusBound(new float[]{0.0f, 0.0f}, 5) + ) + ) + .aggregators( + Arrays.asList( + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") + ) + ) + .build(); + + List> expectedResults = Arrays.asList( + new Result<>( + new DateTime("2013-01-01T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 13L) + .build() + ) + ) + ); + try { + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + + QueryRunner runner = new FinalizeResultsQueryRunner( + factory.createRunner(segment), + factory.getToolchest() + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } + + @Test + public void testSpatialQueryMorePoints() + { + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .granularity(QueryGranularity.DAY) + .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) + .filters( + new SpatialDimFilter( + "dim.geo", + new RectangularBound(new float[]{0.0f, 0.0f}, new float[]{9.0f, 9.0f}) + ) + ) + .aggregators( + Arrays.asList( + new CountAggregatorFactory("rows"), + new LongSumAggregatorFactory("val", "val") + ) + ) + .build(); + + List> expectedResults = Arrays.asList( + new Result<>( + new DateTime("2013-01-01T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 17L) + .build() + ) + ), + new Result<>( + new DateTime("2013-01-02T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 29L) + .build() + ) + ), + new Result<>( + new DateTime("2013-01-03T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 13L) + .build() + ) + ), + new Result<>( + new DateTime("2013-01-04T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 91L) + .build() + ) + ), + new Result<>( + new DateTime("2013-01-05T00:00:00.000Z"), + new TimeseriesResultValue( + ImmutableMap.builder() + .put("rows", 1L) + .put("val", 47L) + .build() + ) + ) + ); + try { + TimeseriesQueryRunnerFactory factory = new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ); + + QueryRunner runner = new FinalizeResultsQueryRunner( + factory.createRunner(segment), + factory.getToolchest() + ); + + TestHelper.assertExpectedResults(expectedResults, runner.run(query, Maps.newHashMap())); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index 6875caf9076..a4829257922 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -36,6 +36,7 @@ import java.util.Iterator; public class TestHelper { private static final IndexMerger INDEX_MERGER; + private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); @@ -52,6 +53,7 @@ public class TestHelper } ); INDEX_MERGER = new IndexMerger(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); } @@ -60,6 +62,11 @@ public class TestHelper return INDEX_MERGER; } + public static IndexMergerV9 getTestIndexMergerV9() + { + return INDEX_MERGER_V9; + } + public static IndexIO getTestIndexIO() { return INDEX_IO; diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java index 2f7688a6964..c9044dade89 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java @@ -79,6 +79,8 @@ public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyT } ); + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + IndexedFloats floats = CompressedFloatsIndexedSupplier .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) .get(); @@ -90,4 +92,45 @@ public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyT floats.close(); } + + @Test + public void testEmpty() throws Exception + { + final ByteOrder order = ByteOrder.nativeOrder(); + final int sizePer = 999; + CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer( + sizePer, + new GenericIndexedWriter>( + new IOPeonForTesting(), + "test", + CompressedFloatBufferObjectStrategy.getBufferForOrder( + order, + compressionStrategy, + sizePer + ) + ), + compressionStrategy + ); + serializer.open(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.closeAndConsolidate( + new OutputSupplier() + { + @Override + public OutputStream getOutput() throws IOException + { + return baos; + } + } + ); + + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + IndexedFloats floats = CompressedFloatsIndexedSupplier + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) + .get(); + + Assert.assertEquals(0, floats.size()); + floats.close(); + } + } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java index 6cbfdb0e6e6..64ea94c5556 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java @@ -115,8 +115,10 @@ public class CompressedIntsIndexedWriterTest for (int val : vals) { writer.add(val); } + writer.close(); + long writtenLength = writer.getSerializedSize(); final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - long writtenLength = writer.closeAndWriteToChannel(outputChannel); + writer.writeToChannel(outputChannel); outputChannel.close(); assertEquals(writtenLength, supplierFromList.getSerializedSize()); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java index 39a5dcee207..eef04640948 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java @@ -75,6 +75,8 @@ public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTe } ); + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + IndexedLongs longs = CompressedLongsIndexedSupplier .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) .get(); @@ -85,4 +87,40 @@ public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTe } longs.close(); } + + @Test + public void testEmpty() throws Exception + { + final ByteOrder order = ByteOrder.nativeOrder(); + final int sizePer = 999; + CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer( + sizePer, + new GenericIndexedWriter>( + new IOPeonForTesting(), + "test", + CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer) + ), + compressionStrategy + ); + serializer.open(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.closeAndConsolidate( + new OutputSupplier() + { + @Override + public OutputStream getOutput() throws IOException + { + return baos; + } + } + ); + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + + IndexedLongs longs = CompressedLongsIndexedSupplier + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) + .get(); + + Assert.assertEquals(0, longs.size()); + longs.close(); + } } diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java index 414468d510e..23374ccecaf 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -132,9 +132,10 @@ public class CompressedVSizeIndexedV3WriterTest for (int[] val : vals) { writer.add(val); } - + writer.close(); + long writtenLength = writer.getSerializedSize(); final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - long writtenLength = writer.closeAndWriteToChannel(outputChannel); + writer.writeToChannel(outputChannel); outputChannel.close(); assertEquals(writtenLength, supplierFromIterable.getSerializedSize()); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java index 5863428ef4b..fd2c9686c6f 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java @@ -114,8 +114,10 @@ public class CompressedVSizeIntsIndexedWriterTest for (int val : vals) { writer.add(val); } + writer.close(); + long writtenLength = writer.getSerializedSize(); final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - long writtenLength = writer.closeAndWriteToChannel(outputChannel); + writer.writeToChannel(outputChannel); outputChannel.close(); assertEquals(writtenLength, supplierFromList.getSerializedSize()); diff --git a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java new file mode 100644 index 00000000000..a322e6744ea --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java @@ -0,0 +1,111 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.primitives.Ints; +import org.apache.commons.io.IOUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.Random; + +import static org.junit.Assert.assertEquals; + +public class VSizeIndexedIntsWriterTest +{ + private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; + + private final IOPeon ioPeon = new TmpFileIOPeon(); + private final Random rand = new Random(0); + private int[] vals; + + @Before + public void setUp() throws Exception + { + vals = null; + } + + @After + public void tearDown() throws Exception + { + ioPeon.cleanup(); + } + + private void generateVals(final int totalSize, final int maxValue) throws IOException + { + vals = new int[totalSize]; + for (int i = 0; i < vals.length; ++i) { + vals[i] = rand.nextInt(maxValue); + } + } + + private void checkSerializedSizeAndData() throws Exception + { + int maxValue = vals.length == 0 ? 0 : Ints.max(vals); + VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter( + ioPeon, "test", maxValue + ); + + VSizeIndexedInts intsFromList = VSizeIndexedInts.fromList( + Ints.asList(vals), maxValue + ); + writer.open(); + for (int val : vals) { + writer.add(val); + } + writer.close(); + long writtenLength = writer.getSerializedSize(); + final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); + writer.writeToChannel(outputChannel); + outputChannel.close(); + + assertEquals(writtenLength, intsFromList.getSerializedSize()); + + // read from ByteBuffer and check values + VSizeIndexedInts intsFromByteBuffer = VSizeIndexedInts.readFromByteBuffer( + ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))) + ); + assertEquals(vals.length, intsFromByteBuffer.size()); + for (int i = 0; i < vals.length; ++i) { + assertEquals(vals[i], intsFromByteBuffer.get(i)); + } + } + + @Test + public void testAdd() throws Exception + { + for (int maxValue : MAX_VALUES) { + generateVals(rand.nextInt(100) + 10, maxValue); + checkSerializedSizeAndData(); + } + } + + @Test + public void testWriteEmpty() throws Exception + { + vals = new int[0]; + checkSerializedSizeAndData(); + } +} \ No newline at end of file From 82ff98c2bfa46163660a1a418f34db45bbca9f6b Mon Sep 17 00:00:00 2001 From: Kurt Young Date: Wed, 13 Jan 2016 14:06:35 +0800 Subject: [PATCH 3/3] add config for build v9 directly and update docs --- docs/content/ingestion/batch-ingestion.md | 4 +- docs/content/ingestion/realtime-ingestion.md | 1 + .../indexer/HadoopDruidIndexerConfig.java | 8 + .../io/druid/indexer/HadoopTuningConfig.java | 23 +- .../io/druid/indexer/IndexGeneratorJob.java | 26 +- .../indexer/BatchDeltaIngestionTest.java | 1 + .../DetermineHashedPartitionsJobTest.java | 1 + .../indexer/DeterminePartitionsJobTest.java | 1 + .../indexer/HadoopDruidIndexerConfigTest.java | 1 + .../druid/indexer/HadoopTuningConfigTest.java | 1 + .../druid/indexer/IndexGeneratorJobTest.java | 1 + .../java/io/druid/indexer/JobHelperTest.java | 1 + .../indexer/path/GranularityPathSpecTest.java | 2 +- .../updater/HadoopConverterJobTest.java | 1 + .../io/druid/indexing/common/TaskToolbox.java | 11 +- .../indexing/common/TaskToolboxFactory.java | 9 +- .../druid/indexing/common/task/IndexTask.java | 23 +- .../common/task/RealtimeIndexTask.java | 6 +- .../indexing/common/TaskToolboxTest.java | 5 +- .../io/druid/indexing/common/TestUtils.java | 7 + .../indexing/common/task/IndexTaskTest.java | 11 +- .../common/task/RealtimeIndexTaskTest.java | 4 +- .../indexing/common/task/TaskSerdeTest.java | 7 +- .../IngestSegmentFirehoseFactoryTest.java | 6 +- ...estSegmentFirehoseFactoryTimelineTest.java | 6 +- .../indexing/overlord/TaskLifecycleTest.java | 13 +- .../worker/WorkerTaskMonitorTest.java | 6 +- .../java/io/druid/segment/IndexMerger.java | 12 +- .../java/io/druid/segment/IndexMergerV9.java | 336 +----------------- .../indexing/RealtimeTuningConfig.java | 20 +- .../segment/realtime/FireDepartmentTest.java | 2 +- .../segment/realtime/RealtimeManagerTest.java | 1 + .../plumber/RealtimePlumberSchoolTest.java | 1 + .../segment/realtime/plumber/SinkTest.java | 1 + 34 files changed, 185 insertions(+), 374 deletions(-) diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index a480e1f075c..d2661269969 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -104,7 +104,8 @@ The spec\_file is a path to a file that contains JSON and an example looks like: "ignoreInvalidRows" : false, "jobProperties" : { }, "combineText" : false, - "rowFlushBoundary" : 300000 + "rowFlushBoundary" : 300000, + "buildV9Directly" : false } } ``` @@ -205,6 +206,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |ignoreInvalidRows|Boolean|Ignore rows found to have problems.|no (default == false)| |useCombiner|Boolean|Use hadoop combiner to merge rows at mapper if possible.|no (default == false)| |jobProperties|Object|a map of properties to add to the Hadoop job configuration.|no (default == null)| +|buildV9Directly|Boolean|Whether to build v9 index directly instead of building v8 index and convert it to v9 format|no (default = false)| ### Partitioning specification diff --git a/docs/content/ingestion/realtime-ingestion.md b/docs/content/ingestion/realtime-ingestion.md index 1a13f885ce8..5c5fe5b93d1 100644 --- a/docs/content/ingestion/realtime-ingestion.md +++ b/docs/content/ingestion/realtime-ingestion.md @@ -142,6 +142,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |rejectionPolicy|Object|Controls how data sets the data acceptance policy for creating and handing off segments. More on this below.|no (default=='serverTime')| |maxPendingPersists|Integer|Maximum number of persists that can be pending, but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|no (default == 0; meaning one persist can be running concurrently with ingestion, and none can be queued up)| |shardSpec|Object|This describes the shard that is represented by this server. This must be specified properly in order to have multiple realtime nodes indexing the same data stream in a [sharded fashion](#sharding).|no (default == 'NoneShardSpec'| +|buildV9Directly|Boolean|Whether to build v9 index directly instead of building v8 index and convert it to v9 format|no (default = false)| #### Rejection Policy diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java index fd3a1f9c97c..11a385e1a4e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerConfig.java @@ -51,6 +51,7 @@ import io.druid.indexer.path.PathSpec; import io.druid.initialization.Initialization; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.server.DruidNode; @@ -89,6 +90,7 @@ public class HadoopDruidIndexerConfig public static final ObjectMapper JSON_MAPPER; public static final IndexIO INDEX_IO; public static final IndexMerger INDEX_MERGER; + public static final IndexMergerV9 INDEX_MERGER_V9; private static final String DEFAULT_WORKING_PATH = "/tmp/druid-indexing"; @@ -112,6 +114,7 @@ public class HadoopDruidIndexerConfig JSON_MAPPER = injector.getInstance(ObjectMapper.class); INDEX_IO = injector.getInstance(IndexIO.class); INDEX_MERGER = injector.getInstance(IndexMerger.class); + INDEX_MERGER_V9 = injector.getInstance(IndexMergerV9.class); } public static enum IndexJobCounters @@ -351,6 +354,11 @@ public class HadoopDruidIndexerConfig return schema.getTuningConfig().getShardSpecs().get(bucket.time).get(bucket.partitionNum); } + public boolean isBuildV9Directly() + { + return schema.getTuningConfig().getBuildV9Directly(); + } + /** * Job instance should have Configuration set (by calling {@link #addJobProperties(Job)} * or via injected system properties) before this method is called. The {@link PathSpec} may diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java index 2f93f195681..522511d0a77 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopTuningConfig.java @@ -42,6 +42,7 @@ public class HadoopTuningConfig implements TuningConfig private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 80000; private static final boolean DEFAULT_USE_COMBINER = false; + private static final Boolean DEFAULT_BUILD_V9_DIRECTLY = Boolean.FALSE; public static HadoopTuningConfig makeDefaultTuningConfig() { @@ -59,7 +60,8 @@ public class HadoopTuningConfig implements TuningConfig null, false, false, - null + null, + DEFAULT_BUILD_V9_DIRECTLY ); } @@ -76,6 +78,7 @@ public class HadoopTuningConfig implements TuningConfig private final Map jobProperties; private final boolean combineText; private final boolean useCombiner; + private final Boolean buildV9Directly; @JsonCreator public HadoopTuningConfig( @@ -93,7 +96,8 @@ public class HadoopTuningConfig implements TuningConfig final @JsonProperty("combineText") boolean combineText, final @JsonProperty("useCombiner") Boolean useCombiner, // See https://github.com/druid-io/druid/pull/1922 - final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT + final @JsonProperty("rowFlushBoundary") Integer maxRowsInMemoryCOMPAT, + final @JsonProperty("buildV9Directly") Boolean buildV9Directly ) { this.workingPath = workingPath; @@ -111,6 +115,7 @@ public class HadoopTuningConfig implements TuningConfig : ImmutableMap.copyOf(jobProperties)); this.combineText = combineText; this.useCombiner = useCombiner == null ? DEFAULT_USE_COMBINER : useCombiner.booleanValue(); + this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly; } @JsonProperty @@ -191,6 +196,11 @@ public class HadoopTuningConfig implements TuningConfig return useCombiner; } + @JsonProperty + public Boolean getBuildV9Directly() { + return buildV9Directly; + } + public HadoopTuningConfig withWorkingPath(String path) { return new HadoopTuningConfig( @@ -207,7 +217,8 @@ public class HadoopTuningConfig implements TuningConfig jobProperties, combineText, useCombiner, - null + null, + buildV9Directly ); } @@ -227,7 +238,8 @@ public class HadoopTuningConfig implements TuningConfig jobProperties, combineText, useCombiner, - null + null, + buildV9Directly ); } @@ -247,7 +259,8 @@ public class HadoopTuningConfig implements TuningConfig jobProperties, combineText, useCombiner, - null + null, + buildV9Directly ); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 56059c866e7..177fde0635d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -467,9 +467,15 @@ public class IndexGeneratorJob implements Jobby final ProgressIndicator progressIndicator ) throws IOException { - return HadoopDruidIndexerConfig.INDEX_MERGER.persist( - index, interval, file, null, config.getIndexSpec(), progressIndicator - ); + if (config.isBuildV9Directly()) { + return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist( + index, interval, file, null, config.getIndexSpec(), progressIndicator + ); + } else { + return HadoopDruidIndexerConfig.INDEX_MERGER.persist( + index, interval, file, null, config.getIndexSpec(), progressIndicator + ); + } } protected File mergeQueryableIndex( @@ -479,9 +485,15 @@ public class IndexGeneratorJob implements Jobby ProgressIndicator progressIndicator ) throws IOException { - return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex( - indexes, aggs, file, config.getIndexSpec(), progressIndicator - ); + if (config.isBuildV9Directly()) { + return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex( + indexes, aggs, file, config.getIndexSpec(), progressIndicator + ); + } else { + return HadoopDruidIndexerConfig.INDEX_MERGER.mergeQueryableIndex( + indexes, aggs, file, config.getIndexSpec(), progressIndicator + ); + } } @Override @@ -587,7 +599,7 @@ public class IndexGeneratorJob implements Jobby indexes.add(HadoopDruidIndexerConfig.INDEX_IO.loadIndex(file)); } mergedBase = mergeQueryableIndex( - indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator + indexes, aggregators, new File(baseFlushFile, "merged"), progressIndicator ); } final FileSystem outputFS = new Path(config.getSchema().getIOConfig().getSegmentOutputPath()) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 602e9968491..6860d703446 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -381,6 +381,7 @@ public class BatchDeltaIngestionTest null, false, false, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java index 02b20a448b1..7aec077c1a9 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -160,6 +160,7 @@ public class DetermineHashedPartitionsJobTest null, false, false, + null, null ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java index 203dc46ecb4..aad381e7720 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java @@ -264,6 +264,7 @@ public class DeterminePartitionsJobTest null, false, false, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index ab9bac38862..9ce7ff16aba 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -207,6 +207,7 @@ public class HadoopDruidIndexerConfigTest null, false, false, + null, null ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java index 10b8211f004..bd632c3c252 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopTuningConfigTest.java @@ -53,6 +53,7 @@ public class HadoopTuningConfigTest null, true, true, + null, null ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index 339a1ccce57..1458714668d 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -395,6 +395,7 @@ public class IndexGeneratorJobTest ImmutableMap.of(JobContext.NUM_REDUCES, "0"), //verifies that set num reducers is ignored false, useCombiner, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index 50bfcc00b83..4116ea29464 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -115,6 +115,7 @@ public class JobHelperTest ), false, false, + null, null ) ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index 302446cad81..77e31e79fe1 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -120,7 +120,7 @@ public class GranularityPathSpecTest jsonMapper ), new HadoopIOConfig(null, null, null), - new HadoopTuningConfig(null, null, null, null, null, null, false, false, false, false, null, false, false, null) + new HadoopTuningConfig(null, null, null, null, null, null, false, false, false, false, null, false, false, null, null) ); granularityPathSpec.setDataGranularity(Granularity.HOUR); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 770fe061422..126a37b1bfa 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -201,6 +201,7 @@ public class HadoopConverterJobTest null, false, false, + null, null ) ) diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 3dfc545dde6..36767cc24d2 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -37,6 +37,7 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -80,7 +81,7 @@ public class TaskToolbox private final IndexIO indexIO; private final Cache cache; private final CacheConfig cacheConfig; - + private final IndexMergerV9 indexMergerV9; public TaskToolbox( TaskConfig config, @@ -102,7 +103,8 @@ public class TaskToolbox IndexMerger indexMerger, IndexIO indexIO, Cache cache, - CacheConfig cacheConfig + CacheConfig cacheConfig, + IndexMergerV9 indexMergerV9 ) { this.config = config; @@ -125,6 +127,7 @@ public class TaskToolbox this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; this.cacheConfig = cacheConfig; + this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); } public TaskConfig getConfig() @@ -247,4 +250,8 @@ public class TaskToolbox { return cacheConfig; } + + public IndexMergerV9 getIndexMergerV9() { + return indexMergerV9; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index 055da6a3a48..dcede304884 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -34,6 +34,7 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -67,6 +68,7 @@ public class TaskToolboxFactory private final IndexIO indexIO; private final Cache cache; private final CacheConfig cacheConfig; + private final IndexMergerV9 indexMergerV9; @Inject public TaskToolboxFactory( @@ -87,7 +89,8 @@ public class TaskToolboxFactory IndexMerger indexMerger, IndexIO indexIO, Cache cache, - CacheConfig cacheConfig + CacheConfig cacheConfig, + IndexMergerV9 indexMergerV9 ) { this.config = config; @@ -108,6 +111,7 @@ public class TaskToolboxFactory this.indexIO = Preconditions.checkNotNull(indexIO, "Null IndexIO"); this.cache = cache; this.cacheConfig = cacheConfig; + this.indexMergerV9 = indexMergerV9; } public TaskToolbox build(Task task) @@ -133,7 +137,8 @@ public class TaskToolboxFactory indexMerger, indexIO, cache, - cacheConfig + cacheConfig, + indexMergerV9 ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 4330afe2e1c..40854146d16 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -48,6 +48,7 @@ import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.index.YeOldePlumberSchool; import io.druid.query.aggregation.hyperloglog.HyperLogLogCollector; +import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.IOConfig; @@ -134,7 +135,8 @@ public class IndexTask extends AbstractFixedIntervalTask null, null, shardSpec, - indexSpec + indexSpec, + null ); } @@ -353,12 +355,15 @@ public class IndexTask extends AbstractFixedIntervalTask final FireDepartmentMetrics metrics = new FireDepartmentMetrics(); final Firehose firehose = firehoseFactory.connect(ingestionSchema.getDataSchema().getParser()); final Supplier committerSupplier = Committers.supplierFromFirehose(firehose); + final IndexMerger indexMerger = ingestionSchema.getTuningConfig().getBuildV9Directly() + ? toolbox.getIndexMergerV9() + : toolbox.getIndexMerger(); final Plumber plumber = new YeOldePlumberSchool( interval, version, wrappedDataSegmentPusher, tmpDir, - toolbox.getIndexMerger(), + indexMerger, toolbox.getIndexIO() ).findPlumber( schema, @@ -434,7 +439,7 @@ public class IndexTask extends AbstractFixedIntervalTask this.dataSchema = dataSchema; this.ioConfig = ioConfig; - this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null, null) : tuningConfig; + this.tuningConfig = tuningConfig == null ? new IndexTuningConfig(0, 0, null, null, null) : tuningConfig; } @Override @@ -485,18 +490,21 @@ public class IndexTask extends AbstractFixedIntervalTask private static final int DEFAULT_TARGET_PARTITION_SIZE = 5000000; private static final int DEFAULT_ROW_FLUSH_BOUNDARY = 500000; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); + private static final Boolean DEFAULT_BUILD_V9_DIRECTLY = Boolean.FALSE; private final int targetPartitionSize; private final int rowFlushBoundary; private final int numShards; private final IndexSpec indexSpec; + private final Boolean buildV9Directly; @JsonCreator public IndexTuningConfig( @JsonProperty("targetPartitionSize") int targetPartitionSize, @JsonProperty("rowFlushBoundary") int rowFlushBoundary, @JsonProperty("numShards") @Nullable Integer numShards, - @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly ) { this.targetPartitionSize = targetPartitionSize == 0 ? DEFAULT_TARGET_PARTITION_SIZE : targetPartitionSize; @@ -508,6 +516,7 @@ public class IndexTask extends AbstractFixedIntervalTask this.targetPartitionSize == -1 || this.numShards == -1, "targetPartitionsSize and shardCount both cannot be set" ); + this.buildV9Directly = buildV9Directly == null ? DEFAULT_BUILD_V9_DIRECTLY : buildV9Directly; } @JsonProperty @@ -533,5 +542,11 @@ public class IndexTask extends AbstractFixedIntervalTask { return indexSpec; } + + @JsonProperty + public Boolean getBuildV9Directly() + { + return buildV9Directly; + } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index cfec3e08322..0822c55bbc8 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -47,6 +47,7 @@ import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerFactory; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QueryToolChest; +import io.druid.segment.IndexMerger; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -286,6 +287,9 @@ public class RealtimeIndexTask extends AbstractTask ); this.queryRunnerFactoryConglomerate = toolbox.getQueryRunnerFactoryConglomerate(); + IndexMerger indexMerger = spec.getTuningConfig().getBuildV9Directly() + ? toolbox.getIndexMergerV9() + : toolbox.getIndexMerger(); // NOTE: This pusher selects path based purely on global configuration and the DataSegment, which means // NOTE: that redundant realtime tasks will upload to the same location. This can cause index.zip and // NOTE: descriptor.json to mismatch, or it can cause historical nodes to load different instances of the @@ -298,7 +302,7 @@ public class RealtimeIndexTask extends AbstractTask segmentPublisher, toolbox.getSegmentHandoffNotifierFactory(), toolbox.getQueryExecutorService(), - toolbox.getIndexMerger(), + indexMerger, toolbox.getIndexIO(), toolbox.getCache(), toolbox.getCacheConfig(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 5a6da202126..86a513816ab 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -31,6 +31,7 @@ import io.druid.indexing.common.task.Task; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -77,6 +78,7 @@ public class TaskToolboxTest private SegmentLoaderLocalCacheManager mockSegmentLoaderLocalCacheManager = EasyMock.createMock(SegmentLoaderLocalCacheManager.class); private Task task = EasyMock.createMock(Task.class); private IndexMerger mockIndexMerger = EasyMock.createMock(IndexMerger.class); + private IndexMergerV9 mockIndexMergerV9 = EasyMock.createMock(IndexMergerV9.class); private IndexIO mockIndexIO = EasyMock.createMock(IndexIO.class); private Cache mockCache = EasyMock.createMock(Cache.class); private CacheConfig mockCacheConfig = EasyMock.createMock(CacheConfig.class); @@ -109,7 +111,8 @@ public class TaskToolboxTest mockIndexMerger, mockIndexIO, mockCache, - mockCacheConfig + mockCacheConfig, + mockIndexMergerV9 ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index ec323598714..b646dfdb2c2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -28,6 +28,7 @@ import io.druid.guice.ServerModule; import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.column.ColumnConfig; import java.util.List; @@ -39,6 +40,7 @@ public class TestUtils { private final ObjectMapper jsonMapper; private final IndexMerger indexMerger; + private final IndexMergerV9 indexMergerV9; private final IndexIO indexIO; public TestUtils() @@ -56,6 +58,7 @@ public class TestUtils } ); indexMerger = new IndexMerger(jsonMapper, indexIO); + indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO); final List list = new ServerModule().getJacksonModules(); for (Module module : list) { @@ -80,6 +83,10 @@ public class TestUtils return indexMerger; } + public IndexMergerV9 getTestIndexMergerV9() { + return indexMergerV9; + } + public IndexIO getTestIndexIO() { return indexIO; diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 156ba36dff1..e7f3ee80365 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -39,6 +39,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeTuningConfig; @@ -70,6 +71,7 @@ public class IndexTaskTest private final IndexSpec indexSpec; private final ObjectMapper jsonMapper; private IndexMerger indexMerger; + private IndexMergerV9 indexMergerV9; private IndexIO indexIO; public IndexTaskTest() @@ -78,6 +80,7 @@ public class IndexTaskTest TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); indexMerger = testUtils.getTestIndexMerger(); + indexMergerV9 = testUtils.getTestIndexMergerV9(); indexIO = testUtils.getTestIndexIO(); } @@ -140,7 +143,8 @@ public class IndexTaskTest 2, 0, null, - indexSpec + indexSpec, + null ) ), jsonMapper, @@ -252,7 +256,7 @@ public class IndexTaskTest return segment; } }, null, null, null, null, null, null, null, null, null, null, temporaryFolder.newFolder(), - indexMerger, indexIO, null, null + indexMerger, indexIO, null, null, indexMergerV9 ) ); @@ -332,7 +336,8 @@ public class IndexTaskTest 100, 1000, null, - new IndexSpec() + new IndexSpec(), + null ); RealtimeTuningConfig realtimeTuningConfig = IndexTask.convertTuningConfig( spec, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index b41b753cf7c..454b97e3285 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -571,6 +571,7 @@ public class RealtimeIndexTaskTest null, null, null, + null, null ); return new RealtimeIndexTask( @@ -715,7 +716,8 @@ public class RealtimeIndexTaskTest testUtils.getTestIndexMerger(), testUtils.getTestIndexIO(), MapCache.create(1024), - new CacheConfig() + new CacheConfig(), + testUtils.getTestIndexMergerV9() ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 32e668f2382..efb2fead456 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -91,7 +91,7 @@ public class TaskSerdeTest jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)), - new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) + new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null) ), jsonMapper, null @@ -132,7 +132,7 @@ public class TaskSerdeTest jsonMapper ), new IndexTask.IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null)), - new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) + new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null) ), jsonMapper, null @@ -332,7 +332,8 @@ public class TaskSerdeTest null, 1, new NoneShardSpec(), - indexSpec + indexSpec, + null ) ), null diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 0210067b170..37b1d01df36 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -64,6 +64,7 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.OnheapIncrementalIndex; @@ -109,12 +110,14 @@ public class IngestSegmentFirehoseFactoryTest { private static final ObjectMapper MAPPER; private static final IndexMerger INDEX_MERGER; + private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; static { TestUtils testUtils = new TestUtils(); MAPPER = setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); INDEX_MERGER = testUtils.getTestIndexMerger(); + INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); INDEX_IO = testUtils.getTestIndexIO(); } @@ -274,7 +277,8 @@ public class IngestSegmentFirehoseFactoryTest INDEX_MERGER, INDEX_IO, null, - null + null, + INDEX_MERGER_V9 ); Collection values = new LinkedList<>(); for (InputRowParser parser : Arrays.asList( diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 7b28bd90a54..bca91545fb6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -55,6 +55,7 @@ import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.filter.NoopDimFilter; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; @@ -111,12 +112,14 @@ public class IngestSegmentFirehoseFactoryTimelineTest private static final ObjectMapper MAPPER; private static final IndexMerger INDEX_MERGER; private static final IndexIO INDEX_IO; + private static final IndexMergerV9 INDEX_MERGER_V9; static { TestUtils testUtils = new TestUtils(); MAPPER = IngestSegmentFirehoseFactoryTest.setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); INDEX_MERGER = testUtils.getTestIndexMerger(); INDEX_IO = testUtils.getTestIndexIO(); + INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); } public IngestSegmentFirehoseFactoryTimelineTest( @@ -334,7 +337,8 @@ public class IngestSegmentFirehoseFactoryTimelineTest INDEX_MERGER, INDEX_IO, null, - null + null, + INDEX_MERGER_V9 ); final Injector injector = Guice.createInjector( new Module() diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 8b0f02a8479..825d42cd963 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -82,6 +82,7 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; @@ -132,12 +133,14 @@ public class TaskLifecycleTest { private static final ObjectMapper MAPPER; private static final IndexMerger INDEX_MERGER; + private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; static { TestUtils testUtils = new TestUtils(); MAPPER = testUtils.getTestObjectMapper(); INDEX_MERGER = testUtils.getTestIndexMerger(); + INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); INDEX_IO = testUtils.getTestIndexIO(); } @@ -532,7 +535,8 @@ public class TaskLifecycleTest INDEX_MERGER, INDEX_IO, MapCache.create(0), - FireDepartmentTest.NO_CACHE_CONFIG + FireDepartmentTest.NO_CACHE_CONFIG, + INDEX_MERGER_V9 ); tr = new ThreadPoolTaskRunner(tb, taskConfig, emitter); tq = new TaskQueue(tqc, ts, tr, tac, tl, emitter); @@ -565,7 +569,7 @@ public class TaskLifecycleTest mapper ), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)), - new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) + new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null) ), mapper, null @@ -623,7 +627,7 @@ public class TaskLifecycleTest mapper ), new IndexTask.IndexIOConfig(new MockExceptionalFirehoseFactory()), - new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) + new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null) ), mapper, null @@ -969,7 +973,7 @@ public class TaskLifecycleTest mapper ), new IndexTask.IndexIOConfig(new MockFirehoseFactory(false)), - new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec) + new IndexTask.IndexTuningConfig(10000, 10, -1, indexSpec, null) ), mapper, null @@ -1080,6 +1084,7 @@ public class TaskLifecycleTest null, null, null, + null, null ); FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig); diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index c021dd4f33f..8996794fd9e 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -41,6 +41,7 @@ import io.druid.indexing.overlord.ThreadPoolTaskRunner; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; +import io.druid.segment.IndexMergerV9; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; import io.druid.segment.loading.StorageLocationConfig; @@ -80,6 +81,7 @@ public class WorkerTaskMonitorTest private Worker worker; private ObjectMapper jsonMapper; private IndexMerger indexMerger; + private IndexMergerV9 indexMergerV9; private IndexIO indexIO; public WorkerTaskMonitorTest() @@ -87,6 +89,7 @@ public class WorkerTaskMonitorTest TestUtils testUtils = new TestUtils(); jsonMapper = testUtils.getTestObjectMapper(); indexMerger = testUtils.getTestIndexMerger(); + indexMergerV9 = testUtils.getTestIndexMergerV9(); indexIO = testUtils.getTestIndexIO(); } @@ -183,7 +186,8 @@ public class WorkerTaskMonitorTest indexMerger, indexIO, null, - null + null, + indexMergerV9 ), taskConfig, new NoopServiceEmitter() diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index da573e1eabc..039303eb0e1 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -104,12 +104,12 @@ public class IndexMerger { private static final Logger log = new Logger(IndexMerger.class); - private static final SerializerUtils serializerUtils = new SerializerUtils(); - private static final int INVALID_ROW = -1; - private static final Splitter SPLITTER = Splitter.on(","); + protected static final SerializerUtils serializerUtils = new SerializerUtils(); + protected static final int INVALID_ROW = -1; + protected static final Splitter SPLITTER = Splitter.on(","); - private final ObjectMapper mapper; - private final IndexIO indexIO; + protected final ObjectMapper mapper; + protected final IndexIO indexIO; @Inject public IndexMerger( @@ -473,7 +473,7 @@ public class IndexMerger return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec); } - private File makeIndexFiles( + protected File makeIndexFiles( final List indexes, final File outDir, final ProgressIndicator progress, diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index e83b76a60dc..8cc5e8e3045 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -100,335 +100,21 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; -public class IndexMergerV9 +public class IndexMergerV9 extends IndexMerger { private static final Logger log = new Logger(IndexMergerV9.class); - private static final SerializerUtils serializerUtils = new SerializerUtils(); - private static final int INVALID_ROW = -1; - private static final Splitter SPLITTER = Splitter.on(","); - - private final ObjectMapper mapper; - private final IndexIO indexIO; - @Inject public IndexMergerV9( ObjectMapper mapper, IndexIO indexIO ) { - this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); - this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); + super(mapper, indexIO); } - public File persist( - final IncrementalIndex index, - File outDir, - Map segmentMetadata, - IndexSpec indexSpec - ) throws IOException - { - return persist(index, index.getInterval(), outDir, segmentMetadata, indexSpec); - } - - public File persist( - final IncrementalIndex index, - final Interval dataInterval, - File outDir, - Map segmentMetadata, - IndexSpec indexSpec - ) throws IOException - { - return persist( - index, - dataInterval, - outDir, - segmentMetadata, - indexSpec, - new BaseProgressIndicator() - ); - } - - public File persist( - final IncrementalIndex index, - final Interval dataInterval, - File outDir, - Map segmentMetadata, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - if (index.isEmpty()) { - throw new IAE("Trying to persist an empty index!"); - } - - final long firstTimestamp = index.getMinTime().getMillis(); - final long lastTimestamp = index.getMaxTime().getMillis(); - if (!(dataInterval.contains(firstTimestamp) && dataInterval.contains(lastTimestamp))) { - throw new IAE( - "interval[%s] does not encapsulate the full range of timestamps[%s, %s]", - dataInterval, - new DateTime(firstTimestamp), - new DateTime(lastTimestamp) - ); - } - - if (!outDir.exists()) { - outDir.mkdirs(); - } - if (!outDir.isDirectory()) { - throw new ISE("Can only persist to directories, [%s] wasn't a directory", outDir); - } - - log.info("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); - return merge( - Arrays.asList( - new IncrementalIndexAdapter( - dataInterval, - index, - indexSpec.getBitmapSerdeFactory().getBitmapFactory() - ) - ), - index.getMetricAggs(), - outDir, - segmentMetadata, - indexSpec, - progress - ); - } - - public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException - { - return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File convert( - final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress - ) throws IOException - { - try (QueryableIndex index = indexIO.loadIndex(inDir)) { - final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); - return makeIndexFiles( - ImmutableList.of(adapter), - outDir, - progress, - Lists.newArrayList(adapter.getDimensionNames()), - Lists.newArrayList(adapter.getMetricNames()), - null, - new Function>, Iterable>() - { - @Nullable - @Override - public Iterable apply(ArrayList> input) - { - return input.get(0); - } - }, - indexSpec - ); - } - } - - public File mergeQueryableIndex( - List indexes, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec - ) throws IOException - { - return mergeQueryableIndex(indexes, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File mergeQueryableIndex( - List indexes, - final AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - // We are materializing the list for performance reasons. Lists.transform - // only creates a "view" of the original list, meaning the function gets - // applied every time you access an element. - List indexAdapteres = Lists.newArrayList( - Iterables.transform( - indexes, - new Function() - { - @Override - public IndexableAdapter apply(final QueryableIndex input) - { - return new QueryableIndexIndexableAdapter(input); - } - } - ) - ); - return merge( - indexAdapteres, - metricAggs, - outDir, - null, - indexSpec, - progress - ); - } - - public File merge( - List indexes, - final AggregatorFactory[] metricAggs, - File outDir, - Map segmentMetadata, - IndexSpec indexSpec - ) throws IOException - { - return merge(indexes, metricAggs, outDir, segmentMetadata, indexSpec, new BaseProgressIndicator()); - } - - public File merge( - List adapters, - final AggregatorFactory[] metricAggs, - File outDir, - Map segmentMetadata, - IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException - { - FileUtils.deleteDirectory(outDir); - if (!outDir.mkdirs()) { - throw new ISE("Couldn't make outdir[%s].", outDir); - } - - final List mergedDimensions = IndexMerger.getMergedDimensions(adapters); - - final List mergedMetrics = Lists.transform( - IndexMerger.mergeIndexed( - Lists.newArrayList( - FunctionalIterable - .create(adapters) - .transform( - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return input.getMetricNames(); - } - } - ) - .concat(Arrays.>asList(new IndexMerger.AggFactoryStringIndexed(metricAggs))) - ) - ), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } - ); - if (mergedMetrics.size() != metricAggs.length) { - throw new IAE("Bad number of metrics[%d], expected [%d]", mergedMetrics.size(), metricAggs.length); - } - - final AggregatorFactory[] sortedMetricAggs = new AggregatorFactory[mergedMetrics.size()]; - for (int i = 0; i < metricAggs.length; i++) { - AggregatorFactory metricAgg = metricAggs[i]; - sortedMetricAggs[mergedMetrics.indexOf(metricAgg.getName())] = metricAgg; - } - - for (int i = 0; i < mergedMetrics.size(); i++) { - if (!sortedMetricAggs[i].getName().equals(mergedMetrics.get(i))) { - throw new IAE( - "Metric mismatch, index[%d] [%s] != [%s]", - i, - metricAggs[i].getName(), - mergedMetrics.get(i) - ); - } - } - - Function>, Iterable> rowMergerFn = - new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable ArrayList> boats - ) - { - return CombiningIterable.create( - new MergeIterable<>( - Ordering.natural().nullsFirst(), - boats - ), - Ordering.natural().nullsFirst(), - new IndexMerger.RowboatMergeFunction(sortedMetricAggs) - ); - } - }; - - return makeIndexFiles( - adapters, outDir, progress, mergedDimensions, mergedMetrics, segmentMetadata, rowMergerFn, indexSpec - ); - } - - public File append( - List indexes, File outDir, IndexSpec indexSpec - ) throws IOException - { - return append(indexes, outDir, indexSpec, new BaseProgressIndicator()); - } - - public File append( - List indexes, File outDir, IndexSpec indexSpec, ProgressIndicator progress - ) throws IOException - { - FileUtils.deleteDirectory(outDir); - if (!outDir.mkdirs()) { - throw new ISE("Couldn't make outdir[%s].", outDir); - } - - final List mergedDimensions = IndexMerger.getMergedDimensions(indexes); - - final List mergedMetrics = IndexMerger.mergeIndexed( - Lists.transform( - indexes, - new Function>() - { - @Override - public Iterable apply(@Nullable IndexableAdapter input) - { - return Iterables.transform( - input.getMetricNames(), - new Function() - { - @Override - public String apply(@Nullable String input) - { - return input; - } - } - ); - } - } - ) - ); - - Function>, Iterable> rowMergerFn = new Function>, Iterable>() - { - @Override - public Iterable apply( - @Nullable final ArrayList> boats - ) - { - return new MergeIterable( - Ordering.natural().nullsFirst(), - boats - ); - } - }; - - return makeIndexFiles(indexes, outDir, progress, mergedDimensions, mergedMetrics, null, rowMergerFn, indexSpec); - } - - private File makeIndexFiles( + @Override + protected File makeIndexFiles( final List adapters, final File outDir, final ProgressIndicator progress, @@ -829,7 +515,7 @@ public class IndexMergerV9 List> convertedInverteds = Lists.newArrayListWithCapacity(adapters.size()); for (int j = 0; j < adapters.size(); ++j) { convertedInverteds.add( - new IndexMerger.ConvertingIndexedInts( + new ConvertingIndexedInts( bitmapIndexSeeker[j].seek(dimVal), rowNumConversions.get(j) ) ); @@ -1094,7 +780,7 @@ public class IndexMergerV9 } boats.add( - new IndexMerger.MMappedIndexRowIterable( + new MMappedIndexRowIterable( Iterables.transform( adapters.get(i).getRows(), new Function() @@ -1177,14 +863,14 @@ public class IndexMergerV9 List> dimValueLookups = Lists.newArrayListWithCapacity(adapters.size()); // each converter converts dim values of this dimension to global dictionary - IndexMerger.DimValueConverter[] converters = new IndexMerger.DimValueConverter[adapters.size()]; + DimValueConverter[] converters = new DimValueConverter[adapters.size()]; boolean existNullColumn = false; for (int i = 0; i < adapters.size(); i++) { Indexed dimValues = adapters.get(i).getDimValueLookup(dimension); - if (!IndexMerger.isNullColumn(dimValues)) { + if (!isNullColumn(dimValues)) { dimValueLookups.add(dimValues); - converters[i] = new IndexMerger.DimValueConverter(dimValues); + converters[i] = new DimValueConverter(dimValues); } else { existNullColumn = true; } @@ -1233,7 +919,7 @@ public class IndexMergerV9 writer.write(value); for (int i = 0; i < adapters.size(); i++) { - IndexMerger.DimValueConverter converter = converters[i]; + DimValueConverter converter = converters[i]; if (converter != null) { converter.convert(value, cardinality); } @@ -1259,7 +945,7 @@ public class IndexMergerV9 // make the conversion for (int i = 0; i < adapters.size(); ++i) { - IndexMerger.DimValueConverter converter = converters[i]; + DimValueConverter converter = converters[i]; if (converter != null) { dimConversions.get(i).put(dimension, converters[i].getConversionBuffer()); } diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 4be0ccf4d91..adb1c054a59 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -46,6 +46,7 @@ public class RealtimeTuningConfig implements TuningConfig private static final int defaultMaxPendingPersists = 0; private static final ShardSpec defaultShardSpec = new NoneShardSpec(); private static final IndexSpec defaultIndexSpec = new IndexSpec(); + private static final Boolean defaultBuildV9Directly = Boolean.FALSE; // Might make sense for this to be a builder public static RealtimeTuningConfig makeDefaultTuningConfig() @@ -59,7 +60,8 @@ public class RealtimeTuningConfig implements TuningConfig defaultRejectionPolicyFactory, defaultMaxPendingPersists, defaultShardSpec, - defaultIndexSpec + defaultIndexSpec, + defaultBuildV9Directly ); } @@ -72,6 +74,7 @@ public class RealtimeTuningConfig implements TuningConfig private final int maxPendingPersists; private final ShardSpec shardSpec; private final IndexSpec indexSpec; + private final Boolean buildV9Directly; @JsonCreator public RealtimeTuningConfig( @@ -83,7 +86,8 @@ public class RealtimeTuningConfig implements TuningConfig @JsonProperty("rejectionPolicy") RejectionPolicyFactory rejectionPolicyFactory, @JsonProperty("maxPendingPersists") Integer maxPendingPersists, @JsonProperty("shardSpec") ShardSpec shardSpec, - @JsonProperty("indexSpec") IndexSpec indexSpec + @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("buildV9Directly") Boolean buildV9Directly ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -99,6 +103,7 @@ public class RealtimeTuningConfig implements TuningConfig this.maxPendingPersists = maxPendingPersists == null ? defaultMaxPendingPersists : maxPendingPersists; this.shardSpec = shardSpec == null ? defaultShardSpec : shardSpec; this.indexSpec = indexSpec == null ? defaultIndexSpec : indexSpec; + this.buildV9Directly = buildV9Directly == null ? defaultBuildV9Directly : buildV9Directly; } @JsonProperty @@ -155,6 +160,11 @@ public class RealtimeTuningConfig implements TuningConfig return indexSpec; } + @JsonProperty + public Boolean getBuildV9Directly() { + return buildV9Directly; + } + public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( @@ -166,7 +176,8 @@ public class RealtimeTuningConfig implements TuningConfig rejectionPolicyFactory, maxPendingPersists, shardSpec, - indexSpec + indexSpec, + buildV9Directly ); } @@ -181,7 +192,8 @@ public class RealtimeTuningConfig implements TuningConfig rejectionPolicyFactory, maxPendingPersists, shardSpec, - indexSpec + indexSpec, + buildV9Directly ); } } diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 9117ed21bd3..b76a795074a 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -116,7 +116,7 @@ public class FireDepartmentTest null ), new RealtimeTuningConfig( - null, null, null, null, null, null, null, null, null + null, null, null, null, null, null, null, null, null, null ) ); diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index f078d5ae2c4..5fc212061aa 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -150,6 +150,7 @@ public class RealtimeManagerTest null, null, null, + null, null ); plumber = new TestPlumber(new Sink(new Interval("0/P5000Y"), schema, tuningConfig, new DateTime().toString())); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index c20451f6236..747b68d5992 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -185,6 +185,7 @@ public class RealtimePlumberSchoolTest rejectionPolicy, null, null, + null, null ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 9a5664c2b9e..41024a2abb9 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -66,6 +66,7 @@ public class SinkTest null, null, null, + null, null ); final Sink sink = new Sink(interval, schema, tuningConfig, version);