You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by paul-rogers <gi...@git.apache.org> on 2016/12/28 02:54:43 UTC

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

GitHub user paul-rogers opened a pull request:

    https://github.com/apache/drill/pull/708

    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.
    
    Allows tests to write queries of the form:
    ```
    select id_i, name_s50 from `mock`.`employee_1K` ...
    ```
    Where id_i is a field of random, uniformly distributed integers and name_s50 is a VARCHAR column of width 50 of randomly generated strings. The _1K suffix says to generate 1000 rows. The names are just for convenience, the suffixes tell the mock data source what to generate.
    
    Examples of use will appear in a later commit that includes a revised test framework. Existing tests that use the physical plan version of the mock data source work as before.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/paul-rogers/drill DRILL-5152

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/708.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #708
    
----
commit e9741ce59621209e18cf405c8fe4a614d955ed2a
Author: Paul Rogers <pr...@maprtech.com>
Date:   2016-12-22T05:47:20Z

    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.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95064583
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java ---
    @@ -40,12 +39,12 @@
     
       private final String url;
       protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
    -//  private final OperatorCost cost;
    -//  private final Size size;
    -  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
    +  private boolean extended;
     
       @JsonCreator
    -  public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
    +  public MockSubScanPOP(@JsonProperty("url") String url,
    --- End diff --
    
    This is a constructor used for Jackson serialization. Not sure that Jackson is smart enough to choose among a set of constructors. As it turns out, Jackson will provide a null or default values for parameters that don't' appear in the serialized format. This is why the parameter is a `Boolean`: Jackson will fill in a null value for older physical plans that are non-extended.
    
    Added comments to explain this (admittedly obscure) usage.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95051723
  
    --- Diff: 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<ColumnDef> defs = new ArrayList<>( );
    +
    +    // Look for duplicate names. Bad things happen when the sama name
    +    // appears twice.
    +
    +    Set<String> 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) {
    --- End diff --
    
    `length` parameter is not used anywhere ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95068533
  
    --- Diff: 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 ) {
    --- End diff --
    
    Earlier I though extra spaces were intentional but since you fixed in some places it looks more to be editor problem ? I am still seeing at multiple places, please fix if needed. E.g. `ColumnDef( .... )`, `makeGenerator(  )` and in all the Gen.java files.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95051644
  
    --- Diff: 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<ColumnDef> defs = new ArrayList<>( );
    +
    +    // Look for duplicate names. Bad things happen when the sama name
    --- End diff --
    
    same


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95063860
  
    --- Diff: 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<ColumnDef> defs = new ArrayList<>( );
    +
    +    // Look for duplicate names. Bad things happen when the sama name
    --- End diff --
    
    Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95068500
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java ---
    @@ -125,9 +125,10 @@ public void init() throws DrillbitStartupException {
         availablePlugins = findAvailablePlugins(classpathScan);
     
         // create registered plugins defined in "storage-plugins.json"
    -    this.plugins.putAll(createPlugins());
    +    plugins.putAll(createPlugins());
       }
     
    +  @SuppressWarnings({ "resource" })
    --- End diff --
    
    {} array notation not required.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r94893412
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java ---
    @@ -40,12 +39,12 @@
     
       private final String url;
       protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
    -//  private final OperatorCost cost;
    -//  private final Size size;
    -  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
    +  private boolean extended;
    --- End diff --
    
    final ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95068660
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java ---
    @@ -56,11 +79,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; }
    --- End diff --
    
    formatting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95068510
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java ---
    @@ -346,6 +368,7 @@ public SchemaFactory getSchemaFactory() {
     
       public class DrillSchemaFactory implements SchemaFactory {
     
    +    @SuppressWarnings({ "resource" })
    --- End diff --
    
    same here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r94893846
  
    --- Diff: 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( ) {
    --- End diff --
    
    shouldn't be private for all the types ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95051613
  
    --- Diff: 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);
    --- End diff --
    
    Below should also work. no need to reassign.
    `name += Integer.toString(rep);`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95064584
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java ---
    @@ -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) {
    --- End diff --
    
    See explanation below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #708: DRILL-5152: Enhance the mock data source: better data, SQL...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/708
  
    There is a older version, usable only via a physical plan. This PR is for a new version accessible from SQL. With luck, the new version will be available in 1.10 (and in 1.9 developer builds after the PR is accepted.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/708


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95064123
  
    --- Diff: 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( ) {
    --- End diff --
    
    Fixed. Also added comments. This is test code, so I was a bit sloppy...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95063773
  
    --- Diff: 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);
    --- End diff --
    
    Typo. Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95047914
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java ---
    @@ -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) {
    --- End diff --
    
    Same here - for overloading constructor.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #708: DRILL-5152: Enhance the mock data source: better data, SQL...

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/708
  
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95064163
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java ---
    @@ -40,12 +39,12 @@
     
       private final String url;
       protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
    -//  private final OperatorCost cost;
    -//  private final Size size;
    -  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
    +  private boolean extended;
    --- End diff --
    
    Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95063923
  
    --- Diff: 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<ColumnDef> defs = new ArrayList<>( );
    +
    +    // Look for duplicate names. Bad things happen when the sama name
    +    // appears twice.
    +
    +    Set<String> 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) {
    --- End diff --
    
    Reused this from the original implementation. Taking a closer look; this method is pretty useless. It doesn't even get a vector, just the meta-data description of the field. Eliminated the method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #708: DRILL-5152: Enhance the mock data source: better da...

Posted by sohami <gi...@git.apache.org>.
Github user sohami commented on a diff in the pull request:

    https://github.com/apache/drill/pull/708#discussion_r95047869
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java ---
    @@ -40,12 +39,12 @@
     
       private final String url;
       protected final List<MockGroupScanPOP.MockScanEntry> readEntries;
    -//  private final OperatorCost cost;
    -//  private final Size size;
    -  private  LinkedList<MockGroupScanPOP.MockScanEntry>[] mappings;
    +  private boolean extended;
     
       @JsonCreator
    -  public MockSubScanPOP(@JsonProperty("url") String url, @JsonProperty("entries") List<MockGroupScanPOP.MockScanEntry> readEntries) {
    +  public MockSubScanPOP(@JsonProperty("url") String url,
    --- End diff --
    
    Can we overload the constructor instead ? That way for new parameters default value will be initialized and we don't have to change the older usage.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #708: DRILL-5152: Enhance the mock data source: better data, SQL...

Posted by cgivre <gi...@git.apache.org>.
Github user cgivre commented on the issue:

    https://github.com/apache/drill/pull/708
  
    HI Paul, 
    Is the mock data source actually in Drill 1.9?  I tried executing this query and it threw errors.
    Thx,
    \u2014C 
    
    > On Dec 27, 2016, at 21:54, Paul Rogers <no...@github.com> wrote:
    > 
    > Provides an enhanced version of the mock data source. See the JIRA entry for motivation, package-info.java for details of operation.
    > 
    > Allows tests to write queries of the form:
    > 
    > select id_i, name_s50 from `mock`.`employee_1K` ...
    > Where id_i is a field of random, uniformly distributed integers and name_s50 is a VARCHAR column of width 50 of randomly generated strings. The _1K suffix says to generate 1000 rows. The names are just for convenience, the suffixes tell the mock data source what to generate.
    > 
    > Examples of use will appear in a later commit that includes a revised test framework. Existing tests that use the physical plan version of the mock data source work as before.
    > 
    > You can view, comment on, or merge this pull request online at:
    > 
    >   https://github.com/apache/drill/pull/708 <https://github.com/apache/drill/pull/708>
    > Commit Summary
    > 
    > DRILL-5152: Enhance the mock data source: better data, SQL access
    > File Changes
    > 
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractRecordReader.java <https://github.com/apache/drill/pull/708/files#diff-0> (2)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java <https://github.com/apache/drill/pull/708/files#diff-1> (3)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java <https://github.com/apache/drill/pull/708/files#diff-2> (2)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistryImpl.java <https://github.com/apache/drill/pull/708/files#diff-3> (24)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ColumnDef.java <https://github.com/apache/drill/pull/708/files#diff-4> (178)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DateGen.java <https://github.com/apache/drill/pull/708/files#diff-5> (53)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/DoubleGen.java <https://github.com/apache/drill/pull/708/files#diff-6> (42)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/ExtendedMockRecordReader.java <https://github.com/apache/drill/pull/708/files#diff-7> (149)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/FieldGen.java <https://github.com/apache/drill/pull/708/files#diff-8> (37)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/IntGen.java <https://github.com/apache/drill/pull/708/files#diff-9> (42)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockGroupScanPOP.java <https://github.com/apache/drill/pull/708/files#diff-10> (127)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java <https://github.com/apache/drill/pull/708/files#diff-11> (8)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockScanBatchCreator.java <https://github.com/apache/drill/pull/708/files#diff-12> (8)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java <https://github.com/apache/drill/pull/708/files#diff-13> (79)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngineConfig.java <https://github.com/apache/drill/pull/708/files#diff-14> (9)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorePOP.java <https://github.com/apache/drill/pull/708/files#diff-15> (3)
    > M exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockSubScanPOP.java <https://github.com/apache/drill/pull/708/files#diff-16> (20)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MoneyGen.java <https://github.com/apache/drill/pull/708/files#diff-17> (42)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/StringGen.java <https://github.com/apache/drill/pull/708/files#diff-18> (49)
    > A exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/package-info.java <https://github.com/apache/drill/pull/708/files#diff-19> (130)
    > M exec/java-exec/src/test/java/org/apache/drill/exec/TestOpSerialization.java <https://github.com/apache/drill/pull/708/files#diff-20> (2)
    > M exec/java-exec/src/test/java/org/apache/drill/exec/fn/interp/ExpressionInterpreterTest.java <https://github.com/apache/drill/pull/708/files#diff-21> (4)
    > Patch Links:
    > 
    > https://github.com/apache/drill/pull/708.patch <https://github.com/apache/drill/pull/708.patch>
    > https://github.com/apache/drill/pull/708.diff <https://github.com/apache/drill/pull/708.diff>
    > \u2014
    > You are receiving this because you are subscribed to this thread.
    > Reply to this email directly, view it on GitHub <https://github.com/apache/drill/pull/708>, or mute the thread <https://github.com/notifications/unsubscribe-auth/AFQfviitiJqMVi2vYWaYP7mAceYPFv7Gks5rMc91gaJpZM4LWo0d>.
    > 
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---