You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by am...@apache.org on 2018/12/06 22:40:52 UTC

[drill] branch master updated (2dbd609 -> 637354e)

This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git.


    from 2dbd609  Add Vitalii's GPG key
     new f3eef38  DRILL-6882: Handle the cases where RowKeyJoin's left pipeline being called multiple times.
     new abc51d8  DRILL-6876: Enable CircleCI builds for JDK 9-11
     new 7d37637  DRILL-4546: Only generate one zip archive when using apache-release profile
     new 637354e  DRILL-6751: Upgrade Apache parent POM to version 21

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .circleci/config.yml                               | 135 +++++++++++-
 distribution/pom.xml                               |  49 +++--
 .../src/assemble/bin-assembly-dir.xml              |  23 ++-
 .../src/assemble/bin-assembly.xml                  |  23 ++-
 .../src/assemble/{bin.xml => component.xml}        | 227 ++++++++++-----------
 drill-shaded/pom.xml                               |   3 +-
 .../exec/physical/config/IteratorValidator.java    |  14 +-
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |   5 +
 .../validate/IteratorValidatorBatchIterator.java   |  20 +-
 .../impl/validate/IteratorValidatorCreator.java    |   2 +-
 .../impl/validate/IteratorValidatorInjector.java   |  33 ++-
 .../java-exec/src/test/resources/drill-udf/pom.xml |   3 -
 pom.xml                                            |  41 ++--
 src/main/resources/assemblies/source-assembly.xml  |  99 ---------
 14 files changed, 380 insertions(+), 297 deletions(-)
 copy src/main/resources/checkstyle-suppressions.xml => distribution/src/assemble/bin-assembly-dir.xml (58%)
 copy src/main/resources/checkstyle-suppressions.xml => distribution/src/assemble/bin-assembly.xml (59%)
 rename distribution/src/assemble/{bin.xml => component.xml} (72%)
 delete mode 100644 src/main/resources/assemblies/source-assembly.xml


[drill] 01/04: DRILL-6882: Handle the cases where RowKeyJoin's left pipeline being called multiple times.

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit f3eef38fd633be28f961139b371f524f5172914f
Author: Hanumath Rao Maduri <hm...@maprtech.com>
AuthorDate: Sat Sep 30 13:26:40 2017 -0700

    DRILL-6882: Handle the cases where RowKeyJoin's left pipeline being called multiple times.
    
    close apache/drill#1562
---
 .../exec/physical/config/IteratorValidator.java    | 14 +++++++--
 .../exec/physical/impl/join/RowKeyJoinBatch.java   |  5 ++++
 .../validate/IteratorValidatorBatchIterator.java   | 20 +++++++++----
 .../impl/validate/IteratorValidatorCreator.java    |  2 +-
 .../impl/validate/IteratorValidatorInjector.java   | 33 +++++++++++++++++++---
 5 files changed, 62 insertions(+), 12 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
index 9fbef97..4f73b00 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/IteratorValidator.java
@@ -23,10 +23,20 @@ import org.apache.drill.exec.physical.base.PhysicalVisitor;
 
 public class IteratorValidator extends AbstractSingle{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidator.class);
+  /* isRepeatable flag will be set to true if this validator is created by a Repeatable pipeline.
+   * In a repeatable pipeline some state transitions are valid i.e downstream operator
+   * can call the upstream operator even after receiving NONE.
+   */
+  public final boolean isRepeatable;
 
-  public IteratorValidator(PhysicalOperator child) {
+  public IteratorValidator(PhysicalOperator child, boolean repeatable) {
     super(child);
     setCost(child.getCost());
+    this.isRepeatable = repeatable;
+  }
+
+  public IteratorValidator(PhysicalOperator child) {
+    this(child, false);
   }
 
   @Override
@@ -36,7 +46,7 @@ public class IteratorValidator extends AbstractSingle{
 
   @Override
   protected PhysicalOperator getNewWithChild(PhysicalOperator child) {
-    return new IteratorValidator(child);
+    return new IteratorValidator(child, isRepeatable);
   }
 
   @Override
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
index 941f321..2910da5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/RowKeyJoinBatch.java
@@ -132,6 +132,11 @@ public class RowKeyJoinBatch extends AbstractRecordBatch<RowKeyJoinPOP> implemen
         return IterOutcome.OK;
       }
 
+      if (rightUpstream == IterOutcome.NONE) {
+        rkJoinState = RowKeyJoinState.DONE;
+        state = BatchState.DONE;
+        return rightUpstream;
+      }
       rightUpstream = next(right);
 
       logger.debug("right input IterOutcome: {}", rightUpstream);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
index 1ea3895..5c70f5d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
@@ -47,6 +47,9 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
   /** For logging/debuggability only. */
   private static volatile int instanceCount;
 
+  /** @see org.apache.drill.exec.physical.config.IteratorValidator */
+  private final boolean isRepeatable;
+
   /** For logging/debuggability only. */
   private final int instNum;
   {
@@ -102,12 +105,17 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
    */
   private boolean validateBatches;
 
-  public IteratorValidatorBatchIterator(RecordBatch incoming) {
+  public IteratorValidatorBatchIterator(RecordBatch incoming, boolean isRepeatable) {
     this.incoming = incoming;
     batchTypeName = incoming.getClass().getSimpleName();
+    this.isRepeatable = isRepeatable;
 
     // (Log construction and close() at same level to bracket instance's activity.)
-    logger.trace( "[#{}; on {}]: Being constructed.", instNum, batchTypeName);
+    logger.trace( "[#{}; on {}; repeatable: {}]: Being constructed.", instNum, batchTypeName, isRepeatable);
+  }
+
+  public IteratorValidatorBatchIterator(RecordBatch incoming) {
+    this(incoming, false);
   }
 
 
@@ -217,7 +225,7 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
                 instNum, batchTypeName, exceptionState, batchState));
       }
       // (Note:  This could use validationState.)
