From e9741ce59621209e18cf405c8fe4a614d955ed2a Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Wed, 21 Dec 2016 21:47:20 -0800 Subject: [PATCH 1/3] DRILL-5152: Enhance the mock data source: better data, SQL access Provides an enhanced version of the mock data source. See the JIRA entry for motivation, package-info.java for details of operation. --- .../exec/store/AbstractRecordReader.java | 2 +- .../drill/exec/store/StoragePlugin.java | 3 +- .../exec/store/StoragePluginRegistry.java | 2 +- .../exec/store/StoragePluginRegistryImpl.java | 24 ++- .../drill/exec/store/mock/ColumnDef.java | 178 ++++++++++++++++++ .../apache/drill/exec/store/mock/DateGen.java | 53 ++++++ .../drill/exec/store/mock/DoubleGen.java | 42 +++++ .../store/mock/ExtendedMockRecordReader.java | 149 +++++++++++++++ .../drill/exec/store/mock/FieldGen.java | 37 ++++ .../apache/drill/exec/store/mock/IntGen.java | 42 +++++ .../exec/store/mock/MockGroupScanPOP.java | 127 +++++++++---- .../exec/store/mock/MockRecordReader.java | 8 +- .../exec/store/mock/MockScanBatchCreator.java | 8 +- .../exec/store/mock/MockStorageEngine.java | 79 +++++++- .../store/mock/MockStorageEngineConfig.java | 9 +- .../drill/exec/store/mock/MockStorePOP.java | 3 +- .../drill/exec/store/mock/MockSubScanPOP.java | 20 +- .../drill/exec/store/mock/MoneyGen.java | 42 +++++ .../drill/exec/store/mock/StringGen.java | 49 +++++ .../drill/exec/store/mock/package-info.java | 130 +++++++++++++ .../drill/exec/TestOpSerialization.java | 2 +- .../fn/interp/ExpressionInterpreterTest.java | 4 +- 22 files changed, 936 insertions(+), 77 deletions(-) create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java create mode 100644 exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java index 16118d92418..2152025631b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java @@ -67,7 +67,7 @@ protected final void setColumns(Collection projected) { Collection columnsToRead = projected; // If no column is required (SkipQuery), by default it will use DEFAULT_COLS_TO_READ . - // Handling SkipQuery is storage-plugin specif : JSON, text reader, parquet will override, in order to + // Handling SkipQuery is storage-plugin specific : JSON, text reader, parquet will override, in order to // improve query performance. if (projected.isEmpty()) { columnsToRead = getDefaultColumnsToRead(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java index 112bc154989..8324e3062ba 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java @@ -71,7 +71,7 @@ public interface StoragePlugin extends SchemaFactory, AutoCloseable { public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List columns) throws IOException; - /** Method returns a jackson serializable object that extends a StoragePluginConfig + /** Method returns a Jackson serializable object that extends a StoragePluginConfig * @return an extension of StoragePluginConfig */ public StoragePluginConfig getConfig(); @@ -80,5 +80,4 @@ public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, * Initialize the storage plugin. The storage plugin will not be used until this method is called. */ public void start() throws IOException; - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java index 7018ce8aa14..da29978e889 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java @@ -51,7 +51,7 @@ public interface StoragePluginRegistry extends Iterable createPlugins() throws DrillbitStartupException { @@ -145,7 +147,7 @@ private Map createPlugins() throws DrillbitStartupExcepti String pluginsData = Resources.toString(url, Charsets.UTF_8); StoragePlugins plugins = lpPersistence.getMapper().readValue(pluginsData, StoragePlugins.class); for (Map.Entry config : plugins) { - if (!pluginSystemTable.putIfAbsent(config.getKey(), config.getValue())) { + if (!definePluginConfig(config.getKey(), config.getValue())) { logger.warn("Duplicate plugin instance '{}' defined in [{}, {}], ignoring the later one.", config.getKey(), pluginURLMap.get(config.getKey()), url); continue; @@ -185,6 +187,24 @@ private Map createPlugins() throws DrillbitStartupExcepti } } + /** + * Add a plugin and configuration. Assumes neither exists. Primarily + * for testing. + * + * @param name plugin name + * @param config plugin config + * @param plugin plugin implementation + */ + + public void definePlugin(String name, StoragePluginConfig config, StoragePlugin plugin) { + addPlugin(name, plugin); + definePluginConfig(name, config); + } + + private boolean definePluginConfig(String name, StoragePluginConfig config) { + return pluginSystemTable.putIfAbsent(name, config); + } + @Override public void addPlugin(String name, StoragePlugin plugin) { plugins.put(name, plugin); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java new file mode 100644 index 00000000000..6fc94c445fc --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java @@ -0,0 +1,178 @@ +/* + * 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.drill.exec.store.mock; + +import org.apache.drill.common.types.TypeProtos.MinorType; +import org.apache.drill.exec.expr.TypeHelper; +import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn; + +/** + * Defines a column for the "enhanced" version of the mock data + * source. This class is built from the column definitions in either + * the physical plan or an SQL statement (which gives rise to a + * physical plan.) + */ + +public class ColumnDef { + public MockColumn mockCol; + public String name; + public int width; + public FieldGen generator; + + public ColumnDef( MockColumn mockCol ) { + this.mockCol = mockCol; + name = mockCol.getName(); + width = TypeHelper.getSize(mockCol.getMajorType()); + makeGenerator( ); + } + + /** + * Create the data generator class for this column. The generator is + * created to match the data type by default. Or, the plan can + * specify a generator class (in which case the plan must ensure that + * the generator produces the correct value for the column data type.) + * The generator names a class: either a fully qualified name, or a + * class in this package. + */ + + private void makeGenerator( ) { + String genName = mockCol.getGenerator( ); + if ( genName != null ) { + if ( ! genName.contains(".") ) { + genName = "org.apache.drill.exec.store.mock." + genName; + } + try { + ClassLoader cl = getClass( ).getClassLoader(); + Class genClass = cl.loadClass(genName); + generator = (FieldGen) genClass.newInstance( ); + } catch (ClassNotFoundException | InstantiationException + | IllegalAccessException | ClassCastException e) { + throw new IllegalArgumentException( "Generator " + genName + " is undefined for mock field " + name ); + } + generator.setup( this ); + return; + } + + makeDefaultGenerator( ); + } + + private void makeDefaultGenerator( ) { + + MinorType minorType = mockCol.getMinorType(); + switch ( minorType ) { + case BIGINT: + break; + case BIT: + break; + case DATE: + break; + case DECIMAL18: + break; + case DECIMAL28DENSE: + break; + case DECIMAL28SPARSE: + break; + case DECIMAL38DENSE: + break; + case DECIMAL38SPARSE: + break; + case DECIMAL9: + break; + case FIXED16CHAR: + break; + case FIXEDBINARY: + break; + case FIXEDCHAR: + break; + case FLOAT4: + break; + case FLOAT8: + generator = new DoubleGen( ); + break; + case GENERIC_OBJECT: + break; + case INT: + generator = new IntGen( ); + break; + case INTERVAL: + break; + case INTERVALDAY: + break; + case INTERVALYEAR: + break; + case LATE: + break; + case LIST: + break; + case MAP: + break; + case MONEY: + break; + case NULL: + break; + case SMALLINT: + break; + case TIME: + break; + case TIMESTAMP: + break; + case TIMESTAMPTZ: + break; + case TIMETZ: + break; + case TINYINT: + break; + case UINT1: + break; + case UINT2: + break; + case UINT4: + break; + case UINT8: + break; + case UNION: + break; + case VAR16CHAR: + break; + case VARBINARY: + break; + case VARCHAR: + generator = new StringGen( ); + break; + default: + break; + } + if ( generator == null ) { + throw new IllegalArgumentException( "No default column generator for column " + name + " of type " + minorType ); + } + generator.setup(this); + } + + public ColumnDef( MockColumn mockCol, int rep ) { + this( mockCol ); + name = name += Integer.toString(rep); + } + + public MockColumn getConfig() { + return mockCol; + } + + public String getName() { + return name; + } +} \ No newline at end of file diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java new file mode 100644 index 00000000000..759fa3f8c3b --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java @@ -0,0 +1,53 @@ +/* + * 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.drill.exec.store.mock; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Random; + +import org.apache.drill.exec.vector.ValueVector; +import org.apache.drill.exec.vector.VarCharVector; + +public class DateGen implements FieldGen { + + private Random rand = new Random( ); + private long baseTime; + private SimpleDateFormat fmt; + + public DateGen( ) { + // Start a year ago. + baseTime = System.currentTimeMillis() - 365 * 24 * 60 * 60 * 1000; + fmt = new SimpleDateFormat( "yyyy-mm-DD" ); + } + + @Override + public void setup(ColumnDef colDef) { } + + public long value( ) { + return baseTime + rand.nextInt( 365 ) * 24 * 60 * 60 * 1000; + } + + @Override + public void setValue( ValueVector v, int index ) { + VarCharVector vector = (VarCharVector) v; + long randTime = baseTime + value( ); + String str = fmt.format(new Date(randTime)); + vector.getMutator().setSafe(index, str.getBytes()); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java new file mode 100644 index 00000000000..1416cd28d5d --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java @@ -0,0 +1,42 @@ +/* + * 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.drill.exec.store.mock; + +import java.util.Random; + +import org.apache.drill.exec.vector.Float8Vector; +import org.apache.drill.exec.vector.ValueVector; + +public class DoubleGen implements FieldGen { + + Random rand = new Random( ); + + @Override + public void setup(ColumnDef colDef) { } + + public double value( ) { + return rand.nextDouble() * 1_000_000; + } + + @Override + public void setValue( ValueVector v, int index ) { + Float8Vector vector = (Float8Vector) v; + vector.getMutator().set(index, value()); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java new file mode 100644 index 00000000000..c43082fe005 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java @@ -0,0 +1,149 @@ +/* + * 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.drill.exec.store.mock; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.drill.common.exceptions.ExecutionSetupException; +import org.apache.drill.common.types.TypeProtos.MajorType; +import org.apache.drill.exec.exception.OutOfMemoryException; +import org.apache.drill.exec.exception.SchemaChangeException; +import org.apache.drill.exec.expr.TypeHelper; +import org.apache.drill.exec.ops.FragmentContext; +import org.apache.drill.exec.ops.OperatorContext; +import org.apache.drill.exec.physical.impl.OutputMutator; +import org.apache.drill.exec.record.MaterializedField; +import org.apache.drill.exec.store.AbstractRecordReader; +import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockColumn; +import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry; +import org.apache.drill.exec.vector.AllocationHelper; +import org.apache.drill.exec.vector.ValueVector; + +public class ExtendedMockRecordReader extends AbstractRecordReader { + + private ValueVector[] valueVectors; + private int batchRecordCount; + private int recordsRead; + + private final MockScanEntry config; + private final FragmentContext context; + private final ColumnDef fields[]; + + public ExtendedMockRecordReader(FragmentContext context, MockScanEntry config) { + this.context = context; + this.config = config; + + fields = buildColumnDefs( ); + } + + private ColumnDef[] buildColumnDefs() { + List defs = new ArrayList<>( ); + + // Look for duplicate names. Bad things happen when the sama name + // appears twice. + + Set names = new HashSet<>(); + MockColumn cols[] = config.getTypes(); + for ( int i = 0; i < cols.length; i++ ) { + MockColumn col = cols[i]; + if (names.contains(col.name)) { + throw new IllegalArgumentException("Duplicate column name: " + col.name); + } + names.add(col.name); + int repeat = Math.min( 1, col.getRepeatCount( ) ); + if ( repeat == 1 ) { + defs.add( new ColumnDef(col) ); + } else { + for ( int j = 0; j < repeat; j++ ) { + defs.add( new ColumnDef(col, j+1) ); + } + } + } + ColumnDef[] defArray = new ColumnDef[defs.size()]; + defs.toArray(defArray); + return defArray; + } + + private int getEstimatedRecordSize(MockColumn[] types) { + int size = 0; + for (int i = 0; i < fields.length; i++) { + size += TypeHelper.getSize(fields[i].getConfig().getMajorType()); + } + return size; + } + + private MaterializedField getVector(String name, MajorType type, int length) { + assert context != null : "Context shouldn't be null."; + return MaterializedField.create(name, type); + } + + @Override + public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { + try { + final int estimateRowSize = getEstimatedRecordSize(config.getTypes()); + valueVectors = new ValueVector[config.getTypes().length]; + batchRecordCount = 250000 / estimateRowSize; + + for (int i = 0; i < fields.length; i++) { + final ColumnDef col = fields[i]; + final MajorType type = col.getConfig( ).getMajorType(); + final MaterializedField field = getVector(col.getName(), type, batchRecordCount); + final Class vvClass = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getDataMode()); + valueVectors[i] = output.addField(field, vvClass); + } + } catch (SchemaChangeException e) { + throw new ExecutionSetupException("Failure while setting up fields", e); + } + } + + @Override + public int next() { + if (recordsRead >= this.config.getRecords()) { + return 0; + } + + final int recordSetSize = Math.min(batchRecordCount, this.config.getRecords() - recordsRead); + recordsRead += recordSetSize; + for ( int i = 0; i < recordSetSize; i++ ) { + int j = 0; + for (final ValueVector v : valueVectors) { + fields[j++].generator.setValue(v, i); + } + } + + return recordSetSize; + } + + @Override + public void allocate(Map vectorMap) throws OutOfMemoryException { + try { + for (final ValueVector v : vectorMap.values()) { + AllocationHelper.allocate(v, Character.MAX_VALUE, 50, 10); + } + } catch (NullPointerException e) { + throw new OutOfMemoryException(); + } + } + + @Override + public void close() { } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java new file mode 100644 index 00000000000..a9eab04cb68 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java @@ -0,0 +1,37 @@ +/* + * 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.drill.exec.store.mock; + +import org.apache.drill.exec.vector.ValueVector; + +/** + * Interface which all mock column data generators must + * implement. Each has a {@link #setup} method which is given + * the column definition for the column. This definition may have + * additional configuration information for the column (column width, + * precision, etc.). Each also has a {@link #setValue} method that + * does the work of setting a specific value vector position to the + * generated value. The implementation is expected to cast the + * vector to the type supported by that particular generator. + * (This is test code; we're not overly concerned with the overhead + * of such casts.) + */ +public interface FieldGen { + void setup(ColumnDef colDef); + void setValue( ValueVector v, int index ); +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java new file mode 100644 index 00000000000..f1d5aed5af6 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java @@ -0,0 +1,42 @@ +/* + * 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.drill.exec.store.mock; + +import java.util.Random; + +import org.apache.drill.exec.vector.IntVector; +import org.apache.drill.exec.vector.ValueVector; + +public class IntGen implements FieldGen { + + Random rand = new Random( ); + + @Override + public void setup(ColumnDef colDef) { } + + public int value( ) { + return rand.nextInt(); + } + + @Override + public void setValue( ValueVector v, int index ) { + IntVector vector = (IntVector) v; + vector.getMutator().set(index, value()); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java index bb71c31a794..20dbe552bb6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,17 +17,17 @@ */ package org.apache.drill.exec.store.mock; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos.DataMode; import org.apache.drill.common.types.TypeProtos.MajorType; import org.apache.drill.common.types.TypeProtos.MinorType; -import org.apache.drill.exec.expr.TypeHelper; -import org.apache.drill.exec.physical.EndpointAffinity; import org.apache.drill.exec.physical.base.AbstractGroupScan; import org.apache.drill.exec.physical.base.GroupScan; import org.apache.drill.exec.physical.base.PhysicalOperator; @@ -49,15 +49,20 @@ public class MockGroupScanPOP extends AbstractGroupScan { private final String url; protected final List readEntries; - private LinkedList[] mappings; + private LinkedList[] mappings; + private boolean extended; @JsonCreator - public MockGroupScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List readEntries) { + public MockGroupScanPOP(@JsonProperty("url") String url, + @JsonProperty("extended") Boolean extended, + @JsonProperty("entries") List readEntries) { super((String)null); this.readEntries = readEntries; this.url = url; + this.extended = extended == null ? false : extended; } + @Override public ScanStats getScanStats() { return ScanStats.TRIVIAL_TABLE; } @@ -71,22 +76,15 @@ public List getReadEntries() { return readEntries; } - public static class MockScanEntry{ + public static class MockScanEntry { private final int records; private final MockColumn[] types; - private final int recordSize; - @JsonCreator public MockScanEntry(@JsonProperty("records") int records, @JsonProperty("types") MockColumn[] types) { this.records = records; this.types = types; - int size = 0; - for (MockColumn dt : types) { - size += TypeHelper.getSize(dt.getMajorType()); - } - this.recordSize = size; } public int getRecords() { @@ -104,44 +102,46 @@ public String toString() { } @JsonInclude(Include.NON_NULL) - public static class MockColumn{ + public static class MockColumn { @JsonProperty("type") public MinorType minorType; public String name; public DataMode mode; public Integer width; public Integer precision; public Integer scale; - + public String generator; + public Integer repeat; @JsonCreator - public MockColumn(@JsonProperty("name") String name, @JsonProperty("type") MinorType minorType, @JsonProperty("mode") DataMode mode, @JsonProperty("width") Integer width, @JsonProperty("precision") Integer precision, @JsonProperty("scale") Integer scale) { + public MockColumn(@JsonProperty("name") String name, + @JsonProperty("type") MinorType minorType, + @JsonProperty("mode") DataMode mode, + @JsonProperty("width") Integer width, + @JsonProperty("precision") Integer precision, + @JsonProperty("scale") Integer scale, + @JsonProperty("generator") String generator, + @JsonProperty("repeat") Integer repeat) { this.name = name; this.minorType = minorType; this.mode = mode; this.width = width; this.precision = precision; this.scale = scale; + this.generator = generator; + this.repeat = repeat; } @JsonProperty("type") - public MinorType getMinorType() { - return minorType; - } - public String getName() { - return name; - } - public DataMode getMode() { - return mode; - } - public Integer getWidth() { - return width; - } - public Integer getPrecision() { - return precision; - } - public Integer getScale() { - return scale; - } + public MinorType getMinorType() { return minorType; } + public String getName() { return name; } + public DataMode getMode() { return mode; } + public Integer getWidth() { return width; } + public Integer getPrecision() { return precision; } + public Integer getScale() { return scale; } + public String getGenerator( ) { return generator; } + public Integer getRepeat() { return repeat; } + @JsonIgnore + public int getRepeatCount() { return repeat == null ? 1 : repeat; } @JsonIgnore public MajorType getMajorType() { @@ -164,7 +164,6 @@ public MajorType getMajorType() { public String toString() { return "MockColumn [minorType=" + minorType + ", name=" + name + ", mode=" + mode + "]"; } - } @SuppressWarnings("unchecked") @@ -192,7 +191,7 @@ public void applyAssignments(List endpoints) { @Override public SubScan getSpecificScan(int minorFragmentId) { assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId); - return new MockSubScanPOP(url, mappings[minorFragmentId]); + return new MockSubScanPOP(url, extended, mappings[minorFragmentId]); } @Override @@ -204,13 +203,58 @@ public int getMaxParallelizationWidth() { @JsonIgnore public PhysicalOperator getNewWithChildren(List children) { Preconditions.checkArgument(children.isEmpty()); - return new MockGroupScanPOP(url, readEntries); + return new MockGroupScanPOP(url, extended, readEntries); } @Override public GroupScan clone(List columns) { - return this; + if (columns.isEmpty()) { + throw new IllegalArgumentException("No columns for mock scan"); + } + List mockCols = new ArrayList<>( ); + Pattern p = Pattern.compile( "(\\w+)_([isd])(\\d*)" ); + for (SchemaPath path : columns) { + String col = path.getLastSegment().getNameSegment().getPath(); + if (col.equals("*")) { + return this; + } + Matcher m = p.matcher(col); + if (! m.matches()) { + throw new IllegalArgumentException( "Badly formatted mock column name: " + col ); + } + @SuppressWarnings("unused") + String name = m.group(1); + String type = m.group(2); + String length = m.group(3); + int width = 10; + if ( ! length.isEmpty() ) { + width = Integer.parseInt(length); + } + MinorType minorType; + switch(type) { + case "i": + minorType = MinorType.INT; + break; + case "s": + minorType = MinorType.VARCHAR; + break; + case "d": + minorType = MinorType.FLOAT8; + break; + default: + throw new IllegalArgumentException( "Unsupported field type " + type + " for mock column " + col ); + } + MockColumn mockCol = new MockColumn(col, minorType, DataMode.REQUIRED, width, 0, 0, null, 1); + mockCols.add(mockCol); + } + MockScanEntry entry = readEntries.get(0); + MockColumn types[] = new MockColumn[mockCols.size()]; + mockCols.toArray(types); + MockScanEntry newEntry = new MockScanEntry( entry.records, types ); + List newEntries = new ArrayList<>( ); + newEntries.add(newEntry); + return new MockGroupScanPOP( url, true, newEntries ); } @Override @@ -224,4 +268,9 @@ public String toString() { + ", readEntries=" + readEntries + "]"; } + @Override + @JsonIgnore + public boolean canPushdownProjects(List columns) { + return true; + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java index ed3decb1161..6f8cb3980a8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -20,7 +20,6 @@ import java.util.Map; import org.apache.drill.common.exceptions.ExecutionSetupException; -import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.types.TypeProtos.MajorType; import org.apache.drill.exec.exception.OutOfMemoryException; import org.apache.drill.exec.exception.SchemaChangeException; @@ -43,9 +42,9 @@ public class MockRecordReader extends AbstractRecordReader { private ValueVector[] valueVectors; private int recordsRead; private int batchRecordCount; + @SuppressWarnings("unused") private OperatorContext operatorContext; - public MockRecordReader(FragmentContext context, MockScanEntry config) { this.context = context; this.config = config; @@ -111,6 +110,5 @@ public void allocate(Map vectorMap) throws OutOfMemoryExcep } @Override - public void close() { - } + public void close() { } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java index 6cdbc3cc2cc..9cdb7adfb06 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -40,7 +40,11 @@ public ScanBatch getBatch(FragmentContext context, MockSubScanPOP config, List entries = config.getReadEntries(); final List readers = Lists.newArrayList(); for(final MockScanEntry e : entries) { - readers.add(new MockRecordReader(context, e)); + if ( config.isExtended( ) ) { + readers.add(new ExtendedMockRecordReader(context, e)); + } else { + readers.add(new MockRecordReader(context, e)); + } } return new ScanBatch(config, context, readers.iterator()); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java index d68fd52887b..2ab56049bb2 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -19,44 +19,64 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.calcite.schema.SchemaPlus; - +import org.apache.calcite.schema.Table; import org.apache.drill.common.JSONOptions; import org.apache.drill.common.expression.SchemaPath; import org.apache.drill.common.logical.StoragePluginConfig; import org.apache.drill.exec.physical.base.AbstractGroupScan; +import org.apache.drill.exec.planner.logical.DynamicDrillTable; import org.apache.drill.exec.server.DrillbitContext; +import org.apache.drill.exec.store.AbstractSchema; import org.apache.drill.exec.store.AbstractStoragePlugin; import org.apache.drill.exec.store.SchemaConfig; import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; public class MockStorageEngine extends AbstractStoragePlugin { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class); private final MockStorageEngineConfig configuration; + private final MockSchema schema; public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context, String name) { this.configuration = configuration; + this.schema = new MockSchema(this); } @Override public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List columns) throws IOException { - ArrayList readEntries = selection.getListWith(new ObjectMapper(), + List readEntries = selection.getListWith(new ObjectMapper(), new TypeReference>() { }); - return new MockGroupScanPOP(null, readEntries); + // The classic (logical-plan based) and extended (SQL-based) paths + // come through here. If this is a SQL query, then no columns are + // defined in the plan. + + assert ! readEntries.isEmpty(); + boolean extended = readEntries.size() == 1; + if (extended) { + MockScanEntry entry = readEntries.get(0); + extended = entry.getTypes() == null; + } + return new MockGroupScanPOP(null, extended, readEntries); } @Override public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException { + parent.add(schema.getName(), schema); } @Override @@ -64,5 +84,56 @@ public StoragePluginConfig getConfig() { return configuration; } + @Override + public boolean supportsRead() { + return true; + } + +// public static class ImplicitTable extends DynamicDrillTable { +// +// public ImplicitTable(StoragePlugin plugin, String storageEngineName, +// Object selection) { +// super(plugin, storageEngineName, selection); +// } +// +// } + + private static class MockSchema extends AbstractSchema { + private MockStorageEngine engine; + + public MockSchema(MockStorageEngine engine) { + super(ImmutableList.of(), MockStorageEngineConfig.NAME); + this.engine = engine; + } + + @Override + public Table getTable(String name) { + Pattern p = Pattern.compile( "(\\w+)_(\\d+)(k|m)?", Pattern.CASE_INSENSITIVE); + Matcher m = p.matcher(name); + if ( ! m.matches() ) { + return null; + } + @SuppressWarnings("unused") + String baseName = m.group(1); + int n = Integer.parseInt(m.group(2)); + String unit = m.group(3); + if (unit.equalsIgnoreCase("K")) { n *= 1000; } + else if (unit.equalsIgnoreCase("M")) { n *= 1_000_000; } + MockScanEntry entry = new MockScanEntry(n, null); + List list = new ArrayList<>(); + list.add( entry ); + return new DynamicDrillTable(engine, this.name, list ); + } + + @Override + public Set getTableNames() { + return new HashSet<>( ); + } + + @Override + public String getTypeName() { + return MockStorageEngineConfig.NAME; + } + } } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java index 2f7ea18f38a..afa6b3326ce 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -24,13 +24,12 @@ import com.fasterxml.jackson.annotation.JsonTypeName; @JsonTypeName(MockStorageEngineConfig.NAME) -public class MockStorageEngineConfig extends StoragePluginConfigBase{ - - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngineConfig.class); +public class MockStorageEngineConfig extends StoragePluginConfigBase { private String url; public static final String NAME = "mock"; + public static final MockStorageEngineConfig INSTANCE = new MockStorageEngineConfig( "mock:///" ); @JsonCreator public MockStorageEngineConfig(@JsonProperty("url") String url) { @@ -41,7 +40,6 @@ public String getUrl() { return url; } - @Override public boolean equals(Object o) { if (this == o) { @@ -64,5 +62,4 @@ public boolean equals(Object o) { public int hashCode() { return url != null ? url.hashCode() : 0; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java index 4c12d571235..9fee5c714e4 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -39,6 +39,7 @@ public MockStorePOP(@JsonProperty("child") PhysicalOperator child) { super(child); } + @Override public int getMaxWidth() { return 1; } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java index 705452d1fb9..98e4b1cb8f5 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -18,7 +18,6 @@ package org.apache.drill.exec.store.mock; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import org.apache.drill.exec.physical.base.AbstractBase; @@ -40,12 +39,12 @@ public class MockSubScanPOP extends AbstractBase implements SubScan { private final String url; protected final List readEntries; -// private final OperatorCost cost; -// private final Size size; - private LinkedList[] mappings; + private boolean extended; @JsonCreator - public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List readEntries) { + public MockSubScanPOP(@JsonProperty("url") String url, + @JsonProperty("extended") Boolean extended, + @JsonProperty("entries") List readEntries) { this.readEntries = readEntries; // OperatorCost cost = new OperatorCost(0,0,0,0); // Size size = new Size(0,0); @@ -56,11 +55,11 @@ public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") // this.cost = cost; // this.size = size; this.url = url; + this.extended = extended == null ? false : extended; } - public String getUrl() { - return url; - } + public String getUrl() { return url; } + public boolean isExtended( ) { return extended; } @JsonProperty("entries") public List getReadEntries() { @@ -88,7 +87,7 @@ public T accept(PhysicalVisitor physicalVis @JsonIgnore public PhysicalOperator getNewWithChildren(List children) { Preconditions.checkArgument(children.isEmpty()); - return new MockSubScanPOP(url, readEntries); + return new MockSubScanPOP(url, extended, readEntries); } @@ -96,5 +95,4 @@ public PhysicalOperator getNewWithChildren(List children) { public int getOperatorType() { return CoreOperatorType.MOCK_SUB_SCAN_VALUE; } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java new file mode 100644 index 00000000000..9281100efd1 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java @@ -0,0 +1,42 @@ +/* + * 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.drill.exec.store.mock; + +import java.util.Random; + +import org.apache.drill.exec.vector.Float8Vector; +import org.apache.drill.exec.vector.ValueVector; + +public class MoneyGen implements FieldGen { + + Random rand = new Random( ); + + @Override + public void setup(ColumnDef colDef) { } + + public double value( ) { + return Math.ceil( rand.nextDouble() * 1_000_000 * 100) / 100; + } + + @Override + public void setValue( ValueVector v, int index ) { + Float8Vector vector = (Float8Vector) v; + vector.getMutator().set(index, value()); + } + +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java new file mode 100644 index 00000000000..8afbeb77374 --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.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.drill.exec.store.mock; + +import java.util.Random; + +import org.apache.drill.exec.vector.ValueVector; +import org.apache.drill.exec.vector.VarCharVector; + +public class StringGen implements FieldGen { + + Random rand = new Random( ); + int length; + + @Override + public void setup( ColumnDef colDef ) { + length = colDef.width; + } + + public String value( ) { + String c = Character.toString( (char) (rand.nextInt(26) + 'A') ); + StringBuilder buf = new StringBuilder( ); + for ( int i = 0; i < length; i++ ) { + buf.append( c ); + } + return buf.toString(); + } + + @Override + public void setValue( ValueVector v, int index ) { + VarCharVector vector = (VarCharVector) v; + vector.getMutator().setSafe(index, value().getBytes()); + } +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java new file mode 100644 index 00000000000..e99cfc5023a --- /dev/null +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java @@ -0,0 +1,130 @@ +/* + * 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. + */ +/** + * Defines a mock data source which generates dummy test data for use + * in testing. The data source operates in two modes: + *
    + *
  • Classic: used in physical plans in many unit tests. + * The plan specifies a set of columns; data is generated by the + * vectors themselves based on two alternating values.
  • + *
  • Enhanced: available for use in newer unit tests. + * Enhances the physical plan description to allow specifying a data + * generator class (for various types, data formats, etc.) Also + * provides a data storage engine framework to allow using mock + * tables in SQL queries.
  • + *
+ *

Classic Mode

+ * Create a scan operator that looks like the following (from + * ): + *

+ *    graph:[
+ *      {
+ *        {@literal @}id:1,
+ *        pop:"mock-scan",
+ *        url: "http://apache.org",
+ *        entries:[
+ *          {records: 1000000, types: [
+ *             {name: "blue", type: "INT", mode: "REQUIRED"},
+ *             {name: "green", type: "INT", mode: "REQUIRED"}
+ *        ]}
+ *      ]
+ *    }, ...
+ * 
+ * Here: + *
    + *
  • The pop must be mock-scan.
  • + *
  • The url is unused.
  • + *
  • The entries section can have one or more entries. If + * more than one entry, the storage engine will enable parallel scans + * up to the number of entries, as though each entry was a different + * file or group.
  • + *
  • The entry name is arbitrary, though color names seem + * to be the traditional names used in Drill tests.
  • + *
  • The type is one of the supported Drill + * {@link MinorType} names.
  • + *
  • The mode is one of the supported Drill + * {@link DataMode} names: usually OPTIONAL or REQUIRED.
  • + *
+ * + *

Enhanced Mode

+ * Enhanced builds on the Classic mode to add additional capabilities. + * Enhanced mode can be used either in a physical plan or in SQL. Data + * is randomly generated over a wide range of values and can be + * controlled by custom generator classes. When + * in a physical plan, the records section has additional + * attributes as described in {@link MockGroupScanPOP.MockColumn}: + *
    + *
  • The generator lets you specify a class to generate the + * sample data. Rules for the class name is that it can either contain + * a full package path, or just a class name. If just a class name, the + * class is assumed to reside in this package. For example, to generate + * an ISO date into a string, use DateGen. Additional generators + * can (and should) be added as the need arises.
  • + *
  • The repeat attribute lets you create a very wide row by + * repeating a column the specified number of times. Actual column names + * have a numeric suffix. For example, if the base name is "blue" and + * is repeated twice, actual columns are "blue1" and "blue2".
  • + *
+ * When used in SQL, use the mock name space as follows: + *

+ * SELECT id_i, name_s50 FROM `mock`.`employee_500`;
+ * 
+ * Both the column names and table names encode information that specifies + * what data to generate. + *

+ * Columns are of the form name_typelength?. + *

    + *
  • The name is anything you want ("id" and "name" in the example.)
  • + *
  • The underscore is required to separate the type from the name.
  • + *
  • The type is one of "i" (integer), "d" (double) or "s" (string). + * Other types can be added as needed: n (decimal number), l (long), etc.
  • + *
  • The length is optional and is used only for string (VARCHAR) + * columns. The default string length is 10.
  • + *
  • Columns do not yet support nulls. When they do, the encoding will + * be "_npercent" where the percent specifies the percent of rows + * that should contain null values in this column. + *
  • The column is known to SQL as its full name, that is "id_i" or + * "name_s50".
  • + *
+ *

+ * Tables are of the form name_rowsunit? where: + *

    + *
  • The name is anything you want. ("employee" in the example.)
  • + *
  • The underscore is required to separate the row count from the name.
  • + *
  • The row count specifies the number of rows to return.
  • + *
  • The count unit can be none, K (multiply count by 1000) or M + * (multiply row count by one million), case insensitive.
  • + *
  • Another field (not yet implemented) might specify the split count.
  • + *
+ *

Data Generators

+ * The classic mode uses data generators built into each vector to generate + * the sample data. These generators use a very simple black/white alternating + * series of two values. Simple, but limited. The enhanced mode allows custom + * data generators. Unfortunately, this requires a separate generator class for + * each data type. As a result, we presently support just a few key data types. + * On the other hand, the custom generators do allow tests to specify a custom + * generator class to generate the kind of data needed for that test. + *

+ * All data generators implement the {@link FieldGen} interface, and must have + * a non-argument constructor to allow dynamic instantiation. The mock data + * source either picks a default generator (if no generator is provided) + * or uses the custom generator specified in generator. Generators + * are independent (though one could, perhaps, write generators that correlate + * field values.) + */ +package org.apache.drill.exec.store.mock; \ No newline at end of file diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java index f4fe2da5ead..40f4f45463b 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java @@ -47,7 +47,7 @@ public class TestOpSerialization { public void testSerializedDeserialize() throws Throwable { DrillConfig c = DrillConfig.create(); PhysicalPlanReader reader = PhysicalPlanReaderTestFactory.defaultPhysicalPlanReader(c); - MockSubScanPOP s = new MockSubScanPOP("abc", null); + MockSubScanPOP s = new MockSubScanPOP("abc", false, null); s.setOperatorId(3); Filter f = new Filter(s, new ValueExpressions.BooleanExpression("true", ExpressionPosition.UNKNOWN), 0.1f); f.setOperatorId(2); diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java index 722d45ed2d0..900fbf35743 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java @@ -165,11 +165,11 @@ protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorT final MockGroupScanPOP.MockColumn[] columns = new MockGroupScanPOP.MockColumn[colNames.length]; for (int i = 0; i < colNames.length; i++ ) { - columns[i] = new MockGroupScanPOP.MockColumn(colNames[i], colTypes[i].getMinorType(), colTypes[i].getMode(),0,0,0); + columns[i] = new MockGroupScanPOP.MockColumn(colNames[i], colTypes[i].getMinorType(), colTypes[i].getMode(), 0, 0, 0, null, null); } final MockGroupScanPOP.MockScanEntry entry = new MockGroupScanPOP.MockScanEntry(10, columns); - final MockSubScanPOP scanPOP = new MockSubScanPOP("testTable", java.util.Collections.singletonList(entry)); + final MockSubScanPOP scanPOP = new MockSubScanPOP("testTable", false, java.util.Collections.singletonList(entry)); final ScanBatch batch = createMockScanBatch(bit1, scanPOP, planFragment); From 5423ccf7cc51fb24983fa6999ae777fdd18de50b Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Sat, 7 Jan 2017 12:54:45 -0800 Subject: [PATCH 2/3] Revisions suggested by code review Also includes additional comments and a few more compiler warning cleanups. --- .../exec/record/AbstractRecordBatch.java | 2 +- .../drill/exec/store/StoragePlugin.java | 2 +- .../exec/store/StoragePluginRegistry.java | 2 +- .../exec/store/StoragePluginRegistryImpl.java | 9 +- .../drill/exec/store/mock/ColumnDef.java | 4 +- .../apache/drill/exec/store/mock/DateGen.java | 24 ++- .../drill/exec/store/mock/DoubleGen.java | 12 +- .../store/mock/ExtendedMockRecordReader.java | 23 ++- .../apache/drill/exec/store/mock/IntGen.java | 11 +- .../exec/store/mock/MockGroupScanPOP.java | 174 ++++++++++++++---- .../drill/exec/store/mock/MockSubScanPOP.java | 26 ++- .../drill/exec/store/mock/MoneyGen.java | 12 +- .../drill/exec/store/mock/StringGen.java | 13 +- .../drill/exec/TestOpSerialization.java | 2 +- .../fn/interp/ExpressionInterpreterTest.java | 14 +- 15 files changed, 249 insertions(+), 81 deletions(-) diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java index 998665c162e..d82c1547a20 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/AbstractRecordBatch.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java index 8324e3062ba..2969d4f437e 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java index da29978e889..82f18f8d50b 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java index 273e752a129..7a174904df8 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -51,8 +51,6 @@ import org.apache.drill.exec.store.dfs.FormatPlugin; import org.apache.drill.exec.store.ischema.InfoSchemaConfig; import org.apache.drill.exec.store.ischema.InfoSchemaStoragePlugin; -import org.apache.drill.exec.store.mock.MockStorageEngine; -import org.apache.drill.exec.store.mock.MockStorageEngineConfig; import org.apache.drill.exec.store.sys.PersistentStore; import org.apache.drill.exec.store.sys.PersistentStoreConfig; import org.apache.drill.exec.store.sys.SystemTablePlugin; @@ -130,6 +128,7 @@ public void init() throws DrillbitStartupException { plugins.putAll(createPlugins()); } + @SuppressWarnings({ "resource" }) private Map createPlugins() throws DrillbitStartupException { try { /* @@ -212,6 +211,7 @@ public void addPlugin(String name, StoragePlugin plugin) { @Override public void deletePlugin(String name) { + @SuppressWarnings("resource") StoragePlugin plugin = plugins.remove(name); closePlugin(plugin); pluginSystemTable.delete(name); @@ -229,6 +229,7 @@ private void closePlugin(StoragePlugin plugin) { } } + @SuppressWarnings("resource") @Override public StoragePlugin createOrUpdate(String name, StoragePluginConfig config, boolean persist) throws ExecutionSetupException { @@ -319,6 +320,7 @@ public StoragePlugin getPlugin(StoragePluginConfig config) throws ExecutionSetup } } + @SuppressWarnings("resource") @Override public FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig) throws ExecutionSetupException { @@ -366,6 +368,7 @@ public SchemaFactory getSchemaFactory() { public class DrillSchemaFactory implements SchemaFactory { + @SuppressWarnings({ "resource" }) @Override public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException { Stopwatch watch = Stopwatch.createStarted(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java index 6fc94c445fc..d41888fc756 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java @@ -165,7 +165,7 @@ private void makeDefaultGenerator( ) { public ColumnDef( MockColumn mockCol, int rep ) { this( mockCol ); - name = name += Integer.toString(rep); + name += Integer.toString(rep); } public MockColumn getConfig() { @@ -175,4 +175,4 @@ public MockColumn getConfig() { public String getName() { return name; } -} \ No newline at end of file +} diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java index 759fa3f8c3b..3d97cc015d3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java @@ -24,23 +24,39 @@ import org.apache.drill.exec.vector.ValueVector; import org.apache.drill.exec.vector.VarCharVector; +/** + * Very simple date vaue generator that produces ISO dates + * uniformly distributed over the last year. ISO format + * is: 2016-12-07. + *

+ * There are many possible date formats; this class does not + * attempt to generate all of them. Drill provides a date + * type, but we use a string format because example cases from + * people using the product often read text files. Finally, we + * (reluctantly) use the old-style date formats instead of the + * new Java 8 classes because Drill prefers to build with Java 7. + */ + public class DateGen implements FieldGen { - private Random rand = new Random( ); + private final int ONE_DAY = 24 * 60 * 60 * 1000; + private final int ONE_YEAR = ONE_DAY * 365; + + private final Random rand = new Random( ); private long baseTime; private SimpleDateFormat fmt; public DateGen( ) { // Start a year ago. - baseTime = System.currentTimeMillis() - 365 * 24 * 60 * 60 * 1000; + baseTime = System.currentTimeMillis() - ONE_YEAR; fmt = new SimpleDateFormat( "yyyy-mm-DD" ); } @Override public void setup(ColumnDef colDef) { } - public long value( ) { - return baseTime + rand.nextInt( 365 ) * 24 * 60 * 60 * 1000; + private long value( ) { + return baseTime + rand.nextInt( 365 ) * ONE_DAY; } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java index 1416cd28d5d..ab909c70408 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java @@ -22,15 +22,21 @@ import org.apache.drill.exec.vector.Float8Vector; import org.apache.drill.exec.vector.ValueVector; +/** + * Generates random field values uniformly distributed over + * the range +-1 million, with any number of digits past + * the decimal point. + */ + public class DoubleGen implements FieldGen { - Random rand = new Random( ); + private final Random rand = new Random( ); @Override public void setup(ColumnDef colDef) { } - public double value( ) { - return rand.nextDouble() * 1_000_000; + private double value( ) { + return rand.nextDouble() * 2_000_000 - 1_000_000; } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java index c43082fe005..8563954020c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java @@ -38,6 +38,16 @@ import org.apache.drill.exec.vector.AllocationHelper; import org.apache.drill.exec.vector.ValueVector; +/** + * Extended form of the mock record reader that uses generator class + * instances to create the mock values. This is a work in progress. + * Generators exist for a few simple required types. One also exists + * to generate strings that contain dates. + *

+ * The definition is provided inside the sub scan used to create the + * {@link ScanBatch} used to create this record reader. + */ + public class ExtendedMockRecordReader extends AbstractRecordReader { private ValueVector[] valueVectors; @@ -58,8 +68,10 @@ public ExtendedMockRecordReader(FragmentContext context, MockScanEntry config) { private ColumnDef[] buildColumnDefs() { List defs = new ArrayList<>( ); - // Look for duplicate names. Bad things happen when the sama name - // appears twice. + // Look for duplicate names. Bad things happen when the same name + // appears twice. We must do this here because some tests create + // a physical plan directly, meaning that this is the first + // opportunity to review the column definitions. Set names = new HashSet<>(); MockColumn cols[] = config.getTypes(); @@ -91,11 +103,6 @@ private int getEstimatedRecordSize(MockColumn[] types) { return size; } - private MaterializedField getVector(String name, MajorType type, int length) { - assert context != null : "Context shouldn't be null."; - return MaterializedField.create(name, type); - } - @Override public void setup(OperatorContext context, OutputMutator output) throws ExecutionSetupException { try { @@ -106,7 +113,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio for (int i = 0; i < fields.length; i++) { final ColumnDef col = fields[i]; final MajorType type = col.getConfig( ).getMajorType(); - final MaterializedField field = getVector(col.getName(), type, batchRecordCount); + final MaterializedField field = MaterializedField.create(col.getName(), type); final Class vvClass = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getDataMode()); valueVectors[i] = output.addField(field, vvClass); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java index f1d5aed5af6..ae5e32432d6 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java @@ -22,14 +22,20 @@ import org.apache.drill.exec.vector.IntVector; import org.apache.drill.exec.vector.ValueVector; +/** + * Generates integer values uniformly randomly distributed over + * the entire 32-bit integer range from + * {@link Integer.MIN_VALUE} to {@link Integer.MAX_VALUE}. + */ + public class IntGen implements FieldGen { - Random rand = new Random( ); + private final Random rand = new Random( ); @Override public void setup(ColumnDef colDef) { } - public int value( ) { + private int value( ) { return rand.nextInt(); } @@ -38,5 +44,4 @@ public void setValue( ValueVector v, int index ) { IntVector vector = (IntVector) v; vector.getMutator().set(index, value()); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java index 20dbe552bb6..2e8af425762 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java @@ -43,20 +43,44 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +/** + * Describes a "group" scan of a (logical) mock table. The mock table has a + * schema described by the {@link MockScanEntry}. Class. To simulate a scan that + * can be parallelized, this group scan can contain a list of + * {@link MockScanEntry}, each of which simulates a separate file on disk, or + * block within a file. Each will give rise to a separate minor fragment + * (assuming sufficient parallelization.) + */ + @JsonTypeName("mock-scan") public class MockGroupScanPOP extends AbstractGroupScan { - static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class); + static final org.slf4j.Logger logger = org.slf4j.LoggerFactory + .getLogger(MockGroupScanPOP.class); + /** + * URL for the scan. Unused. Appears to be a vestige of an earlier design that + * required them. + */ private final String url; + + /** + * The set of simulated files to scan. + */ protected final List readEntries; private LinkedList[] mappings; + + /** + * Whether this group scan uses a newer "extended" schema definition, or the + * original (non-extended) definition. + */ + private boolean extended; @JsonCreator public MockGroupScanPOP(@JsonProperty("url") String url, - @JsonProperty("extended") Boolean extended, - @JsonProperty("entries") List readEntries) { - super((String)null); + @JsonProperty("extended") Boolean extended, + @JsonProperty("entries") List readEntries) { + super((String) null); this.readEntries = readEntries; this.url = url; this.extended = extended == null ? false : extended; @@ -76,13 +100,20 @@ public List getReadEntries() { return readEntries; } + /** + * Describes one simulated file (or block) within the logical file scan + * described by this group scan. Each block can have a distinct schema to test + * for schema changes. + */ + public static class MockScanEntry { private final int records; private final MockColumn[] types; @JsonCreator - public MockScanEntry(@JsonProperty("records") int records, @JsonProperty("types") MockColumn[] types) { + public MockScanEntry(@JsonProperty("records") int records, + @JsonProperty("types") MockColumn[] types) { this.records = records; this.types = types; } @@ -97,30 +128,62 @@ public MockColumn[] getTypes() { @Override public String toString() { - return "MockScanEntry [records=" + records + ", columns=" + Arrays.toString(types) + "]"; + return "MockScanEntry [records=" + records + ", columns=" + + Arrays.toString(types) + "]"; } } + /** + * Meta-data description of the columns we wish to create during a simulated + * scan. + */ + @JsonInclude(Include.NON_NULL) public static class MockColumn { - @JsonProperty("type") public MinorType minorType; + + /** + * Column type given as a Drill minor type (that is, a type without the + * extra information such as cardinality, width, etc. + */ + + @JsonProperty("type") + public MinorType minorType; public String name; public DataMode mode; public Integer width; public Integer precision; public Integer scale; + + /** + * The scan can request to use a specific data generator class. The name of + * that class appears here. The name can be a simple class name, if that + * class resides in this Java package. Or, it can be a fully qualified name + * of a class that resides elsewhere. If null, the default generator for the + * data type is used. + */ + public String generator; + + /** + * Some tests want to create a very wide row with many columns. This field + * eases that task: specify a value other than 1 and the data source will + * generate that many copies of the column, each with separately generated + * random values. For example, to create 20 copies of field, "foo", set + * repeat to 20 and the actual generated batches will contain fields + * foo1, foo2, ... foo20. + */ + public Integer repeat; @JsonCreator public MockColumn(@JsonProperty("name") String name, - @JsonProperty("type") MinorType minorType, - @JsonProperty("mode") DataMode mode, - @JsonProperty("width") Integer width, - @JsonProperty("precision") Integer precision, - @JsonProperty("scale") Integer scale, - @JsonProperty("generator") String generator, - @JsonProperty("repeat") Integer repeat) { + @JsonProperty("type") MinorType minorType, + @JsonProperty("mode") DataMode mode, + @JsonProperty("width") Integer width, + @JsonProperty("precision") Integer precision, + @JsonProperty("scale") Integer scale, + @JsonProperty("generator") String generator, + @JsonProperty("repeat") Integer repeat) { this.name = name; this.minorType = minorType; this.mode = mode; @@ -132,16 +195,42 @@ public MockColumn(@JsonProperty("name") String name, } @JsonProperty("type") - public MinorType getMinorType() { return minorType; } - public String getName() { return name; } - public DataMode getMode() { return mode; } - public Integer getWidth() { return width; } - public Integer getPrecision() { return precision; } - public Integer getScale() { return scale; } - public String getGenerator( ) { return generator; } - public Integer getRepeat() { return repeat; } + public MinorType getMinorType() { + return minorType; + } + + public String getName() { + return name; + } + + public DataMode getMode() { + return mode; + } + + public Integer getWidth() { + return width; + } + + public Integer getPrecision() { + return precision; + } + + public Integer getScale() { + return scale; + } + + public String getGenerator() { + return generator; + } + + public Integer getRepeat() { + return repeat; + } + @JsonIgnore - public int getRepeatCount() { return repeat == null ? 1 : repeat; } + public int getRepeatCount() { + return repeat == null ? 1 : repeat; + } @JsonIgnore public MajorType getMajorType() { @@ -162,7 +251,8 @@ public MajorType getMajorType() { @Override public String toString() { - return "MockColumn [minorType=" + minorType + ", name=" + name + ", mode=" + mode + "]"; + return "MockColumn [minorType=" + minorType + ", name=" + name + ", mode=" + + mode + "]"; } } @@ -173,7 +263,7 @@ public void applyAssignments(List endpoints) { mappings = new LinkedList[endpoints.size()]; - int i =0; + int i = 0; for (MockScanEntry e : this.getReadEntries()) { if (i == endpoints.size()) { i -= endpoints.size(); @@ -190,7 +280,9 @@ public void applyAssignments(List endpoints) { @Override public SubScan getSpecificScan(int minorFragmentId) { - assert minorFragmentId < mappings.length : String.format("Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", mappings.length, minorFragmentId); + assert minorFragmentId < mappings.length : String.format( + "Mappings length [%d] should be longer than minor fragment id [%d] but it isn't.", + mappings.length, minorFragmentId); return new MockSubScanPOP(url, extended, mappings[minorFragmentId]); } @@ -204,7 +296,6 @@ public int getMaxParallelizationWidth() { public PhysicalOperator getNewWithChildren(List children) { Preconditions.checkArgument(children.isEmpty()); return new MockGroupScanPOP(url, extended, readEntries); - } @Override @@ -212,27 +303,28 @@ public GroupScan clone(List columns) { if (columns.isEmpty()) { throw new IllegalArgumentException("No columns for mock scan"); } - List mockCols = new ArrayList<>( ); - Pattern p = Pattern.compile( "(\\w+)_([isd])(\\d*)" ); + List mockCols = new ArrayList<>(); + Pattern p = Pattern.compile("(\\w+)_([isd])(\\d*)"); for (SchemaPath path : columns) { String col = path.getLastSegment().getNameSegment().getPath(); if (col.equals("*")) { return this; } Matcher m = p.matcher(col); - if (! m.matches()) { - throw new IllegalArgumentException( "Badly formatted mock column name: " + col ); + if (!m.matches()) { + throw new IllegalArgumentException( + "Badly formatted mock column name: " + col); } @SuppressWarnings("unused") String name = m.group(1); String type = m.group(2); String length = m.group(3); int width = 10; - if ( ! length.isEmpty() ) { + if (!length.isEmpty()) { width = Integer.parseInt(length); } MinorType minorType; - switch(type) { + switch (type) { case "i": minorType = MinorType.INT; break; @@ -243,18 +335,20 @@ public GroupScan clone(List columns) { minorType = MinorType.FLOAT8; break; default: - throw new IllegalArgumentException( "Unsupported field type " + type + " for mock column " + col ); + throw new IllegalArgumentException( + "Unsupported field type " + type + " for mock column " + col); } - MockColumn mockCol = new MockColumn(col, minorType, DataMode.REQUIRED, width, 0, 0, null, 1); + MockColumn mockCol = new MockColumn(col, minorType, DataMode.REQUIRED, + width, 0, 0, null, 1); mockCols.add(mockCol); } MockScanEntry entry = readEntries.get(0); MockColumn types[] = new MockColumn[mockCols.size()]; mockCols.toArray(types); - MockScanEntry newEntry = new MockScanEntry( entry.records, types ); - List newEntries = new ArrayList<>( ); + MockScanEntry newEntry = new MockScanEntry(entry.records, types); + List newEntries = new ArrayList<>(); newEntries.add(newEntry); - return new MockGroupScanPOP( url, true, newEntries ); + return new MockGroupScanPOP(url, true, newEntries); } @Override @@ -264,8 +358,8 @@ public String getDigest() { @Override public String toString() { - return "MockGroupScanPOP [url=" + url - + ", readEntries=" + readEntries + "]"; + return "MockGroupScanPOP [url=" + url + ", readEntries=" + readEntries + + "]"; } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java index 98e4b1cb8f5..843384eba66 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java @@ -33,13 +33,37 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; +/** + * Describes a physical scan operation for the mock data source. Each operator + * can, in general, give rise to one or more actual scans. For the mock data + * source, each sub-scan does exactly one (simulated) scan. + */ + @JsonTypeName("mock-sub-scan") public class MockSubScanPOP extends AbstractBase implements SubScan { static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockGroupScanPOP.class); private final String url; protected final List readEntries; - private boolean extended; + private final boolean extended; + + /** + * This constructor is called from Jackson and is designed to support both + * older physical plans and the newer ("extended") plans. Jackson will fill + * in a null value for the extended field for older plans; we use + * that null value to know that the plan is old, thus not extended. Newer + * plans simply provide the value. + * + * @param url + * not used for the mock plan, appears to be a vestige of creating + * this from a file-based plugin. Must keep it because older physical + * plans contained a dummy URL value. + * @param extended + * see above + * @param readEntries + * a description of the columns to generate in a Jackson-serialized + * form unique to the mock data source plugin. + */ @JsonCreator public MockSubScanPOP(@JsonProperty("url") String url, diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java index 9281100efd1..8697b3c3738 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java @@ -22,14 +22,21 @@ import org.apache.drill.exec.vector.Float8Vector; import org.apache.drill.exec.vector.ValueVector; +/** + * Generates a mock money field as a double over the range 0 + * to 1 million. Values include cents. That is the value + * ranges uniformly over the range 0.00 to + * 999,999.99. + */ + public class MoneyGen implements FieldGen { - Random rand = new Random( ); + private final Random rand = new Random( ); @Override public void setup(ColumnDef colDef) { } - public double value( ) { + private double value( ) { return Math.ceil( rand.nextDouble() * 1_000_000 * 100) / 100; } @@ -38,5 +45,4 @@ public void setValue( ValueVector v, int index ) { Float8Vector vector = (Float8Vector) v; vector.getMutator().set(index, value()); } - } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java index 8afbeb77374..71543ea6b6c 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java @@ -22,17 +22,24 @@ import org.apache.drill.exec.vector.ValueVector; import org.apache.drill.exec.vector.VarCharVector; +/** + * Generates a mock string field of the given length. Fields are composed + * of upper case letters uniformly distributed from A to Z, and repeated + * or the length of the field. Exampled for a 4-character field: + * DDDD, MMMM, AAAA, RRRR, ... + */ + public class StringGen implements FieldGen { - Random rand = new Random( ); - int length; + private final Random rand = new Random( ); + private int length; @Override public void setup( ColumnDef colDef ) { length = colDef.width; } - public String value( ) { + private String value( ) { String c = Character.toString( (char) (rand.nextInt(26) + 'A') ); StringBuilder buf = new StringBuilder( ); for ( int i = 0; i < length; i++ ) { diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java index 40f4f45463b..72371839e00 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java @@ -1,4 +1,4 @@ -/** +/* * 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 diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java index 900fbf35743..e191d352c6e 100644 --- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java +++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java @@ -1,4 +1,4 @@ -/******************************************************************************* +/* * 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 @@ -14,7 +14,7 @@ * 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.drill.exec.fn.interp; import static org.junit.Assert.assertEquals; @@ -22,15 +22,10 @@ import java.nio.ByteBuffer; import java.util.List; -import org.antlr.runtime.ANTLRStringStream; -import org.antlr.runtime.CommonTokenStream; -import org.antlr.runtime.RecognitionException; import org.apache.drill.common.exceptions.DrillRuntimeException; import org.apache.drill.common.expression.ErrorCollector; import org.apache.drill.common.expression.ErrorCollectorImpl; import org.apache.drill.common.expression.LogicalExpression; -import org.apache.drill.common.expression.parser.ExprLexer; -import org.apache.drill.common.expression.parser.ExprParser; import org.apache.drill.common.types.TypeProtos; import org.apache.drill.common.types.Types; import org.apache.drill.common.util.DrillStringUtils; @@ -154,7 +149,9 @@ protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorT } protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorType[] colTypes, String[] expectFirstTwoValues, BitControl.PlanFragment planFragment) throws Exception { + @SuppressWarnings("resource") final RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); + @SuppressWarnings("resource") final Drillbit bit1 = new Drillbit(CONFIG, serviceSet); bit1.run(); @@ -171,10 +168,12 @@ protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorT final MockGroupScanPOP.MockScanEntry entry = new MockGroupScanPOP.MockScanEntry(10, columns); final MockSubScanPOP scanPOP = new MockSubScanPOP("testTable", false, java.util.Collections.singletonList(entry)); + @SuppressWarnings("resource") final ScanBatch batch = createMockScanBatch(bit1, scanPOP, planFragment); batch.next(); + @SuppressWarnings("resource") final ValueVector vv = evalExprWithInterpreter(expressionStr, batch, bit1); // Verify the first 2 values in the output of evaluation. @@ -190,6 +189,7 @@ protected void doTest(String expressionStr, String[] colNames, TypeProtos.MajorT bit1.close(); } + @SuppressWarnings("resource") private ScanBatch createMockScanBatch(Drillbit bit, MockSubScanPOP scanPOP, BitControl.PlanFragment planFragment) { final List children = Lists.newArrayList(); final MockScanBatchCreator creator = new MockScanBatchCreator(); From 221ad6da4c4f859f068a4d499ac219166e24aa73 Mon Sep 17 00:00:00 2001 From: Paul Rogers Date: Mon, 9 Jan 2017 20:43:08 -0800 Subject: [PATCH 3/3] Additional formatting cleanup Remove spaces between parens, other clean ups from second round of code reviews. --- .../exec/store/StoragePluginRegistryImpl.java | 4 +- .../drill/exec/store/mock/ColumnDef.java | 40 +++++++++---------- .../apache/drill/exec/store/mock/DateGen.java | 14 +++---- .../drill/exec/store/mock/DoubleGen.java | 6 +-- .../store/mock/ExtendedMockRecordReader.java | 20 +++++----- .../drill/exec/store/mock/FieldGen.java | 2 +- .../apache/drill/exec/store/mock/IntGen.java | 6 +-- .../exec/store/mock/MockStorageEngine.java | 10 ++--- .../store/mock/MockStorageEngineConfig.java | 2 +- .../drill/exec/store/mock/MockSubScanPOP.java | 2 +- .../drill/exec/store/mock/MoneyGen.java | 8 ++-- .../drill/exec/store/mock/StringGen.java | 16 ++++---- 12 files changed, 65 insertions(+), 65 deletions(-) diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java index 7a174904df8..3fb1c3aff8a 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java @@ -128,7 +128,7 @@ public void init() throws DrillbitStartupException { plugins.putAll(createPlugins()); } - @SuppressWarnings({ "resource" }) + @SuppressWarnings("resource") private Map createPlugins() throws DrillbitStartupException { try { /* @@ -368,7 +368,7 @@ public SchemaFactory getSchemaFactory() { public class DrillSchemaFactory implements SchemaFactory { - @SuppressWarnings({ "resource" }) + @SuppressWarnings("resource") @Override public void registerSchemas(SchemaConfig schemaConfig, SchemaPlus parent) throws IOException { Stopwatch watch = Stopwatch.createStarted(); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java index d41888fc756..cfaacdda6b1 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java @@ -34,11 +34,11 @@ public class ColumnDef { public int width; public FieldGen generator; - public ColumnDef( MockColumn mockCol ) { + public ColumnDef(MockColumn mockCol) { this.mockCol = mockCol; name = mockCol.getName(); width = TypeHelper.getSize(mockCol.getMajorType()); - makeGenerator( ); + makeGenerator(); } /** @@ -50,31 +50,31 @@ public ColumnDef( MockColumn mockCol ) { * class in this package. */ - private void makeGenerator( ) { - String genName = mockCol.getGenerator( ); - if ( genName != null ) { - if ( ! genName.contains(".") ) { + private void makeGenerator() { + String genName = mockCol.getGenerator(); + if (genName != null) { + if (! genName.contains(".")) { genName = "org.apache.drill.exec.store.mock." + genName; } try { - ClassLoader cl = getClass( ).getClassLoader(); + ClassLoader cl = getClass().getClassLoader(); Class genClass = cl.loadClass(genName); - generator = (FieldGen) genClass.newInstance( ); + generator = (FieldGen) genClass.newInstance(); } catch (ClassNotFoundException | InstantiationException | IllegalAccessException | ClassCastException e) { - throw new IllegalArgumentException( "Generator " + genName + " is undefined for mock field " + name ); + throw new IllegalArgumentException("Generator " + genName + " is undefined for mock field " + name); } - generator.setup( this ); + generator.setup(this); return; } - makeDefaultGenerator( ); + makeDefaultGenerator(); } - private void makeDefaultGenerator( ) { + private void makeDefaultGenerator() { MinorType minorType = mockCol.getMinorType(); - switch ( minorType ) { + switch (minorType) { case BIGINT: break; case BIT: @@ -102,12 +102,12 @@ private void makeDefaultGenerator( ) { case FLOAT4: break; case FLOAT8: - generator = new DoubleGen( ); + generator = new DoubleGen(); break; case GENERIC_OBJECT: break; case INT: - generator = new IntGen( ); + generator = new IntGen(); break; case INTERVAL: break; @@ -152,19 +152,19 @@ private void makeDefaultGenerator( ) { case VARBINARY: break; case VARCHAR: - generator = new StringGen( ); + generator = new StringGen(); break; default: break; } - if ( generator == null ) { - throw new IllegalArgumentException( "No default column generator for column " + name + " of type " + minorType ); + if (generator == null) { + throw new IllegalArgumentException("No default column generator for column " + name + " of type " + minorType); } generator.setup(this); } - public ColumnDef( MockColumn mockCol, int rep ) { - this( mockCol ); + public ColumnDef(MockColumn mockCol, int rep) { + this(mockCol); name += Integer.toString(rep); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java index 3d97cc015d3..f7d53ed919d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java @@ -42,27 +42,27 @@ public class DateGen implements FieldGen { private final int ONE_DAY = 24 * 60 * 60 * 1000; private final int ONE_YEAR = ONE_DAY * 365; - private final Random rand = new Random( ); + private final Random rand = new Random(); private long baseTime; private SimpleDateFormat fmt; - public DateGen( ) { + public DateGen() { // Start a year ago. baseTime = System.currentTimeMillis() - ONE_YEAR; - fmt = new SimpleDateFormat( "yyyy-mm-DD" ); + fmt = new SimpleDateFormat("yyyy-mm-DD"); } @Override public void setup(ColumnDef colDef) { } - private long value( ) { - return baseTime + rand.nextInt( 365 ) * ONE_DAY; + private long value() { + return baseTime + rand.nextInt(365) * ONE_DAY; } @Override - public void setValue( ValueVector v, int index ) { + public void setValue(ValueVector v, int index) { VarCharVector vector = (VarCharVector) v; - long randTime = baseTime + value( ); + long randTime = baseTime + value(); String str = fmt.format(new Date(randTime)); vector.getMutator().setSafe(index, str.getBytes()); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java index ab909c70408..e28a3943bcf 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java @@ -30,17 +30,17 @@ public class DoubleGen implements FieldGen { - private final Random rand = new Random( ); + private final Random rand = new Random(); @Override public void setup(ColumnDef colDef) { } - private double value( ) { + private double value() { return rand.nextDouble() * 2_000_000 - 1_000_000; } @Override - public void setValue( ValueVector v, int index ) { + public void setValue(ValueVector v, int index) { Float8Vector vector = (Float8Vector) v; vector.getMutator().set(index, value()); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java index 8563954020c..f3804d4a3ea 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java @@ -62,11 +62,11 @@ public ExtendedMockRecordReader(FragmentContext context, MockScanEntry config) { this.context = context; this.config = config; - fields = buildColumnDefs( ); + fields = buildColumnDefs(); } private ColumnDef[] buildColumnDefs() { - List defs = new ArrayList<>( ); + List defs = new ArrayList<>(); // Look for duplicate names. Bad things happen when the same name // appears twice. We must do this here because some tests create @@ -75,18 +75,18 @@ private ColumnDef[] buildColumnDefs() { Set names = new HashSet<>(); MockColumn cols[] = config.getTypes(); - for ( int i = 0; i < cols.length; i++ ) { + for (int i = 0; i < cols.length; i++) { MockColumn col = cols[i]; if (names.contains(col.name)) { throw new IllegalArgumentException("Duplicate column name: " + col.name); } names.add(col.name); - int repeat = Math.min( 1, col.getRepeatCount( ) ); - if ( repeat == 1 ) { - defs.add( new ColumnDef(col) ); + int repeat = Math.min(1, col.getRepeatCount()); + if (repeat == 1) { + defs.add(new ColumnDef(col)); } else { - for ( int j = 0; j < repeat; j++ ) { - defs.add( new ColumnDef(col, j+1) ); + for (int j = 0; j < repeat; j++) { + defs.add(new ColumnDef(col, j+1)); } } } @@ -112,7 +112,7 @@ public void setup(OperatorContext context, OutputMutator output) throws Executio for (int i = 0; i < fields.length; i++) { final ColumnDef col = fields[i]; - final MajorType type = col.getConfig( ).getMajorType(); + final MajorType type = col.getConfig().getMajorType(); final MaterializedField field = MaterializedField.create(col.getName(), type); final Class vvClass = TypeHelper.getValueVectorClass(field.getType().getMinorType(), field.getDataMode()); valueVectors[i] = output.addField(field, vvClass); @@ -130,7 +130,7 @@ public int next() { final int recordSetSize = Math.min(batchRecordCount, this.config.getRecords() - recordsRead); recordsRead += recordSetSize; - for ( int i = 0; i < recordSetSize; i++ ) { + for (int i = 0; i < recordSetSize; i++) { int j = 0; for (final ValueVector v : valueVectors) { fields[j++].generator.setValue(v, i); diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java index a9eab04cb68..b51077fc955 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java @@ -33,5 +33,5 @@ */ public interface FieldGen { void setup(ColumnDef colDef); - void setValue( ValueVector v, int index ); + void setValue(ValueVector v, int index); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java index ae5e32432d6..be0054156cb 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java @@ -30,17 +30,17 @@ public class IntGen implements FieldGen { - private final Random rand = new Random( ); + private final Random rand = new Random(); @Override public void setup(ColumnDef colDef) { } - private int value( ) { + private int value() { return rand.nextInt(); } @Override - public void setValue( ValueVector v, int index ) { + public void setValue(ValueVector v, int index) { IntVector vector = (IntVector) v; vector.getMutator().set(index, value()); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java index 2ab56049bb2..df8ee509119 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java @@ -109,9 +109,9 @@ public MockSchema(MockStorageEngine engine) { @Override public Table getTable(String name) { - Pattern p = Pattern.compile( "(\\w+)_(\\d+)(k|m)?", Pattern.CASE_INSENSITIVE); + Pattern p = Pattern.compile("(\\w+)_(\\d+)(k|m)?", Pattern.CASE_INSENSITIVE); Matcher m = p.matcher(name); - if ( ! m.matches() ) { + if (! m.matches()) { return null; } @SuppressWarnings("unused") @@ -122,13 +122,13 @@ public Table getTable(String name) { else if (unit.equalsIgnoreCase("M")) { n *= 1_000_000; } MockScanEntry entry = new MockScanEntry(n, null); List list = new ArrayList<>(); - list.add( entry ); - return new DynamicDrillTable(engine, this.name, list ); + list.add(entry); + return new DynamicDrillTable(engine, this.name, list); } @Override public Set getTableNames() { - return new HashSet<>( ); + return new HashSet<>(); } @Override diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java index afa6b3326ce..f20ff452cd3 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java @@ -29,7 +29,7 @@ public class MockStorageEngineConfig extends StoragePluginConfigBase { private String url; public static final String NAME = "mock"; - public static final MockStorageEngineConfig INSTANCE = new MockStorageEngineConfig( "mock:///" ); + public static final MockStorageEngineConfig INSTANCE = new MockStorageEngineConfig("mock:///"); @JsonCreator public MockStorageEngineConfig(@JsonProperty("url") String url) { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java index 843384eba66..f169f51325d 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java @@ -83,7 +83,7 @@ public MockSubScanPOP(@JsonProperty("url") String url, } public String getUrl() { return url; } - public boolean isExtended( ) { return extended; } + public boolean isExtended() { return extended; } @JsonProperty("entries") public List getReadEntries() { diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java index 8697b3c3738..d4e2379a421 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java @@ -31,17 +31,17 @@ public class MoneyGen implements FieldGen { - private final Random rand = new Random( ); + private final Random rand = new Random(); @Override public void setup(ColumnDef colDef) { } - private double value( ) { - return Math.ceil( rand.nextDouble() * 1_000_000 * 100) / 100; + private double value() { + return Math.ceil(rand.nextDouble() * 1_000_000 * 100) / 100; } @Override - public void setValue( ValueVector v, int index ) { + public void setValue(ValueVector v, int index) { Float8Vector vector = (Float8Vector) v; vector.getMutator().set(index, value()); } diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java index 71543ea6b6c..72be10f6495 100644 --- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java +++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java @@ -31,25 +31,25 @@ public class StringGen implements FieldGen { - private final Random rand = new Random( ); + private final Random rand = new Random(); private int length; @Override - public void setup( ColumnDef colDef ) { + public void setup(ColumnDef colDef) { length = colDef.width; } - private String value( ) { - String c = Character.toString( (char) (rand.nextInt(26) + 'A') ); - StringBuilder buf = new StringBuilder( ); - for ( int i = 0; i < length; i++ ) { - buf.append( c ); + private String value() { + String c = Character.toString((char) (rand.nextInt(26) + 'A')); + StringBuilder buf = new StringBuilder(); + for (int i = 0; i < length; i++) { + buf.append(c); } return buf.toString(); } @Override - public void setValue( ValueVector v, int index ) { + public void setValue(ValueVector v, int index) { VarCharVector vector = (VarCharVector) v; vector.getMutator().setSafe(index, value().getBytes()); }