You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2017/01/14 04:11:34 UTC

[1/3] drill git commit: DRILL-4558: BSonReader should prepare buffer size as actual need

Repository: drill
Updated Branches:
  refs/heads/master 34969583b -> 8a4d7a994


DRILL-4558: BSonReader should prepare buffer size as actual need

This closes #696


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

Branch: refs/heads/master
Commit: 1840e49ee1d993a91790cf6ee6ed9a8c570971b6
Parents: 3496958
Author: chunhui-shi <cs...@maprtech.com>
Authored: Tue Dec 13 18:00:49 2016 -0800
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jan 13 20:06:58 2017 -0800

----------------------------------------------------------------------
 .../apache/drill/exec/store/bson/BsonRecordReader.java   | 11 +++++++----
 .../drill/exec/store/bson/TestBsonRecordReader.java      | 11 +++++++++++
 2 files changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1840e49e/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
index 36462b6..c0b6726 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/bson/BsonRecordReader.java
@@ -258,14 +258,17 @@ public class BsonRecordReader {
   }
 
   private void writeString(String readString, final MapOrListWriterImpl writer, String fieldName, boolean isList) {
-    final int length = readString.length();
-    final VarCharHolder vh = new VarCharHolder();
-    ensure(length);
+    int length;
+    byte[] strBytes;
     try {
-      workBuf.setBytes(0, readString.getBytes("UTF-8"));
+      strBytes = readString.getBytes("UTF-8");
     } catch (UnsupportedEncodingException e) {
       throw new DrillRuntimeException("Unable to read string value for field: " + fieldName, e);
     }
+    length = strBytes.length;
+    ensure(length);
+    workBuf.setBytes(0, strBytes);
+    final VarCharHolder vh = new VarCharHolder();
     vh.buffer = workBuf;
     vh.start = 0;
     vh.end = length;

http://git-wip-us.apache.org/repos/asf/drill/blob/1840e49e/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
index 4255924..1919184 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/bson/TestBsonRecordReader.java
@@ -103,6 +103,17 @@ public class TestBsonRecordReader extends BaseTestQuery {
   }
 
   @Test
+  public void testSpecialCharStringType() throws IOException {
+    BsonDocument bsonDoc = new BsonDocument();
+    bsonDoc.append("stringKey", new BsonString("���������1"));
+    writer.reset();
+    bsonReader.write(writer, new BsonDocumentReader(bsonDoc));
+    SingleMapReaderImpl mapReader = (SingleMapReaderImpl) writer.getMapVector().getReader();
+    assertEquals("���������1",
+        mapReader.reader("stringKey").readText().toString());
+  }
+
+  @Test
   public void testObjectIdType() throws IOException {
     BsonDocument bsonDoc = new BsonDocument();
     BsonObjectId value = new BsonObjectId(new ObjectId());


[2/3] drill git commit: DRILL-4868: fix how hive function set DrillBuf.

Posted by pa...@apache.org.
DRILL-4868: fix how hive function set DrillBuf.

This closes #695


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

Branch: refs/heads/master
Commit: a09343333c87fb21eaf4dd480204fa4731741f1a
Parents: 1840e49
Author: chunhui-shi <cs...@maprtech.com>
Authored: Tue Dec 13 15:16:40 2016 -0800
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jan 13 20:07:17 2017 -0800

----------------------------------------------------------------------
 .../templates/ObjectInspectorHelper.java        | 57 ++++++---------
 .../drill/exec/fn/hive/TestInbuiltHiveUDFs.java | 34 +++++++++
 .../java/org/apache/drill/QueryTestUtil.java    | 53 ++++++++++++++
 .../physical/impl/TestConvertFunctions.java     | 74 +++-----------------
 4 files changed, 120 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a0934333/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
index d068868..5d14f81 100644
--- a/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
+++ b/contrib/storage-hive/core/src/main/codegen/templates/ObjectInspectorHelper.java
@@ -172,48 +172,35 @@ public class ObjectInspectorHelper {
             booleanJC._then().assign(returnValueHolder.ref("value"), JExpr.lit(1));
             booleanJC._else().assign(returnValueHolder.ref("value"), JExpr.lit(0));
 
-          <#elseif entry.hiveType == "VARCHAR">
-            JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
-              castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
+          <#elseif entry.hiveType == "VARCHAR" || entry.hiveType == "CHAR" || entry.hiveType == "STRING" || entry.hiveType == "BINARY">
+            <#if entry.hiveType == "VARCHAR">
+              JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
+                  castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
                       .invoke("getValue")
                       .invoke("getBytes"));
-
-            jc._else().add(returnValueHolder.ref("buffer")
-              .invoke("setBytes").arg(JExpr.lit(0)).arg(data));
-
-
-            jc._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));
-            jc._else().assign(returnValueHolder.ref("end"), data.ref("length"));
-
             <#elseif entry.hiveType == "CHAR">
                 JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