-      if (batchState == NONE || batchState == STOP) {
+      if ((!isRepeatable && batchState == NONE) || batchState == STOP) {
         throw new IllegalStateException(
             String.format(
                 "next() [on #%d, %s] called again after it returned %s."
@@ -256,8 +264,10 @@ public class IteratorValidatorBatchIterator implements CloseableRecordBatch {
         case NONE:
           // NONE is allowed even without seeing a OK_NEW_SCHEMA. Such NONE is called
           // FAST NONE.
-          // NONE moves to terminal high-level state.
-          validationState = ValidationState.TERMINAL;
+          // NONE moves to TERMINAL high-level state if NOT repeatable.
+          if (!isRepeatable) {
+            validationState = ValidationState.TERMINAL;
+          }
           break;
         case STOP:
           // STOP is allowed at any time, except if already terminated (checked
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
index 4dc58e5..b7be8ab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorCreator.java
@@ -37,7 +37,7 @@ public class IteratorValidatorCreator implements BatchCreator<IteratorValidator>
       throws ExecutionSetupException {
     Preconditions.checkArgument(children.size() == 1);
     RecordBatch child = children.iterator().next();
-    IteratorValidatorBatchIterator iter = new IteratorValidatorBatchIterator(child);
+    IteratorValidatorBatchIterator iter = new IteratorValidatorBatchIterator(child, config.isRepeatable);
     boolean validateBatches = context.getOptions().getOption(ExecConstants.ENABLE_VECTOR_VALIDATOR) ||
                               context.getConfig().getBoolean(ExecConstants.ENABLE_VECTOR_VALIDATION);
     iter.enableBatchValidation(validateBatches);
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
index 20eba16..6d86fb3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorInjector.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
 import org.apache.drill.exec.physical.base.FragmentRoot;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.IteratorValidator;
+import org.apache.drill.exec.physical.config.RowKeyJoinPOP;
 
 import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
@@ -32,6 +33,17 @@ public class IteratorValidatorInjector extends
     AbstractPhysicalVisitor<PhysicalOperator, FragmentContext, ExecutionSetupException> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidatorInjector.class);
 
+  /* This flag when set creates all the validators as repeatable validators */
+  private final boolean isRepeatablePipeline;
+
+  public IteratorValidatorInjector() {
+    this(false);
+  }
+
+  public IteratorValidatorInjector(boolean repeatablePipeline) {
+    this.isRepeatablePipeline = repeatablePipeline;
+  }
+
   public static FragmentRoot rewritePlanWithIteratorValidator(FragmentContext context, FragmentRoot root) throws ExecutionSetupException {
     IteratorValidatorInjector inject = new IteratorValidatorInjector();
     PhysicalOperator newOp = root.accept(inject, context);
@@ -60,11 +72,24 @@ public class IteratorValidatorInjector extends
     List<PhysicalOperator> newChildren = Lists.newArrayList();
     PhysicalOperator newOp = op;
 
+    if (op instanceof RowKeyJoinPOP) {
+      /* create a RepeatablePipeline for the left side of RowKeyJoin */
+      PhysicalOperator left = new IteratorValidator(((RowKeyJoinPOP) op).getLeft()
+                                   .accept(new IteratorValidatorInjector(true), context), true);
+      left.setOperatorId(op.getOperatorId() + 1000);
+      newChildren.add(left);
+      /* right pipeline is not repeatable pipeline */
+      PhysicalOperator right = new IteratorValidator(((RowKeyJoinPOP) op).getRight()
+              .accept(this, context));
+      right.setOperatorId(op.getOperatorId() + 1000);
+      newChildren.add(right);
+    } else {
     /* Get the list of child operators */
-    for (PhysicalOperator child : op) {
-      PhysicalOperator validator = new IteratorValidator(child.accept(this, context));
-      validator.setOperatorId(op.getOperatorId() + 1000);
-      newChildren.add(validator);
+      for (PhysicalOperator child : op) {
+        PhysicalOperator validator = new IteratorValidator(child.accept(this, context), this.isRepeatablePipeline);
+        validator.setOperatorId(op.getOperatorId() + 1000);
+        newChildren.add(validator);
+      }
     }
 
     /* Inject trace operator */


[drill] 03/04: DRILL-4546: Only generate one zip archive when using apache-release profile

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 7d3763728fd82b3a033936b14ff35545e93d1c5c
Author: Laurent Goujon <la...@dremio.com>
AuthorDate: Mon Mar 28 13:31:53 2016 -0700

    DRILL-4546: Only generate one zip archive when using apache-release profile
    
    Drill root pom doesn't override completely Apache parent pom configuration
    regarding assemblies, which caused a zip archive of the project to be generated
    twice, and deployed to a remote server twice too.
    
    The fix uses the plugin properties to override the configuration. Also remove
    Drill source assembly descriptor as the Apache parent project provides the same one.
---
 pom.xml                                           | 18 ++---
 src/main/resources/assemblies/source-assembly.xml | 99 -----------------------
 2 files changed, 5 insertions(+), 112 deletions(-)

diff --git a/pom.xml b/pom.xml
index c9d2843..790dd11 100644
--- a/pom.xml
+++ b/pom.xml
@@ -38,6 +38,10 @@
   <url>http://drill.apache.org/</url>
 
   <properties>
+    <maven.compiler.source>1.8</maven.compiler.source>
+    <maven.compiler.target>1.8</maven.compiler.target>
+    <sourceReleaseAssemblyDescriptor>source-release-zip-tar</sourceReleaseAssemblyDescriptor>
+
     <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
     <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
     <dep.junit.version>4.12</dep.junit.version>
@@ -471,8 +475,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
         <configuration>
-          <source>1.8</source>
-          <target>1.8</target>
           <maxmem>2048m</maxmem>
           <useIncrementalCompilation>false</useIncrementalCompilation>
           <fork>true</fork>
@@ -1682,25 +1684,15 @@
               <failOnError>false</failOnError>
             </configuration>
           </plugin>
-          <!-- override the parent assembly execution to customize the assembly
-            descriptor and final name -->
+          <!-- override the parent assembly execution to customize the assembly final name -->
           <plugin>
             <artifactId>maven-assembly-plugin</artifactId>
             <executions>
               <execution>
                 <id>source-release-assembly</id>
-                <phase>package</phase>
-                <goals>
-                  <goal>single</goal>
-                </goals>
                 <configuration>
-                  <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
                   <appendAssemblyId>false</appendAssemblyId>
-                  <descriptors>
-                    <descriptor>src/main/resources/assemblies/source-assembly.xml</descriptor>
-                  </descriptors>
                   <finalName>apache-drill-${project.version}-src</finalName>
-                  <tarLongFileMode>gnu</tarLongFileMode>
                 </configuration>
               </execution>
             </executions>
diff --git a/src/main/resources/assemblies/source-assembly.xml b/src/main/resources/assemblies/source-assembly.xml
deleted file mode 100644
index 4ed43f1..0000000
--- a/src/main/resources/assemblies/source-assembly.xml
+++ /dev/null
@@ -1,99 +0,0 @@
-<?xml version='1.0' encoding='UTF-8'?>
-<!--
-
-    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.
-
--->
-<assembly>
-  <id>source-release</id>
-  <formats>
-    <format>zip</format>
-    <format>tar.gz</format>
-  </formats>
-  <fileSets>
-    <!-- main project directory structure -->
-    <fileSet>
-      <directory>.</directory>
-      <outputDirectory>/</outputDirectory>
-      <useDefaultExcludes>true</useDefaultExcludes>
-      <excludes>
-        <!-- build output -->
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]
-        </exclude>
-
-        <!-- NOTE: Most of the following excludes should not be required
-          if the standard release process is followed. This is because the release
-          plugin checks out project sources into a location like target/checkout, then
-          runs the build from there. The result is a source-release archive that comes
-          from a pretty clean directory structure. HOWEVER, if the release plugin is
-          configured to run extra goals or generate a project website, it's definitely
-          possible that some of these files will be present. So, it's safer to exclude
-          them. -->
-
-        <!-- IDEs -->
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iws]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.idea(/.*)?]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?out(/.*)?]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.ipr]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?[^/]*\.iml]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.settings(/.*)?]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.externalToolBuilders(/.*)?]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.deployables(/.*)?]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.wtpmodules(/.*)?]
-        </exclude>
-
-
-        <!-- scm -->
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.gitignore(/.*)?]
-        </exclude>
-
-        <exclude>**/.buildpath</exclude>
-        <exclude>**/sandbox/**</exclude>
-
-        <!-- misc -->
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?cobertura\.ser]
-        </exclude>
-
-        <!-- release-plugin temp files -->
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?pom\.xml\.releaseBackup]
-        </exclude>
-        <exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?release\.properties]
-        </exclude>
-      </excludes>
-    </fileSet>
-    <!-- license, readme, etc. calculated at build time -->
-    <fileSet>
-      <directory>${project.build.directory}/maven-shared-archive-resources/META-INF
-      </directory>
-      <outputDirectory>/</outputDirectory>
-    </fileSet>
-  </fileSets>
-</assembly>


[drill] 04/04: DRILL-6751: Upgrade Apache parent POM to version 21

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 637354e91b4b597a0f1930a385b7f0bff0093a2a
Author: Vitalii Diravka <vi...@apache.org>
AuthorDate: Thu Nov 15 17:40:21 2018 -0800

    DRILL-6751: Upgrade Apache parent POM to version 21
    
    - Update apache.pom file version to 21 (with updating some maven plugins versions)
    - Include Drill's sources jars on assembly stage in <moduleSets> (not <dependencySets>)
      for properincluding jars with last apache-21.pom
    - Separate "distro-assembly" to the two execution stages to avoid:
      [WARNING] Assembly file: <DRILL_HOME>/distribution/target/apache-drill-1.15.0-SNAPSHOT is not a regular
      file (it may be a directory). It cannot be attached to the project build for installation or deployment.
    - Remove unsused <include>/<exclude> in assebly descriptor to avoid:
      [WARNING] The following patterns were never triggered in this artifact inclusion filter
    - Update "maven-assembly-plugin" version
    - Update "slf4j" version
    - Update "mockito-core" version
    - Update "bcpkix-jdk15on" (Bouncy Castle Cryptography APIs) version
    
    close apache/drill#1561
---
 distribution/pom.xml                               |  49 +++--
 distribution/src/assemble/bin-assembly-dir.xml     |  33 +++
 distribution/src/assemble/bin-assembly.xml         |  33 +++
 .../src/assemble/{bin.xml => component.xml}        | 227 ++++++++++-----------
 drill-shaded/pom.xml                               |   3 +-
 .../java-exec/src/test/resources/drill-udf/pom.xml |   3 -
 pom.xml                                            |  23 +--
 7 files changed, 218 insertions(+), 153 deletions(-)

diff --git a/distribution/pom.xml b/distribution/pom.xml
index 4952ca5..4fde4cd 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -156,26 +156,40 @@
 
   <build>
     <plugins>
-    <plugin>
+      <plugin>
         <artifactId>maven-assembly-plugin</artifactId>
         <executions>
-            <execution>
-                <id>distro-assembly</id>
-                <phase>package</phase>
-                <goals>
-                    <goal>single</goal>
-                </goals>
-                <configuration>
-                    <descriptors>
-                        <descriptor>src/assemble/bin.xml</descriptor>
-                    </descriptors>
-                    <appendAssemblyId>false</appendAssemblyId>
-                    <finalName>apache-drill-${project.version}</finalName>
-                    <tarLongFileMode>gnu</tarLongFileMode>
-                </configuration>
-            </execution>
+          <execution>
+            <id>distro-assembly</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <descriptors>
+                <descriptor>src/assemble/bin-assembly.xml</descriptor>
+              </descriptors>
+              <appendAssemblyId>false</appendAssemblyId>
+              <finalName>apache-drill-${project.version}</finalName>
+            </configuration>
+          </execution>
+          <execution>
+            <id>distro-assembly-not-attached</id>
+            <phase>package</phase>
+            <goals>
+              <goal>single</goal>
+            </goals>
+            <configuration>
+              <attach>false</attach>
+              <descriptors>
+                <descriptor>src/assemble/bin-assembly-dir.xml</descriptor>
+              </descriptors>
+              <appendAssemblyId>false</appendAssemblyId>
+              <finalName>apache-drill-${project.version}</finalName>
+            </configuration>
+          </execution>
         </executions>
-    </plugin>
+      </plugin>
       <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
@@ -192,7 +206,6 @@
           </excludes>
         </configuration>
       </plugin>
-
     </plugins>
   </build>
   <profiles>
diff --git a/distribution/src/assemble/bin-assembly-dir.xml b/distribution/src/assemble/bin-assembly-dir.xml
new file mode 100644
index 0000000..f134962
--- /dev/null
+++ b/distribution/src/assemble/bin-assembly-dir.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0"?>
+<!--
+
+    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.
+
+-->
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+  <id>binary-release-directory</id>
+  <formats>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>true</includeBaseDirectory>
+  <componentDescriptors>
+    <componentDescriptor>component.xml</componentDescriptor>
+  </componentDescriptors>
+</assembly>
diff --git a/distribution/src/assemble/bin-assembly.xml b/distribution/src/assemble/bin-assembly.xml
new file mode 100644
index 0000000..5cd3636
--- /dev/null
+++ b/distribution/src/assemble/bin-assembly.xml
@@ -0,0 +1,33 @@
+<?xml version="1.0"?>
+<!--
+
+    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.
+
+-->
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+  <id>binary-release</id>
+  <formats>
+    <format>tar.gz</format>
+  </formats>
+  <includeBaseDirectory>true</includeBaseDirectory>
+  <componentDescriptors>
+    <componentDescriptor>component.xml</componentDescriptor>
+  </componentDescriptors>
+</assembly>
diff --git a/distribution/src/assemble/bin.xml b/distribution/src/assemble/component.xml
similarity index 72%
rename from distribution/src/assemble/bin.xml
rename to distribution/src/assemble/component.xml
index 85a4d5b..2b60299 100644
--- a/distribution/src/assemble/bin.xml
+++ b/distribution/src/assemble/component.xml
@@ -18,27 +18,115 @@
     limitations under the License.
 
 -->
-<assembly
-  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
-  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
-  <id>binary-release</id>
-  <formats>
-    <format>tar.gz</format>
-    <format>dir</format>
-  </formats>
-  <includeBaseDirectory>true</includeBaseDirectory>
+<component xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+           xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+           xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2 http://maven.apache.org/xsd/assembly-1.1.2.xsd">
   <moduleSets>
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
+     <!--Now, select which projects to include in this module-set. -->
+    <includes>
+      <include>org.apache.drill.exec:drill-jdbc:jar</include>
+      <include>org.apache.drill:drill-protocol:jar</include>
+      <include>org.apache.drill:drill-common:jar</include>
+      <include>org.apache.drill:drill-logical:jar</include>
+      <include>org.apache.drill.exec:vector:jar</include>
+      <include>org.apache.drill.memory:drill-memory-base:jar</include>
+      <include>org.apache.drill.exec:drill-rpc:jar</include>
+      <include>org.apache.drill.exec:drill-java-exec:jar</include>
+      <include>org.apache.drill.contrib.storage-hive:drill-storage-hive-core:jar</include>
+      <include>org.apache.drill.contrib.storage-hive:drill-hive-exec-shaded:jar</include>
+      <include>org.apache.drill.contrib.data:tpch-sample-data:jar</include>
+      <include>org.apache.drill.contrib:drill-mongo-storage:jar</include>
+      <include>org.apache.drill.contrib:drill-storage-hbase:jar</include>
+      <include>org.apache.drill.contrib:drill-format-mapr:jar</include>
+      <include>org.apache.drill.contrib:drill-jdbc-storage:jar</include>
+      <include>org.apache.drill.contrib:drill-kudu-storage:jar</include>
+      <include>org.apache.drill.contrib:drill-storage-kafka:jar</include>
+      <include>org.apache.drill.contrib:drill-opentsdb-storage:jar</include>
+      <include>org.apache.drill.contrib:drill-udfs:jar</include>
+    </includes>
+      <sources>
+        <includeModuleDirectory>false</includeModuleDirectory>
+        <fileSets>
+          <fileSet>
+            <directory>target</directory>
+            <outputDirectory>jars</outputDirectory>
+            <includes>
+              <include>*.jar</include>
+            </includes>
+            <excludes>
+              <exclude>*sources.jar</exclude>
+              <exclude>*javadoc.jar</exclude>
+              <exclude>*tests.jar</exclude>
+              <exclude>original-*</exclude>
+            </excludes>
+          </fileSet>
+        </fileSets>
+      </sources>
+    </moduleSet>
 
-  </moduleSets>
-  <dependencySets>
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
+      <!-- Drill-on-YARN goes into its own directory so it does not end up
+           on the Drillbit class path. Define the jars/tools folder as any jar
+           we ship that is not used by drillbits. -->
+      <includes>
+        <include>org.apache.drill:drill-yarn:jar</include>
+      </includes>
+      <sources>
+        <includeModuleDirectory>false</includeModuleDirectory>
+        <fileSets>
+          <fileSet>
+            <directory>target</directory>
+            <outputDirectory>jars/tools</outputDirectory>
+            <includes>
+              <include>*.jar</include>
+            </includes>
+            <excludes>
+              <exclude>*sources.jar</exclude>
+              <exclude>*javadoc.jar</exclude>
+              <exclude>*tests.jar</exclude>
+            </excludes>
+          </fileSet>
+        </fileSets>
+      </sources>
+    </moduleSet>
 
-    <dependencySet>
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
       <includes>
         <include>org.apache.drill.exec:drill-jdbc-all:jar</include>
       </includes>
-      <outputDirectory>jars/jdbc-driver</outputDirectory>
+      <sources>
+        <includeModuleDirectory>false</includeModuleDirectory>
+        <fileSets>
+          <fileSet>
+            <directory>target</directory>
+            <outputDirectory>jars/jdbc-driver</outputDirectory>
+            <includes>
+              <include>*.jar</include>
+            </includes>
+            <excludes>
+              <exclude>*sources.jar</exclude>
+              <exclude>*javadoc.jar</exclude>
+              <exclude>*tests.jar</exclude>
+              <exclude>original-*</exclude>
+            </excludes>
+          </fileSet>
+        </fileSets>
+      </sources>
+    </moduleSet>
+  </moduleSets>
+
+  <dependencySets>
+    <dependencySet>
+      <outputDirectory>jars</outputDirectory>
+      <unpack>false</unpack>
       <useProjectArtifact>false</useProjectArtifact>
+      <includes>
+        <include>org.apache.drill:drill-shaded-guava:jar</include>
+      </includes>
     </dependencySet>
 
     <dependencySet>
@@ -46,14 +134,11 @@
       <unpack>false</unpack>
       <useProjectArtifact>false</useProjectArtifact>
       <includes>
-
         <include>org.glassfish.jersey.containers</include>
         <include>org.glassfish.jersey.core</include>
         <include>org.reflections</include>
-
         <include>org.glassfish.hk2.external</include>
         <include>org.mortbay.jetty</include>
-        <include>org.jboss.spec.javax.transaction</include>
         <include>javax.activation</include>
         <include>javax.annotation</include>
         <include>org.glassfish.jersey.containers</include>
@@ -63,69 +148,15 @@
         <include>org.glassfish.jersey.bundles.repackaged</include>
         <include>org.glassfish.hk2</include>
         <include>javax.servlet</include>
-
-        <include>javax.ws.rs </include>
-        <include>com.sun.xml.bind  </include>
-        <include>com.sun.jersey  </include>
-        <include>org.jvnet.mimepull  </include>
-        <include>com.sun.codemodel  </include>
+        <include>javax.ws.rs</include>
+        <include>org.jvnet.mimepull</include>
+        <include>com.sun.codemodel</include>
         <include>javax.xml.bind</include>
-
-        <include>org.jruby</include>
-        <include>ch.qos.logback </include>
         <include>ch.qos.logback</include>
-        <include>org.eclipse.jdt</include>
-        <include>com.github.stephenc.high-scale-lib</include>
-
-        <include>org.jamon</include>
-
+        <include>ch.qos.logback</include>
       </includes>
       <scope>test</scope>
     </dependencySet>
-
-    <dependencySet>
-      <!-- Now, select which projects to include in this module-set. -->
-      <includes>
-        <include>org.apache.drill.exec:drill-jdbc:jar</include>
-        <include>org.apache.drill:drill-protocol:jar</include>
-        <include>org.apache.drill:drill-common:jar</include>
-        <include>org.apache.drill:drill-logical:jar</include>
-        <include>org.apache.drill.exec:vector:jar</include>
-        <include>org.apache.drill.memory:drill-memory-base:jar</include>
-        <include>org.apache.drill.exec:drill-rpc:jar</include>
-        <include>org.apache.drill.exec:drill-java-exec:jar</include>
-        <include>org.apache.drill.contrib.storage-hive:drill-storage-hive-core</include>
-        <include>org.apache.drill.contrib.storage-hive:drill-hive-exec-shaded</include>
-        <include>org.apache.drill.contrib.data:tpch-sample-data:jar</include>
-        <include>org.apache.drill.contrib:drill-mongo-storage</include>
-        <include>org.apache.drill.contrib:drill-storage-hbase</include>
-        <include>org.apache.drill.contrib:drill-format-mapr</include>
-        <include>org.apache.drill.contrib:drill-jdbc-storage</include>
-        <include>org.apache.drill.contrib:drill-kudu-storage</include>
-        <include>org.apache.drill.contrib:drill-storage-kafka</include>
-        <include>org.apache.drill.contrib:drill-opentsdb-storage</include>
-        <include>org.apache.drill.contrib:drill-udfs</include>
-        <include>org.apache.drill:drill-shaded-guava</include>
-      </includes>
-      <excludes>
-        <exclude>org.apache.drill.contrib.storage-hive:drill-storage-hive-core:jar:tests</exclude>
-      </excludes>
-      <outputDirectory>jars</outputDirectory>
-      <useProjectArtifact>false</useProjectArtifact>
-    </dependencySet>
-
-    <dependencySet>
-      <!-- Drill-on-YARN goes into its own directory so it does not end up
-           on the Drillbit class path. Define the jars/tools folder as any jar
-           we ship that is not used by drillbits. -->
-      <includes>
-        <include>org.apache.drill:drill-yarn:jar</include>
-      </includes>
-      <outputDirectory>jars/tools</outputDirectory>
-      <useProjectArtifact>false</useProjectArtifact>
-    </dependencySet>
-
-
     <dependencySet>
       <outputDirectory>jars/3rdparty/</outputDirectory>
       <unpack>false</unpack>
@@ -139,28 +170,13 @@
         <exclude>org.apache.drill.contrib.storage-hive</exclude>
         <exclude>org.apache.drill.memory</exclude>
         <!-- Below Hive jars are already included in drill-hive-exec-shaded.jar -->
-        <exclude>org.apache.hive:hive-common</exclude>
-        <exclude>org.apache.hive:hive-serde</exclude>
-        <exclude>org.apache.hive:hive-shims</exclude>
-        <exclude>org.apache.hive.shims:hive-shims-0.20</exclude>
-        <exclude>org.apache.hive.shims:hive-shims-0.20S</exclude>
-        <exclude>org.apache.hive.shims:hive-shims-0.23</exclude>
-        <exclude>org.apache.hive.shims:hive-shims-common</exclude>
-        <exclude>org.apache.hive.shims:hive-shims-common-secure</exclude>
-        <exclude>org.hsqldb:hsqldb</exclude>
         <exclude>junit:junit:jar</exclude>
-        <exclude>com.googlecode.jmockit:jmockit:jar</exclude>
         <!-- exclude or sqlline has problems -->
-        <exclude>hsqldb:hsqldb</exclude>
-        <!-- exclude or sqlline has problems -->
-
         <exclude>org.glassfish.jersey.containers</exclude>
         <exclude>org.glassfish.jersey.core</exclude>
         <exclude>org.reflections</exclude>
-
         <exclude>org.glassfish.hk2.external</exclude>
         <exclude>org.mortbay.jetty</exclude>
-        <exclude>org.jboss.spec.javax.transaction</exclude>
         <exclude>javax.activation</exclude>
         <exclude>javax.annotation</exclude>
         <exclude>org.glassfish.jersey.containers</exclude>
@@ -170,28 +186,16 @@
         <exclude>org.glassfish.jersey.bundles.repackaged</exclude>
         <exclude>org.glassfish.hk2</exclude>
         <exclude>javax.servlet</exclude>
-
         <exclude>javax.ws.rs </exclude>
-        <exclude>com.sun.xml.bind  </exclude>
-        <exclude>com.sun.jersey  </exclude>
-        <exclude>org.jvnet.mimepull  </exclude>
-        <exclude>com.sun.codemodel  </exclude>
+        <exclude>org.jvnet.mimepull</exclude>
+        <exclude>com.sun.codemodel</exclude>
         <exclude>javax.xml.bind</exclude>
-
-        <exclude>org.jruby</exclude>
         <exclude>ch.qos.logback </exclude>
         <exclude>ch.qos.logback</exclude>
-        <exclude>org.eclipse.jdt</exclude>
-        <exclude>com.github.stephenc.high-scale-lib</exclude>
-
         <exclude>io.netty:netty-tcnative</exclude>
-
-        <exclude>org.jamon</exclude>
-
       </excludes>
       <scope>test</scope>
     </dependencySet>
-
     <dependencySet>
       <outputDirectory>jars/ext</outputDirectory>
       <unpack>false</unpack>
@@ -201,16 +205,6 @@
       </includes>
       <scope>runtime</scope>
     </dependencySet>
-
-    <dependencySet>
-      <outputDirectory>jars/3rdparty</outputDirectory>
-      <unpack>false</unpack>
-      <useProjectArtifact>false</useProjectArtifact>
-      <includes>
-        <include>com.google.protobuf:protobuf-java:jar:2.5</include>
-      </includes>
-      <scope>test</scope>
-    </dependencySet>
     <dependencySet>
       <outputDirectory>jars/3rdparty/linux</outputDirectory>
       <unpack>false</unpack>
@@ -247,7 +241,6 @@
       </includes>
       <scope>test</scope>
     </dependencySet>
-
     <dependencySet>
       <outputDirectory>winutils/bin</outputDirectory>
       <unpack>true</unpack>
@@ -263,7 +256,6 @@
         <include>org.apache.hadoop:hadoop-winutils</include>
       </includes>
     </dependencySet>
-
   </dependencySets>
 
   <fileSets>
@@ -433,4 +425,5 @@
       <fileMode>0640</fileMode>
     </file>
   </files>
-</assembly>
+
+</component>
diff --git a/drill-shaded/pom.xml b/drill-shaded/pom.xml
index f9c6045..b3bb3f8 100644
--- a/drill-shaded/pom.xml
+++ b/drill-shaded/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache</groupId>
     <artifactId>apache</artifactId>
-    <version>18</version>
+    <version>21</version>
     <relativePath/>
   </parent>
 
@@ -41,7 +41,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-shade-plugin</artifactId>
-          <version>3.1.0</version>
           <executions>
             <execution>
               <phase>package</phase>
diff --git a/exec/java-exec/src/test/resources/drill-udf/pom.xml b/exec/java-exec/src/test/resources/drill-udf/pom.xml
index 931abed..0e25cba 100644
--- a/exec/java-exec/src/test/resources/drill-udf/pom.xml
+++ b/exec/java-exec/src/test/resources/drill-udf/pom.xml
@@ -58,13 +58,10 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-compiler-plugin</artifactId>
-        <version>3.1</version>
         <configuration>
           <includes>
             <include>${include.files}</include>
           </includes>
-          <source>1.8</source>
-          <target>1.8</target>
         </configuration>
       </plugin>
       <plugin>
diff --git a/pom.xml b/pom.xml
index 790dd11..94427b8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -24,7 +24,7 @@
   <parent>
     <groupId>org.apache</groupId>
     <artifactId>apache</artifactId>
-    <version>18</version>
+    <version>21</version>
     <relativePath />
   </parent>
 
@@ -45,7 +45,7 @@
     <target.gen.source.path>${project.basedir}/target/generated-sources</target.gen.source.path>
     <proto.cas.path>${project.basedir}/src/main/protobuf/</proto.cas.path>
     <dep.junit.version>4.12</dep.junit.version>
-    <dep.slf4j.version>1.7.6</dep.slf4j.version>
+    <dep.slf4j.version>1.7.25</dep.slf4j.version>
     <shaded.guava.version>23.0</shaded.guava.version>
     <guava.version>19.0</guava.version>
     <forkCount>2</forkCount>
@@ -89,7 +89,7 @@
     <excludedGroups />
     <memoryMb>4096</memoryMb>
     <directMemoryMb>4096</directMemoryMb>
-    <additionalparam>-Xdoclint:none</additionalparam>
+    <doclint>none</doclint>
     <rat.skip>true</rat.skip>
     <license.skip>true</license.skip>
     <docker.repository>drill/apache-drill</docker.repository>
@@ -574,7 +574,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-jar-plugin</artifactId>
-          <version>3.1.0</version>
           <executions>
             <execution>
               <goals>
@@ -689,7 +688,7 @@
         </plugin>
         <plugin>
           <artifactId>maven-enforcer-plugin</artifactId>
-          <version>3.0.0-M1</version>
+          <version>3.0.0-M2</version>
         </plugin>
         <plugin> <!-- classpath scanning  -->
           <groupId>org.codehaus.mojo</groupId>
@@ -714,7 +713,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.21.0</version>
           <executions>
             <execution>
               <id>default-test</id>
@@ -757,7 +755,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-release-plugin</artifactId>
-          <version>2.5.2</version>
           <configuration>
             <useReleaseProfile>false</useReleaseProfile>
             <pushChanges>false</pushChanges>
@@ -859,7 +856,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-shade-plugin</artifactId>
-          <version>3.1.0</version>
           <executions>
             <execution>
               <phase>package</phase>
@@ -946,7 +942,7 @@
            long as Mockito _contains_ older Hamcrest classes.  See DRILL-2130. -->
       <groupId>org.mockito</groupId>
       <artifactId>mockito-core</artifactId>
-      <version>2.18.3</version>
+      <version>2.23.4</version>
       <scope>test</scope>
     </dependency>
     <dependency>
@@ -1667,11 +1663,11 @@
       <build>
         <plugins>
           <plugin>
-            <groupId>net.ju-n.maven.plugins</groupId>
+            <groupId>net.nicoulaj.maven.plugins</groupId>
             <artifactId>checksum-maven-plugin</artifactId>
-            <version>1.2</version>
             <executions>
               <execution>
+                <id>source-release-checksum</id>
                 <goals>
                   <goal>artifacts</goal>
                 </goals>
@@ -1681,12 +1677,13 @@
               <algorithms>
                 <algorithm>SHA-512</algorithm>
               </algorithms>
-              <failOnError>false</failOnError>
             </configuration>
           </plugin>
+
           <!-- override the parent assembly execution to customize the assembly final name -->
           <plugin>
             <artifactId>maven-assembly-plugin</artifactId>
+            <version>3.1.0</version>
             <executions>
               <execution>
                 <id>source-release-assembly</id>
@@ -2369,7 +2366,7 @@
           <dependency>
             <groupId>org.bouncycastle</groupId>
             <artifactId>bcpkix-jdk15on</artifactId>
-            <version>1.52</version>
+            <version>1.60</version>
           </dependency>
 
           <!-- Test Dependencies -->


[drill] 02/04: DRILL-6876: Enable CircleCI builds for JDK 9-11

Posted by am...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

amansinha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit abc51d80095cf5df2de64a3acbf75bdd88bd1db5
Author: Volodymyr Vysotskyi <vv...@gmail.com>
AuthorDate: Fri Nov 16 15:43:27 2018 +0200

    DRILL-6876: Enable CircleCI builds for JDK 9-11
    
    close apache/drill#1558
---
 .circleci/config.yml | 135 ++++++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 133 insertions(+), 2 deletions(-)

diff --git a/.circleci/config.yml b/.circleci/config.yml
index 1b179df..e2b1141 100644
--- a/.circleci/config.yml
+++ b/.circleci/config.yml
@@ -16,7 +16,7 @@
 version: 2
 general:
 jobs:
-  build:
+  build_jdk8:
     machine:
       enabled: true
       image: circleci/classic:latest
@@ -33,7 +33,7 @@ jobs:
         # TODO: Could be removed, once Machine Executor image is updated https://github.com/circleci/image-builder/issues/140
         # and the possibility of specifying Maven version is added https://github.com/circleci/image-builder/issues/143
         command:
-          curl -fsSL https://git.io/vpDIf | bash -s -- 3.5.4
+          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
     - run:
         name: Update packages list
         command:
@@ -49,3 +49,134 @@ jobs:
         # TODO: 4. Resolving memory issues without "SlowTest" and "UnlikelyTest" excludedGroups in the build
         command:
           mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608 -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest"
+
+  build_jdk9:
+    machine:
+      enabled: true
+      image: circleci/classic:latest
+    parallelism: 1
+
+    working_directory: ~/drill
+
+    steps:
+    - checkout
+
+    - run:
+        name: Update packages list
+        command:
+          sudo apt-get update
+
+    - run:
+        name: Install java 9
+        command:
+          sudo apt-get -y install openjdk-9-jdk
+
+    - run:
+        name: Set default java 9
+        command:
+          sudo update-java-alternatives --set java-1.9.0-openjdk-amd64
+
+    - run:
+        name: Update maven version
+        command:
+          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
+
+    - run:
+        name: Install libaio1.so library for MySQL integration tests
+        command:
+          sudo apt-get install libaio1 libaio-dev
+    - run:
+        name: Drill project build
+        command:
+          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608 -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest"
+
+  build_jdk10:
+    machine:
+      enabled: true
+      image: circleci/classic:latest
+    parallelism: 1
+
+    working_directory: ~/drill
+
+    steps:
+    - checkout
+
+    - run:
+        name: Update packages list
+        command:
+          sudo apt-get update
+
+    - run:
+        name: Install java 10
+        command:
+          sudo apt-get -y install openjdk-10-jdk
+
+    - run:
+        name: Set default java 10
+        command:
+          sudo update-java-alternatives --set java-1.10.0-openjdk-amd64
+
+    - run:
+        name: Update maven version
+        command:
+          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
+
+    - run:
+        name: Install libaio1.so library for MySQL integration tests
+        command:
+          sudo apt-get install libaio1 libaio-dev
+    - run:
+        name: Drill project build
+        command:
+          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608 -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest"
+
+  build_jdk11:
+    machine:
+      enabled: true
+      image: circleci/classic:latest
+    parallelism: 1
+
+    working_directory: ~/drill
+
+    steps:
+    - checkout
+
+    - run:
+        name: Update packages list
+        command:
+          sudo apt-get update
+
+    - run:
+        name: Install java 11
+        command:
+          sudo apt-get -y install openjdk-11-jdk
+
+    - run:
+        name: Set default java 11
+        command:
+          sudo update-java-alternatives --set java-1.11.0-openjdk-amd64
+
+    - run:
+        name: Update maven version
+        command:
+          curl -fsSL https://git.io/vpDIf | bash -s -- 3.6.0
+
+    - run:
+        name: Install libaio1.so library for MySQL integration tests
+        command:
+          sudo apt-get install libaio1 libaio-dev
+    - run:
+        name: Drill project build
+        # Set forkCount to 1 since tests use more memory and memory limitations for CircleCI is reached
+        # for default value of forkCount.
+        command:
+          mvn install -Drat.skip=false -Dlicense.skip=false -DmemoryMb=2560 -DdirectMemoryMb=4608 -DforkCount=1 -DexcludedGroups="org.apache.drill.categories.SlowTest,org.apache.drill.categories.UnlikelyTest"
+
+workflows:
+  version: 2
+  build_and_test:
+    jobs:
+    - build_jdk8
+    - build_jdk9
+    - build_jdk10
+    - build_jdk11