This is an automated email from the ASF dual-hosted git repository. apucher pushed a commit to branch data-and-query-gen-bytes-support in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git
commit f87b94049761a918c4e0cd7cca44260d3763f54f Author: Alexander Pucher <a...@alexpucher.com> AuthorDate: Wed Nov 4 12:30:49 2020 -0800 add support for BYTES columns in data and query generators --- .../pinot/integration/tests/QueryGenerator.java | 15 +++++-- .../plugin/inputformat/avro/AvroSchemaUtil.java | 3 ++ .../tools/admin/command/GenerateDataCommand.java | 13 +++--- .../pinot/tools/data/generator/DataGenerator.java | 6 +-- .../tools/data/generator/DataGeneratorSpec.java | 13 ++++-- .../tools/data/generator/GeneratorFactory.java | 3 ++ .../tools/data/generator/NumberGenerator.java | 7 ++++ .../tools/data/generator/RangeBytesGenerator.java | 49 ++++++++++++++++++++++ 8 files changed, 94 insertions(+), 15 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java index 323e6e0..e57f089 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/QueryGenerator.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -37,6 +38,7 @@ import org.apache.avro.file.DataFileReader; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; +import org.apache.commons.codec.binary.Hex; import org.apache.commons.lang.StringUtils; import org.apache.pinot.spi.utils.JsonUtils; @@ -133,7 +135,7 @@ public class QueryGenerator { _multiValueColumnMaxNumElements.put(fieldName, 0); } else { _singleValueColumnNames.add(fieldName); - if (type != Schema.Type.STRING && type != Schema.Type.BOOLEAN) { + if (type != Schema.Type.STRING && type != Schema.Type.BOOLEAN && type != Schema.Type.BYTES) { _singleValueNumericalColumnNames.add(fieldName); } } @@ -182,7 +184,12 @@ public class QueryGenerator { * @param avroValue Avro value. */ private static void storeAvroValueIntoValueSet(Set<String> valueSet, Object avroValue) { - if (avroValue instanceof Number) { + if (avroValue instanceof ByteBuffer) { + // for raw bytes + String hexRaw = StringUtils.stripStart(Hex.encodeHexString(((ByteBuffer) avroValue).array()), "0"); + String hexAligned = (hexRaw.length() & 0x1) == 0 ? hexRaw : "0" + hexRaw; + valueSet.add("'" + hexAligned + "'"); + } else if (avroValue instanceof Number) { // For Number object, store raw value. valueSet.add(avroValue.toString()); } else { @@ -1012,8 +1019,8 @@ public class QueryGenerator { } } - String pql = String.format(" REGEXP_LIKE(%s, '%s')", columnName, pqlRegexBuilder.toString()); - String sql = String.format(" REGEXP_LIKE(%s, '%s', 'i')", columnName, sqlRegexBuilder.toString()); + String pql = String.format("REGEXP_LIKE(%s, '%s')", columnName, pqlRegexBuilder.toString()); + String sql = String.format("REGEXP_LIKE(%s, '%s', 'i')", columnName, sqlRegexBuilder.toString()); return new StringQueryFragment(pql, sql); } else { String equalsPredicate = String.format("%s = %s", columnName, value); diff --git a/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java b/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java index 9ff22ee..d19aea7 100644 --- a/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java +++ b/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroSchemaUtil.java @@ -69,6 +69,9 @@ public class AvroSchemaUtil { case STRING: jsonSchema.set("type", convertStringsToJsonArray("null", "string")); return jsonSchema; + case BYTES: + jsonSchema.set("type", convertStringsToJsonArray("null", "bytes")); + return jsonSchema; default: throw new UnsupportedOperationException(); } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java index fe3e182..598f4a8 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/GenerateDataCommand.java @@ -127,6 +127,7 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com List<String> columns = new LinkedList<>(); final HashMap<String, DataType> dataTypes = new HashMap<>(); final HashMap<String, FieldType> fieldTypes = new HashMap<>(); + final HashMap<String, Boolean> singleValueFlags = new HashMap<>(); final HashMap<String, TimeUnit> timeUnits = new HashMap<>(); final HashMap<String, Integer> cardinality = new HashMap<>(); @@ -136,7 +137,7 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com buildCardinalityRangeMaps(_schemaAnnFile, cardinality, range, pattern); final DataGeneratorSpec spec = - buildDataGeneratorSpec(schema, columns, dataTypes, fieldTypes, timeUnits, cardinality, range, pattern); + buildDataGeneratorSpec(schema, columns, dataTypes, fieldTypes, singleValueFlags, timeUnits, cardinality, range, pattern); final DataGenerator gen = new DataGenerator(); gen.init(spec); @@ -175,14 +176,16 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com } private DataGeneratorSpec buildDataGeneratorSpec(Schema schema, List<String> columns, - HashMap<String, DataType> dataTypes, HashMap<String, FieldType> fieldTypes, HashMap<String, TimeUnit> timeUnits, - HashMap<String, Integer> cardinality, HashMap<String, IntRange> range, HashMap<String, Map<String, Object>> pattern) { + HashMap<String, DataType> dataTypes, HashMap<String, FieldType> fieldTypes, HashMap<String, Boolean> singleValueFlags, + HashMap<String, TimeUnit> timeUnits, HashMap<String, Integer> cardinality, HashMap<String, IntRange> range, + HashMap<String, Map<String, Object>> pattern) { for (final FieldSpec fs : schema.getAllFieldSpecs()) { String col = fs.getName(); columns.add(col); dataTypes.put(col, fs.getDataType()); fieldTypes.put(col, fs.getFieldType()); + singleValueFlags.put(col, fs.isSingleValueField()); switch (fs.getFieldType()) { case DIMENSION: @@ -215,8 +218,8 @@ public class GenerateDataCommand extends AbstractBaseAdminCommand implements Com } } - return new DataGeneratorSpec(columns, cardinality, range, pattern, dataTypes, fieldTypes, timeUnits, FileFormat.AVRO, - _outDir, _overwrite); + return new DataGeneratorSpec(columns, cardinality, range, pattern, dataTypes, fieldTypes, singleValueFlags, + timeUnits, FileFormat.AVRO, _outDir, _overwrite); } public static void main(String[] args) diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java index 2cdc1cc..428ea59 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java @@ -156,7 +156,7 @@ public class DataGenerator { spec.setName(column); spec.setDataType(dataType); - spec.setSingleValueField(true); + spec.setSingleValueField(genSpec.getSingleValueFlagsMap().getOrDefault(column, true)); return spec; } @@ -178,8 +178,8 @@ public class DataGenerator { cardinality.put(col, 1000); } final DataGeneratorSpec spec = - new DataGeneratorSpec(Arrays.asList(columns), cardinality, range, template, dataTypes, fieldTypes, timeUnits, - FileFormat.AVRO, "/tmp/out", true); + new DataGeneratorSpec(Arrays.asList(columns), cardinality, range, template, dataTypes, fieldTypes, + new HashMap<>(), timeUnits, FileFormat.AVRO, "/tmp/out", true); final DataGenerator gen = new DataGenerator(); gen.init(spec); diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java index c98cd4d..005da05 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGeneratorSpec.java @@ -41,6 +41,7 @@ public class DataGeneratorSpec { private final Map<String, DataType> dataTypesMap; private final Map<String, FieldType> fieldTypesMap; + private final Map<String, Boolean> singleValueFlagsMap; private final Map<String, TimeUnit> timeUnitMap; private final FileFormat outputFileFormat; @@ -49,13 +50,14 @@ public class DataGeneratorSpec { public DataGeneratorSpec() { this(new ArrayList<String>(), new HashMap<>(), new HashMap<>(), new HashMap<>(), - new HashMap<>(), new HashMap<>(), new HashMap<>(), + new HashMap<>(), new HashMap<>(), new HashMap<>(), new HashMap<>(), FileFormat.AVRO, "/tmp/dataGen", true); } public DataGeneratorSpec(List<String> columns, Map<String, Integer> cardinalityMap, Map<String, IntRange> rangeMap, - Map<String, Map<String, Object>> patternMap, Map<String, DataType> dataTypesMap, Map<String, FieldType> fieldTypesMap, Map<String, TimeUnit> timeUnitMap, - FileFormat format, String outputDir, boolean override) { + Map<String, Map<String, Object>> patternMap, Map<String, DataType> dataTypesMap, Map<String, FieldType> fieldTypesMap, + Map<String, Boolean> singleValueFlagsMap, Map<String, TimeUnit> timeUnitMap, FileFormat format, String outputDir, + boolean override) { this.columns = columns; this.cardinalityMap = cardinalityMap; this.rangeMap = rangeMap; @@ -67,6 +69,7 @@ public class DataGeneratorSpec { this.dataTypesMap = dataTypesMap; this.fieldTypesMap = fieldTypesMap; + this.singleValueFlagsMap = singleValueFlagsMap; this.timeUnitMap = timeUnitMap; } @@ -78,6 +81,10 @@ public class DataGeneratorSpec { return fieldTypesMap; } + public Map<String, Boolean> getSingleValueFlagsMap() { + return singleValueFlagsMap; + } + public Map<String, TimeUnit> getTimeUnitMap() { return timeUnitMap; } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java index 215bf9f..5430610 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/GeneratorFactory.java @@ -18,6 +18,7 @@ */ package org.apache.pinot.tools.data.generator; +import org.apache.commons.lang.RandomStringUtils; import org.apache.pinot.spi.data.FieldSpec.DataType; import java.util.Map; @@ -46,6 +47,8 @@ public class GeneratorFactory { return new RangeFloatGenerator(start, end); case DOUBLE: return new RangeDoubleGenerator(start, end); + case BYTES: + return new RangeBytesGenerator(start, end); default: throw new RuntimeException(String.format("Invalid datatype '%s'", dataType)); } diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/NumberGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/NumberGenerator.java index 32d697f..9900cfd 100644 --- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/NumberGenerator.java +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/NumberGenerator.java @@ -18,9 +18,12 @@ */ package org.apache.pinot.tools.data.generator; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; + +import com.google.common.primitives.Longs; import org.apache.pinot.spi.data.FieldSpec.DataType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +67,8 @@ public class NumberGenerator implements Generator { intValues.add(new Integer(i)); } break; + case BYTES: + // use long case case LONG: longValues = new ArrayList<Long>(); final long longStart = rand.nextInt(cardinality); @@ -113,6 +118,8 @@ public class NumberGenerator implements Generator { return floatValues.get(random.nextInt(cardinality)); case DOUBLE: return doubleValues.get(random.nextInt(cardinality)); + case BYTES: + return ByteBuffer.wrap(Longs.toByteArray(longValues.get(random.nextInt(cardinality)))); default: throw new RuntimeException("number generator can only accept a column of type number and this : " + columnType + " is not a supported number type"); diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/RangeBytesGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/RangeBytesGenerator.java new file mode 100644 index 0000000..a29de09 --- /dev/null +++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/RangeBytesGenerator.java @@ -0,0 +1,49 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pinot.tools.data.generator; + +import com.google.common.primitives.Ints; + +import java.nio.ByteBuffer; +import java.util.Random; + + +public class RangeBytesGenerator implements Generator { + private final int _start; + private final int _end; + private final int _delta; + + Random _randGen = new Random(System.currentTimeMillis()); + + public RangeBytesGenerator(int r1, int r2) { + _start = (r1 < r2) ? r1 : r2; + _end = (r1 > r2) ? r1 : r2; + + _delta = _end - _start; + } + + @Override + public void init() { + } + + @Override + public Object next() { + return ByteBuffer.wrap(Ints.toByteArray(_start + _randGen.nextInt(_delta))); + } +} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org