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 2017/04/17 05:24:06 UTC

[1/4] drill git commit: DRILL-5213: Prepared statement for actual query is missing the query text

Repository: drill
Updated Branches:
  refs/heads/master 9df3403a0 -> 72903d014


DRILL-5213: Prepared statement for actual query is missing the query text

close apache/drill#812


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/dfb5f4bb
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/dfb5f4bb
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/dfb5f4bb

Branch: refs/heads/master
Commit: dfb5f4bb5ac771ea64431b8482141ded02d8d859
Parents: 9df3403
Author: Vitalii Diravka <vi...@gmail.com>
Authored: Mon Apr 10 18:54:21 2017 +0000
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Apr 16 22:20:05 2017 -0700

----------------------------------------------------------------------
 .../server/rest/profile/ProfileResources.java   |  4 +++-
 .../apache/drill/exec/work/foreman/Foreman.java | 15 +++++++++----
 .../drill/exec/work/foreman/QueryManager.java   | 22 +++++++++++++-------
 3 files changed, 29 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/dfb5f4bb/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
index e1f2099..044b792 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileResources.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -227,6 +227,8 @@ public class ProfileResources {
         }
       }
 
+      Collections.sort(finishedQueries, Collections.reverseOrder());
+
       return new QProfiles(runningQueries, finishedQueries, errors);
     } catch (Exception e) {
       throw UserException.resourceError(e)

http://git-wip-us.apache.org/repos/asf/drill/blob/dfb5f4bb/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 94d0658..a2b09a8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -35,7 +35,6 @@ import java.util.concurrent.TimeUnit;
 import org.apache.drill.common.CatastrophicFailure;
 import org.apache.drill.common.EventProcessor;
 import org.apache.drill.common.concurrent.ExtendedLatch;
-import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.common.logical.LogicalPlan;
@@ -537,9 +536,10 @@ public class Foreman implements Runnable {
           .build(logger);
     }
 
-    final String sql = serverState.getSqlQuery();
-    logger.info("Prepared statement query for QueryId {} : {}", queryId, sql);
-    runSQL(sql);
+    queryText = serverState.getSqlQuery();
+    logger.info("Prepared statement query for QueryId {} : {}", queryId, queryText);
+    runSQL(queryText);
+
   }
 
   private static void validatePlan(final PhysicalPlan plan) throws ForemanSetupException {
@@ -1222,6 +1222,13 @@ public class Foreman implements Runnable {
   }
 
   /**
+   * @return sql query text of the query request
+   */
+  public String getQueryText() {
+    return queryText;
+  }
+
+  /**
    * Used by {@link FragmentSubmitListener} to track the number of submission failures.
    */
   private static class FragmentSubmitFailures {

http://git-wip-us.apache.org/repos/asf/drill/blob/dfb5f4bb/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index 7305025..77c20a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -18,7 +18,6 @@
 package org.apache.drill.exec.work.foreman;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import io.netty.buffer.ByteBuf;
 
 import java.util.List;
@@ -329,14 +328,19 @@ public class QueryManager implements AutoCloseable {
   }
 
   private QueryInfo getQueryInfo() {
-    return QueryInfo.newBuilder()
-        .setQuery(runQuery.getPlan())
+    final String queryText = foreman.getQueryText();
+    QueryInfo.Builder queryInfoBuilder = QueryInfo.newBuilder()
         .setState(foreman.getState())
         .setUser(foreman.getQueryContext().getQueryUserName())
         .setForeman(foreman.getQueryContext().getCurrentEndpoint())
         .setStart(startTime)
-        .setOptionsJson(getQueryOptionsAsJson())
-        .build();
+        .setOptionsJson(getQueryOptionsAsJson());
+
+    if (queryText != null) {
+      queryInfoBuilder.setQuery(queryText);
+    }
+
+    return queryInfoBuilder.build();
   }
 
   public QueryProfile getQueryProfile() {
@@ -344,8 +348,8 @@ public class QueryManager implements AutoCloseable {
   }
 
   private QueryProfile getQueryProfile(UserException ex) {
+    final String queryText = foreman.getQueryText();
     final QueryProfile.Builder profileBuilder = QueryProfile.newBuilder()
-        .setQuery(runQuery.getPlan())
         .setUser(foreman.getQueryContext().getQueryUserName())
         .setType(runQuery.getType())
         .setId(queryId)
@@ -372,6 +376,10 @@ public class QueryManager implements AutoCloseable {
       profileBuilder.setPlan(planText);
     }
 
+    if (queryText != null) {
+      profileBuilder.setQuery(queryText);
+    }
+
     fragmentDataMap.forEach(new OuterIter(profileBuilder));
 
     return profileBuilder.build();


[4/4] drill git commit: DRILL-5424: Fix IOBE for reverse function

Posted by am...@apache.org.
DRILL-5424: Fix IOBE for reverse function

close apache/drill#815


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/72903d01
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/72903d01
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/72903d01

Branch: refs/heads/master
Commit: 72903d01424139057d4309ce6655e0aecee2573e
Parents: 3b71cbd
Author: Volodymyr Vysotskyi <vv...@gmail.com>
Authored: Mon Apr 10 13:16:52 2017 +0000
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Apr 16 22:20:31 2017 -0700

----------------------------------------------------------------------
 .../exec/expr/fn/impl/StringFunctions.java      | 20 +++++------
 .../exec/expr/fn/impl/TestStringFunctions.java  | 37 +++++++++++++++++++-
 2 files changed, 46 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/72903d01/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
index 8196728..d90581e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -1698,20 +1698,20 @@ public class StringFunctions{
       out.start = 0;
       out.end = len;
       out.buffer = buffer = buffer.reallocIfNeeded(len);
-      int charlen = 0;
+      int charLen;
 
-      int index = in.end;
-      int innerindex = 0;
+      int index = out.end;
+      int innerIndex;
 
-      for (int id = in.start; id < in.end; id += charlen) {
-        innerindex = charlen = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.utf8CharLen(in.buffer, id);
+      for (int id = in.start; id < in.end; id += charLen) {
+        innerIndex = charLen = org.apache.drill.exec.expr.fn.impl.StringFunctionUtil.utf8CharLen(in.buffer, id);
 
-        while (innerindex > 0) {
-          out.buffer.setByte(index - innerindex, in.buffer.getByte(id + (charlen - innerindex)));
-          innerindex-- ;
+        while (innerIndex > 0) {
+          out.buffer.setByte(index - innerIndex, in.buffer.getByte(id + (charLen - innerIndex)));
+          innerIndex--;
         }
 
-        index -= charlen;
+        index -= charLen;
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/72903d01/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
index daedd1c..fe099d7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -19,12 +19,17 @@ package org.apache.drill.exec.expr.fn.impl;
 
 import static org.junit.Assert.assertTrue;
 
+import org.apache.commons.io.FileUtils;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.exec.util.Text;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
 
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+
 public class TestStringFunctions extends BaseTestQuery {
 
   @Test
@@ -273,4 +278,34 @@ public class TestStringFunctions extends BaseTestQuery {
         .run();
   }
 
+  @Test
+  public void testReverse() throws Exception {
+    testBuilder()
+      .sqlQuery("select reverse('qwerty') words from (values(1))")
+      .unOrdered()
+      .baselineColumns("words")
+      .baselineValues("ytrewq")
+      .build()
+      .run();
+  }
+
+  @Test // DRILL-5424
+  public void testReverseLongVarChars() throws Exception {
+    File path = new File(BaseTestQuery.getTempDir("input"));
+    try {
+      path.mkdirs();
+      String pathString = path.toPath().toString();
+
+      try (BufferedWriter writer = new BufferedWriter(new FileWriter(new File(path, "table_with_long_varchars.json")))) {
+        for (int i = 0; i < 10; i++) {
+          writer.write("{ \"a\": \"abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz\"}");
+        }
+      }
+
+      test("select reverse(a) from dfs_test.`%s/table_with_long_varchars.json` t", pathString);
+
+    } finally {
+      FileUtils.deleteQuietly(path);
+    }
+  }
 }


[2/4] drill git commit: DRILL-5409 - update MapR version to 5.2.1

Posted by am...@apache.org.
DRILL-5409 - update MapR version to 5.2.1

close apache/drill#813


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/df81e567
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/df81e567
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/df81e567

Branch: refs/heads/master
Commit: df81e5677e2d64b8bf977580e3539c0637431335
Parents: dfb5f4b
Author: Patrick Wong <pw...@maprtech.com>
Authored: Mon Apr 10 15:45:14 2017 -0700
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Apr 16 22:20:30 2017 -0700

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/df81e567/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5bde19b..9a7ed29 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,7 +39,7 @@
     <janino.version>2.7.6</janino.version>
     <sqlline.version>1.1.9-drill-r7</sqlline.version>
     <jackson.version>2.7.1</jackson.version>
-    <mapr.release.version>5.2.0.40963-mapr</mapr.release.version>
+    <mapr.release.version>5.2.1-mapr</mapr.release.version>
     <ojai.version>1.1</ojai.version>
     <kerby.version>1.0.0-RC2</kerby.version>
 


[3/4] drill git commit: DRILL-5415: Improve Fixture Builder to configure client properties and keep collection type properties for server

Posted by am...@apache.org.
DRILL-5415: Improve Fixture Builder to configure client properties and keep collection type properties for server

Updated with review feedback

close apache/drill#807


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/3b71cbdb
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/3b71cbdb
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/3b71cbdb

Branch: refs/heads/master
Commit: 3b71cbdbe74f1c5462f488524427888ceadb3392
Parents: df81e56
Author: Sorabh Hamirwasia <sh...@maprtech.com>
Authored: Wed Apr 5 11:04:58 2017 -0700
Committer: Aman Sinha <as...@maprtech.com>
Committed: Sun Apr 16 22:20:31 2017 -0700

----------------------------------------------------------------------
 .../org/apache/drill/test/ClientFixture.java    |  1 +
 .../org/apache/drill/test/ClusterFixture.java   | 47 +++++++++++++++++---
 .../org/apache/drill/test/FixtureBuilder.java   | 19 +++++++-
 3 files changed, 60 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3b71cbdb/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
index b83d5fc..25dab4f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClientFixture.java
@@ -43,6 +43,7 @@ public class ClientFixture implements AutoCloseable {
 
     protected ClientBuilder(ClusterFixture cluster) {
       this.cluster = cluster;
+      clientProps = cluster.getClientProps();
     }
     /**
      * Specify an optional client property.

http://git-wip-us.apache.org/repos/asf/drill/blob/3b71cbdb/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
index b920edb..90ce206 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/ClusterFixture.java
@@ -28,6 +28,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
 import org.apache.commons.io.FileUtils;
 import org.apache.drill.BaseTestQuery;
 import org.apache.drill.DrillTestWrapper.TestServices;
@@ -133,6 +135,7 @@ public class ClusterFixture implements AutoCloseable {
   private boolean usesZk;
   private boolean preserveLocalFiles;
   private boolean isLocal;
+  private Properties clientProps;
 
   /**
    * Temporary directories created for this test cluster.
@@ -143,6 +146,7 @@ public class ClusterFixture implements AutoCloseable {
 
   ClusterFixture(FixtureBuilder builder) {
 
+    setClientProps(builder);
     configureZk(builder);
     try {
       createConfig(builder);
@@ -157,6 +161,18 @@ public class ClusterFixture implements AutoCloseable {
     }
   }
 
+  /**
+   * Set the client properties to be used by client fixture.
+   * @param builder {@link FixtureBuilder#clientProps}
+   */
+  private void setClientProps(FixtureBuilder builder) {
+      clientProps = builder.clientProps;
+  }
+
+  public Properties getClientProps() {
+    return clientProps;
+  }
+
   private void configureZk(FixtureBuilder builder) {
 
     // Start ZK if requested.
@@ -203,7 +219,7 @@ public class ClusterFixture implements AutoCloseable {
     if (builder.configResource != null) {
       config = DrillConfig.create(builder.configResource);
     } else if (builder.configProps != null) {
-      config = DrillConfig.create(configProperties(builder.configProps));
+      config = configProperties(builder.configProps);
     } else {
       throw new IllegalStateException("Configuration was not provided.");
     }
@@ -325,12 +341,31 @@ public class ClusterFixture implements AutoCloseable {
     }
   }
 
-  private Properties configProperties(Properties configProps) {
-    Properties effectiveProps = new Properties();
-    for (Entry<Object, Object> entry : configProps.entrySet()) {
-      effectiveProps.put(entry.getKey(), entry.getValue().toString());
+  private DrillConfig configProperties(Properties configProps) {
+    Properties stringProps = new Properties();
+    Properties collectionProps = new Properties();
+
+    // Filter out the collection type configs and other configs which can be converted to string.
+    for(Entry<Object, Object> entry : configProps.entrySet()) {
+      if(entry.getValue() instanceof Collection<?>) {
+        collectionProps.put(entry.getKey(), entry.getValue());
+      } else {
+        stringProps.setProperty(entry.getKey().toString(), entry.getValue().toString());
+      }
+    }
+
+    // First create a DrillConfig based on string properties.
+    Config drillConfig = DrillConfig.create(stringProps);
+
+    // Then add the collection properties inside the DrillConfig. Below call to withValue returns
+    // a new reference. Considering mostly properties will be of string type, doing this
+    // later will be less expensive as compared to doing it for all the properties.
+    for(Entry<Object, Object> entry : collectionProps.entrySet()) {
+      drillConfig = drillConfig.withValue(entry.getKey().toString(),
+        ConfigValueFactory.fromAnyRef(entry.getValue()));
     }
-    return effectiveProps;
+
+    return new DrillConfig(drillConfig, true);
   }
 
   public Drillbit drillbit() { return defaultDrillbit; }

http://git-wip-us.apache.org/repos/asf/drill/blob/3b71cbdb/exec/java-exec/src/test/java/org/apache/drill/test/FixtureBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/test/FixtureBuilder.java b/exec/java-exec/src/test/java/org/apache/drill/test/FixtureBuilder.java
index 461371a..f6106ff 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/test/FixtureBuilder.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/test/FixtureBuilder.java
@@ -19,6 +19,7 @@ package org.apache.drill.test;
 
 import java.io.File;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 
@@ -67,6 +68,7 @@ public class FixtureBuilder {
   protected boolean usingZk;
   protected File tempDir;
   protected boolean preserveLocalFiles;
+  protected Properties clientProps;
 
   /**
    * Use the given configuration properties to start the embedded Drillbit.
@@ -117,7 +119,22 @@ public class FixtureBuilder {
     if (configProps == null) {
       configProps = defaultProps();
     }
-    configProps.put(key, value.toString());
+    configProps.put(key, value);
+    return this;
+  }
+
+  /**
+   * Add an additional property for the client connection URL. Convert all the values into
+   * String type.
+   * @param key config property name
+   * @param value property value
+   * @return this builder
+   */
+  public FixtureBuilder configClientProperty(String key, Object value) {
+    if(clientProps == null) {
+      clientProps = new Properties();
+    }
+    clientProps.put(key, value.toString());
     return this;
   }