-                castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
-                    .invoke("getStrippedValue")
-                    .invoke("getBytes"));
-
-            jc._else().add(returnValueHolder.ref("buffer")
-                .invoke("setBytes").arg(JExpr.lit(0)).arg(data));
-
-
-            jc._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));
-            jc._else().assign(returnValueHolder.ref("end"), data.ref("length"));
-
-          <#elseif entry.hiveType == "STRING">
-            JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
-              castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
+                    castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
+                        .invoke("getStrippedValue")
+                        .invoke("getBytes"));
+            <#elseif entry.hiveType == "STRING">
+              JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
+                  castedOI.invoke("getPrimitiveJavaObject").arg(returnValue)
                       .invoke("getBytes"));
-            jc._else().add(returnValueHolder.ref("buffer")
-              .invoke("setBytes").arg(JExpr.lit(0)).arg(data));
-            jc._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));
-            jc._else().assign(returnValueHolder.ref("end"), data.ref("length"));
-          <#elseif entry.hiveType == "BINARY">
+            <#elseif entry.hiveType == "BINARY">
+                JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
+                    castedOI.invoke("getPrimitiveJavaObject").arg(returnValue));
+            </#if>
 
-            JVar data = jc._else().decl(m.directClass(byte[].class.getCanonicalName()), "data",
-              castedOI.invoke("getPrimitiveJavaObject").arg(returnValue));
-            jc._else().add(returnValueHolder.ref("buffer")
+            JConditional jnullif = jc._else()._if(data.eq(JExpr._null()));
+            jnullif._then().assign(returnValueHolder.ref("isSet"), JExpr.lit(0));
+
+            jnullif._else().add(returnValueHolder.ref("buffer")
                 .invoke("setBytes").arg(JExpr.lit(0)).arg(data));
-            jc._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));
-            jc._else().assign(returnValueHolder.ref("end"), data.ref("length"));
+            jnullif._else().assign(returnValueHolder.ref("start"), JExpr.lit(0));
+            jnullif._else().assign(returnValueHolder.ref("end"), data.ref("length"));
+            jnullif._else().add(returnValueHolder.ref("buffer").invoke("setIndex").arg(JExpr.lit(0)).arg(data.ref("length")));
+
           <#elseif entry.hiveType == "TIMESTAMP">
             JVar tsVar = jc._else().decl(m.directClass(java.sql.Timestamp.class.getCanonicalName()), "ts",
               castedOI.invoke("getPrimitiveJavaObject").arg(returnValue));

http://git-wip-us.apache.org/repos/asf/drill/blob/a0934333/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
index 0eb4116..9ca2dbd 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java
@@ -19,10 +19,13 @@ package org.apache.drill.exec.fn.hive;
 
 import com.google.common.collect.Lists;
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.drill.QueryTestUtil;
 import org.apache.drill.TestBuilder;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.hive.HiveTestBase;
+import org.apache.drill.exec.server.options.OptionValue;
 import org.junit.Test;
 
 import java.util.List;
@@ -104,4 +107,35 @@ public class TestInbuiltHiveUDFs extends HiveTestBase {
             .baselineValues(false)
             .go();
   }
+
+  @Test //DRILL-4868
+  public void testEmbeddedHiveFunctionCall() throws Exception {
+    // TODO(DRILL-2326) temporary until we fix the scalar replacement bug for this case
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ClassTransformer.ScalarReplacementOption.TRY);
+
+    try {
+      final String[] queries = {
+          "SELECT convert_from(unhex(key2), 'INT_BE') as intkey \n" +
+              "FROM cp.`functions/conv/conv.json`",
+      };
+
+      for (String query: queries) {
+        testBuilder()
+            .sqlQuery(query)
+            .ordered()
+            .baselineColumns("intkey")
+            .baselineValues(1244739896)
+            .baselineValues(new Object[] { null })
+            .baselineValues(1313814865)
+            .baselineValues(1852782897)
+            .build()
+            .run();
+      }
+
+    } finally {
+      // restore the system option
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a0934333/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
index 1844c32..26bb4d0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/QueryTestUtil.java
@@ -29,13 +29,18 @@ import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.client.PrintingResultsListener;
 import org.apache.drill.exec.client.QuerySubmitter.Format;
+import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.proto.UserBitShared.QueryType;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.AwaitableUserResultsListener;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.server.options.OptionManager;
+import org.apache.drill.exec.server.options.OptionValue;
 import org.apache.drill.exec.util.VectorUtil;
 
 /**
@@ -163,4 +168,52 @@ public class QueryTestUtil {
     final String query = QueryTestUtil.normalizeQuery(queryString);
     drillClient.runQuery(type, query, resultListener);
   }
+
+  /**
+   * Set up the options to test the scalar replacement retry option (see
+   * ClassTransformer.java). Scalar replacement rewrites bytecode to replace
+   * value holders (essentially boxed values) with their member variables as
+   * locals. There is still one pattern that doesn't work, and occasionally new
+   * ones are introduced. This can be used in tests that exercise failing patterns.
+   *
+   * <p>This also flushes the compiled code cache.
+   *
+   * @param drillbit the drillbit
+   * @param srOption the scalar replacement option value to use
+   * @return the original scalar replacement option setting (so it can be restored)
+   */
+  public static OptionValue setupScalarReplacementOption(
+      final Drillbit drillbit, final ClassTransformer.ScalarReplacementOption srOption) {
+    // set the system option
+    final DrillbitContext drillbitContext = drillbit.getContext();
+    final OptionManager optionManager = drillbitContext.getOptionManager();
+    final OptionValue originalOptionValue = optionManager.getOption(ClassTransformer.SCALAR_REPLACEMENT_OPTION);
+    final OptionValue newOptionValue = OptionValue.createString(OptionValue.OptionType.SYSTEM,
+        ClassTransformer.SCALAR_REPLACEMENT_OPTION, srOption.name().toLowerCase());
+    optionManager.setOption(newOptionValue);
+
+    // flush the code cache
+    drillbitContext.getCompiler().flushCache();
+
+    return originalOptionValue;
+  }
+
+  /**
+   * Restore the original scalar replacement option returned from
+   * setupScalarReplacementOption().
+   *
+   * <p>This also flushes the compiled code cache.
+   *
+   * @param drillbit the drillbit
+   * @param srOption the scalar replacement option value to use
+   */
+  public static void restoreScalarReplacementOption(final Drillbit drillbit, final OptionValue srOption) {
+    final DrillbitContext drillbitContext = drillbit.getContext();
+    final OptionManager optionManager = drillbitContext.getOptionManager();
+    optionManager.setOption(srOption);
+
+    // flush the code cache
+    drillbitContext.getCompiler().flushCache();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a0934333/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
index 7d55b2a..16dd0ab 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestConvertFunctions.java
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.drill.BaseTestQuery;
+import org.apache.drill.QueryTestUtil;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.compile.ClassTransformer;
 import org.apache.drill.exec.compile.CodeCompiler;
@@ -39,11 +40,8 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
-import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.server.options.OptionManager;
 import org.apache.drill.exec.server.options.OptionValue;
-import org.apache.drill.exec.server.options.OptionValue.OptionType;
 import org.apache.drill.exec.util.ByteBufUtil.HadoopWritables;
 import org.apache.drill.exec.util.VectorUtil;
 import org.apache.drill.exec.vector.ValueVector;
@@ -95,7 +93,7 @@ public class TestConvertFunctions extends BaseTestQuery {
 
   @Test // DRILL-3854
   public void testConvertFromConvertToInt() throws Exception {
-    final OptionValue srOption = setupScalarReplacementOption(bits[0], ScalarReplacementOption.OFF);
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ScalarReplacementOption.OFF);
     try {
       final String newTblName = "testConvertFromConvertToInt_tbl";
       final String ctasQuery = String.format("CREATE TABLE %s.%s as \n" +
@@ -122,7 +120,7 @@ public class TestConvertFunctions extends BaseTestQuery {
           .run();
     } finally {
       // restore the system option
-      restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
       test("alter session set `planner.slice_target` = " + ExecConstants.SLICE_TARGET_DEFAULT);
     }
   }
@@ -541,73 +539,23 @@ public class TestConvertFunctions extends BaseTestQuery {
     assertTrue(count == 10);
   }
 
-  /**
-   * Set up the options to test the scalar replacement retry option (see
-   * ClassTransformer.java). Scalar replacement rewrites bytecode to replace
-   * value holders (essentially boxed values) with their member variables as
-   * locals. There is still one pattern that doesn't work, and occasionally new
-   * ones are introduced. This can be used in tests that exercise failing patterns.
-   *
-   * <p>This also flushes the compiled code cache.
-   *
-   * <p>TODO this should get moved to QueryTestUtil once DRILL-2245 has been merged
-   *
-   * @param drillbit the drillbit
-   * @param srOption the scalar replacement option value to use
-   * @return the original scalar replacement option setting (so it can be restored)
-   */
-  private static OptionValue setupScalarReplacementOption(
-      final Drillbit drillbit, final ScalarReplacementOption srOption) {
-    // set the system option
-    final DrillbitContext drillbitContext = drillbit.getContext();
-    final OptionManager optionManager = drillbitContext.getOptionManager();
-    final OptionValue originalOptionValue = optionManager.getOption(ClassTransformer.SCALAR_REPLACEMENT_OPTION);
-    final OptionValue newOptionValue = OptionValue.createString(OptionType.SYSTEM,
-        ClassTransformer.SCALAR_REPLACEMENT_OPTION, srOption.name().toLowerCase());
-    optionManager.setOption(newOptionValue);
-
-    // flush the code cache
-    drillbitContext.getCompiler().flushCache();
-
-    return originalOptionValue;
-  }
-
-  /**
-   * Restore the original scalar replacement option returned from
-   * setupScalarReplacementOption().
-   *
-   * <p>This also flushes the compiled code cache.
-   *
-   * <p>TODO this should get moved to QueryTestUtil once DRILL-2245 has been merged
-   *
-   * @param drillbit the drillbit
-   * @param srOption the scalar replacement option value to use
-   */
-  private static void restoreScalarReplacementOption(final Drillbit drillbit, final OptionValue srOption) {
-    final DrillbitContext drillbitContext = drillbit.getContext();
-    final OptionManager optionManager = drillbitContext.getOptionManager();
-    optionManager.setOption(srOption);
-
-    // flush the code cache
-    drillbitContext.getCompiler().flushCache();
-  }
 
   @Test // TODO(DRILL-2326) temporary until we fix the scalar replacement bug for this case
   public void testBigIntVarCharReturnTripConvertLogical_ScalarReplaceTRY() throws Exception {
-    final OptionValue srOption = setupScalarReplacementOption(bits[0], ScalarReplacementOption.TRY);
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ScalarReplacementOption.TRY);
     try {
       // this should work fine
       testBigIntVarCharReturnTripConvertLogical();
     } finally {
       // restore the system option
-      restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
     }
   }
 
   @Test // TODO(DRILL-2326) temporary until we fix the scalar replacement bug for this case
   @Ignore // Because this test sometimes fails, sometimes succeeds
   public void testBigIntVarCharReturnTripConvertLogical_ScalarReplaceON() throws Exception {
-    final OptionValue srOption = setupScalarReplacementOption(bits[0], ScalarReplacementOption.ON);
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ScalarReplacementOption.ON);
     boolean caughtException = false;
     try {
       // this used to fail (with a JUnit assertion) until we fix the SR bug
@@ -617,7 +565,7 @@ public class TestConvertFunctions extends BaseTestQuery {
     } catch(RpcException e) {
       caughtException = true;
     } finally {
-      restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
     }
 
     // Yes: sometimes this works, sometimes it does not...
@@ -626,13 +574,13 @@ public class TestConvertFunctions extends BaseTestQuery {
 
   @Test // TODO(DRILL-2326) temporary until we fix the scalar replacement bug for this case
   public void testBigIntVarCharReturnTripConvertLogical_ScalarReplaceOFF() throws Exception {
-    final OptionValue srOption = setupScalarReplacementOption(bits[0], ScalarReplacementOption.OFF);
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ScalarReplacementOption.OFF);
     try {
       // this should work fine
       testBigIntVarCharReturnTripConvertLogical();
     } finally {
       // restore the system option
-      restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
     }
   }
 
@@ -679,7 +627,7 @@ public class TestConvertFunctions extends BaseTestQuery {
   @Test // DRILL-4862
   public void testBinaryString() throws Exception {
     // TODO(DRILL-2326) temporary until we fix the scalar replacement bug for this case
-    final OptionValue srOption = setupScalarReplacementOption(bits[0], ScalarReplacementOption.TRY);
+    final OptionValue srOption = QueryTestUtil.setupScalarReplacementOption(bits[0], ScalarReplacementOption.TRY);
 
     try {
       final String[] queries = {
@@ -702,7 +650,7 @@ public class TestConvertFunctions extends BaseTestQuery {
 
     } finally {
       // restore the system option
-      restoreScalarReplacementOption(bits[0], srOption);
+      QueryTestUtil.restoreScalarReplacementOption(bits[0], srOption);
     }
   }
 


[3/3] drill git commit: DRILL-5172: Display elapsed time for queries in the UI

Posted by pa...@apache.org.
DRILL-5172: Display elapsed time for queries in the UI

Displays the elapsed time for running queries and the total duration of completed/failed/cancelled queries in the list of query profiles displayed, and within a query's profile page as well.
The query runtime is  displayed in '[hr] [min] sec'.
e.g. A duration of 25,254,321ms is displayed  7 hr 00 min 54.321 sec

This closes #721


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

Branch: refs/heads/master
Commit: 8a4d7a9940499a8c972362ab1133d75f211ec786
Parents: a093433
Author: Kunal Khatua <kk...@maprtech.com>
Authored: Wed Jan 11 16:45:15 2017 -0800
Committer: Parth Chandra <pa...@apache.org>
Committed: Fri Jan 13 20:07:45 2017 -0800

----------------------------------------------------------------------
 .../server/rest/profile/ProfileResources.java   | 42 ++++++++++++++++++--
 .../server/rest/profile/ProfileWrapper.java     |  4 ++
 .../src/main/resources/rest/profile/list.ftl    |  9 ++++-
 .../src/main/resources/rest/profile/profile.ftl |  1 +
 4 files changed, 51 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8a4d7a99/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 d2e953d..24d37b2 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
@@ -68,10 +68,31 @@ public class ProfileResources {
   @Inject DrillUserPrincipal principal;
   @Inject SecurityContext sc;
 
+  /**
+   * Returns elapsed time a human-readable format. If end time is less than the start time, current epoch time is assumed as the end time.
+   * e.g. getPrettyDuration(1468368841695,1468394096016) = '7 hr 00 min 54.321 sec'
+   * @param startTimeMillis Start Time in milliseconds
+   * @param endTimeMillis   End Time in milliseconds
+   * @return                Human-Readable Elapsed Time
+   */
+  public static String getPrettyDuration(long startTimeMillis, long endTimeMillis) {
+    long durationInMillis = (startTimeMillis > endTimeMillis ? System.currentTimeMillis() : endTimeMillis) - startTimeMillis;
+    long hours = TimeUnit.MILLISECONDS.toHours(durationInMillis);
+    long minutes = TimeUnit.MILLISECONDS.toMinutes(durationInMillis) - TimeUnit.HOURS.toMinutes(TimeUnit.MILLISECONDS.toHours(durationInMillis));
+    long seconds = TimeUnit.MILLISECONDS.toSeconds(durationInMillis) - TimeUnit.MINUTES.toSeconds(TimeUnit.MILLISECONDS.toMinutes(durationInMillis));
+    long milliSeconds = durationInMillis - TimeUnit.SECONDS.toMillis(TimeUnit.MILLISECONDS.toSeconds(durationInMillis));
+    String formattedDuration = (hours > 0 ? hours + " hr " : "") +
+      ((minutes + hours) > 0 ? String.format("%02d min ", minutes) : "") +
+      seconds + "." + String.format("%03d sec", milliSeconds) ;
+    return formattedDuration;
+  }
+
   public static class ProfileInfo implements Comparable<ProfileInfo> {
     public static final SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
 
     private String queryId;
+    private long startTime;
+    private long endTime;
     private Date time;
     private String location;
     private String foreman;
@@ -79,9 +100,11 @@ public class ProfileResources {
     private String state;
     private String user;
 
-    public ProfileInfo(String queryId, long time, String foreman, String query, String state, String user) {
+    public ProfileInfo(String queryId, long startTime, long endTime, String foreman, String query, String state, String user) {
       this.queryId = queryId;
-      this.time = new Date(time);
+      this.startTime = startTime;
+      this.endTime = endTime;
+      this.time = new Date(startTime);
       this.foreman = foreman;
       this.location = "http://localhost:8047/profile/" + queryId + ".json";
       this.query = query.substring(0,  Math.min(query.length(), 150));
@@ -105,6 +128,17 @@ public class ProfileResources {
       return format.format(time);
     }
 
+    public long getStartTime() {
+      return startTime;
+    }
+
+    public long getEndTime() {
+      return endTime;
+    }
+
+    public String getDuration() {
+      return getPrettyDuration(startTime, endTime);
+    }
 
     public String getState() {
       return state;
@@ -174,7 +208,7 @@ public class ProfileResources {
           final Map.Entry<String, QueryInfo> runningEntry = runningEntries.next();
           final QueryInfo profile = runningEntry.getValue();
           if (principal.canManageProfileOf(profile.getUser())) {
-            runningQueries.add(new ProfileInfo(runningEntry.getKey(), profile.getStart(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name(), profile.getUser()));
+            runningQueries.add(new ProfileInfo(runningEntry.getKey(), profile.getStart(), System.currentTimeMillis(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name(), profile.getUser()));
           }
         } catch (Exception e) {
           errors.add(e.getMessage());
@@ -192,7 +226,7 @@ public class ProfileResources {
           final Map.Entry<String, QueryProfile> profileEntry = range.next();
           final QueryProfile profile = profileEntry.getValue();
           if (principal.canManageProfileOf(profile.getUser())) {
-            finishedQueries.add(new ProfileInfo(profileEntry.getKey(), profile.getStart(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name(), profile.getUser()));
+            finishedQueries.add(new ProfileInfo(profileEntry.getKey(), profile.getStart(), profile.getEnd(), profile.getForeman().getAddress(), profile.getQuery(), profile.getState().name(), profile.getUser()));
           }
         } catch (Exception e) {
           errors.add(e.getMessage());

http://git-wip-us.apache.org/repos/asf/drill/blob/8a4d7a99/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
index d9edf3a..57223f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/ProfileWrapper.java
@@ -114,6 +114,10 @@ public class ProfileWrapper {
     return profile;
   }
 
+  public String getProfileDuration() {
+    return ProfileResources.getPrettyDuration(profile.getStart(), profile.getEnd());
+  }
+
   public String getQueryId() {
     return id;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/8a4d7a99/exec/java-exec/src/main/resources/rest/profile/list.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/list.ftl b/exec/java-exec/src/main/resources/rest/profile/list.ftl
index 88d1407..1fcffb6 100644
--- a/exec/java-exec/src/main/resources/rest/profile/list.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/list.ftl
@@ -36,6 +36,7 @@
            <td>User</td>
            <td>Query</td>
            <td>State</td>
+           <td>Elapsed</td>
            <td>Foreman</td>
         </thead>
         <tbody>
@@ -62,7 +63,9 @@
               </a>
             </td> 
             <td>
-              <div style="height:100%;width:100%">${query.getState()}</div>          
+              <div style="height:100%;width:100%">${query.getState()}</div>
+            <td>
+              <div style="height:100%;width:100%">${query.getDuration()}</div>
             <td>
                 <div style="height:100%;width:100%">
                   ${query.getForeman()}
@@ -91,6 +94,7 @@
          <!-- <td>Query Id</td> -->
          <td>Query</td>
          <td>State</td>
+         <td>Duration</td>
          <td>Foreman</td>
       </thead>
       <tbody>
@@ -121,6 +125,9 @@
               <div style="height:100%;width:100%">${query.getState()}</div>
           </td>
           <td>
+              <div style="height:100%;width:100%">${query.getDuration()}</div>
+          </td>
+          <td>
               <div style="height:100%;width:100%">
                 ${query.getForeman()}
               </div>

http://git-wip-us.apache.org/repos/asf/drill/blob/8a4d7a99/exec/java-exec/src/main/resources/rest/profile/profile.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/resources/rest/profile/profile.ftl b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
index 792739f..e9a8632 100644
--- a/exec/java-exec/src/main/resources/rest/profile/profile.ftl
+++ b/exec/java-exec/src/main/resources/rest/profile/profile.ftl
@@ -106,6 +106,7 @@
   <p>STATE: ${model.getProfile().getState().name()}</p>
   <p>FOREMAN: ${model.getProfile().getForeman().getAddress()}</p>
   <p>TOTAL FRAGMENTS: ${model.getProfile().getTotalFragments()}</p>
+  <p>DURATION: ${model.getProfileDuration()}</p>
 
   <#assign options = model.getOptions()>
   <#if (options?keys?size > 0)>