You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by to...@apache.org on 2018/07/12 02:47:17 UTC

[1/3] impala git commit: IMPALA-7129: Fixing path to UBSAN suppressions.

Repository: impala
Updated Branches:
  refs/heads/master 5d672457c -> e8a669bf9


IMPALA-7129: Fixing path to UBSAN suppressions.

This pins the suppression file path at build time.

It turns out that calling getenv("IMPALA_HOME") was returning null at
the time the undefined behavior was being initialized. I suspect this is
an initialization ordering issue. To step around the issue, I'm simply
using a preprocessor macro to get at the suppression file path.

I was able to reproduce the error by running the catalogd binary
outside of $IMPALA_HOME. Before this change, that would fail reliably.

Change-Id: I5295c2d1bddbea585a761b85a51eadecf10d191d
Reviewed-on: http://gerrit.cloudera.org:8080/10895
Reviewed-by: Jim Apple <jb...@apache.org>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 9b4e6d82219ed1cccfdc6e735000183b6a7f866b
Parents: 5d67245
Author: Philip Zeyliger <ph...@cloudera.com>
Authored: Mon Jul 9 14:22:27 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Thu Jul 12 01:08:13 2018 +0000

----------------------------------------------------------------------
 be/CMakeLists.txt     | 8 +++++++-
 be/src/common/init.cc | 5 +----
 2 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9b4e6d82/be/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/CMakeLists.txt b/be/CMakeLists.txt
index da78dad..faefadc 100644
--- a/be/CMakeLists.txt
+++ b/be/CMakeLists.txt
@@ -112,7 +112,13 @@ SET(CXX_FLAGS_ADDRESS_SANITIZER
 
 # Set the flags to the undefined behavior sanitizer, also known as "ubsan"
 # Turn on sanitizer and debug symbols to get stack traces:
-SET(CXX_FLAGS_UBSAN "${CXX_CLANG_FLAGS} -ggdb3 -fno-omit-frame-pointer -fsanitize=undefined -DUNDEFINED_SANITIZER")
+SET(CXX_FLAGS_UBSAN "${CXX_CLANG_FLAGS} -ggdb3 -fno-omit-frame-pointer -fsanitize=undefined")
+# Set preprocessor macros to facilitate initialization the relevant configuration.
+SET(CXX_FLAGS_UBSAN "${CXX_FLAGS_UBSAN} -DUNDEFINED_SANITIZER")
+# Calling getenv() in __ubsan_default_options doesn't work, likely because of
+# initialization ordering. We need to double-quote to create a macro that expands
+# to a string-literal.
+SET(CXX_FLAGS_UBSAN "${CXX_FLAGS_UBSAN} -DUNDEFINED_SANITIZER_SUPPRESSIONS=\\\"$ENV{IMPALA_HOME}/bin/ubsan-suppressions.txt\\\"")
 # Add flags to enable symbol resolution in the stack traces:
 SET(CXX_FLAGS_UBSAN "${CXX_FLAGS_UBSAN} -rtlib=compiler-rt -lgcc_s")
 # Ignore a number of noisy errors with too many false positives:

http://git-wip-us.apache.org/repos/asf/impala/blob/9b4e6d82/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index 30c194e..ffbb165 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -304,9 +304,6 @@ extern "C" const char *__tsan_default_options() {
 // Default UBSAN_OPTIONS. Override by setting environment variable $UBSAN_OPTIONS.
 #if defined(UNDEFINED_SANITIZER)
 extern "C" const char *__ubsan_default_options() {
-  static const string default_options = Substitute(
-      "print_stacktrace=1 suppressions=$0/bin/ubsan-suppressions.txt",
-      getenv("IMPALA_HOME") == nullptr ? "." : getenv("IMPALA_HOME"));
-  return default_options.c_str();
+  return "print_stacktrace=1 suppressions=" UNDEFINED_SANITIZER_SUPPRESSIONS;
 }
 #endif


[2/3] impala git commit: IMPALA-7140 (part 6): fetch column stats for LocalTable

Posted by to...@apache.org.
IMPALA-7140 (part 6): fetch column stats for LocalTable

This adds fetching of column statistics for LocalTable. Currently, all
column stats are fetched when the table is loaded, even for simple
statements like 'DESCRIBE' where they aren't necessary. This is because
I couldn't find a convenient spot during analysis at which time the set
of necessary columns are known. I left a TODO for this potential
improvement.

With this change I can see that 'SHOW COLUMN STATS' shows the expected
results for functional.alltypes. A new simple unit test verifies this.

Planner tests still don't pass due to some NullPointerExceptions related
to loading functions from the builtins DB -- most of the tests seem to
rely on simple built-ins like COUNT and CAST.

Change-Id: Ib6403c2bedf4ee29c5e6f90e947382cb44f46e0c
Reviewed-on: http://gerrit.cloudera.org:8080/10797
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 9df9efc5f33d2e31f218b69a6053b575a30696a6
Parents: 9b4e6d8
Author: Todd Lipcon <to...@cloudera.com>
Authored: Wed Jun 20 15:42:39 2018 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Thu Jul 12 02:36:55 2018 +0000

----------------------------------------------------------------------
 .../apache/impala/catalog/FeCatalogUtils.java   | 29 ++++++++++++++++++++
 .../java/org/apache/impala/catalog/Table.java   | 19 ++-----------
 .../catalog/local/DirectMetaProvider.java       | 10 +++++++
 .../impala/catalog/local/LocalFsTable.java      | 24 ++++++++++++++++
 .../apache/impala/catalog/local/LocalTable.java | 29 +++++++++++++++++---
 .../impala/catalog/local/MetaProvider.java      |  7 +++++
 .../impala/catalog/local/LocalCatalogTest.java  | 16 +++++++++++
 7 files changed, 114 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
index dc16629..12b152f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
@@ -24,6 +24,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.LiteralExpr;
@@ -129,6 +130,34 @@ public abstract class FeCatalogUtils {
   }
 
   /**
+   * Given the list of column stats returned from the metastore, inject those
+   * stats into matching columns in 'table'.
+   */
+  public static void injectColumnStats(List<ColumnStatisticsObj> colStats,
+      FeTable table) {
+    for (ColumnStatisticsObj stats: colStats) {
+      Column col = table.getColumn(stats.getColName());
+      Preconditions.checkNotNull(col, "Unable to find column %s in table %s",
+          stats.getColName(), table.getFullName());
+      if (!ColumnStats.isSupportedColType(col.getType())) {
+        LOG.warn(String.format(
+            "Statistics for %s, column %s are not supported as column " +
+            "has type %s", table.getFullName(), col.getName(), col.getType()));
+        continue;
+      }
+
+      if (!col.updateStats(stats.getStatsData())) {
+        LOG.warn(String.format(
+            "Failed to load column stats for %s, column %s. Stats may be " +
+            "incompatible with column type %s. Consider regenerating statistics " +
+            "for %s.", table.getFullName(), col.getName(), col.getType(),
+            table.getFullName()));
+        continue;
+      }
+    }
+  }
+
+  /**
    * Returns the value of the ROW_COUNT constant, or -1 if not found.
    */
   public static long getRowCount(Map<String, String> parameters) {

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/Table.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index 7fdc3c9..06c4500 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -214,6 +214,8 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
 
     // We need to only query those columns which may have stats; asking HMS for other
     // columns causes loadAllColumnStats() to return nothing.
+    // TODO(todd): this no longer seems to be true - asking for a non-existent column
+    // is just ignored, and the columns that do exist are returned.
     List<String> colNames = getColumnNamesWithHmsStats();
 
     try {
@@ -223,22 +225,7 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
       return;
     }
 
-    for (ColumnStatisticsObj stats: colStats) {
-      Column col = getColumn(stats.getColName());
-      Preconditions.checkNotNull(col);
-      if (!ColumnStats.isSupportedColType(col.getType())) {
-        LOG.warn(String.format("Statistics for %s, column %s are not supported as " +
-                "column has type %s", getFullName(), col.getName(), col.getType()));
-        continue;
-      }
-
-      if (!col.updateStats(stats.getStatsData())) {
-        LOG.warn(String.format("Failed to load column stats for %s, column %s. Stats " +
-            "may be incompatible with column type %s. Consider regenerating statistics " +
-            "for %s.", getFullName(), col.getName(), col.getType(), getFullName()));
-        continue;
-      }
-    }
+    FeCatalogUtils.injectColumnStats(colStats, this);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
index aadbf23..59acd9d 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hive.common.FileUtils;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -160,6 +161,15 @@ class DirectMetaProvider implements MetaProvider {
     return ret;
   }
 
+
+  @Override
+  public List<ColumnStatisticsObj> loadTableColumnStatistics(String dbName,
+      String tblName, List<String> colNames) throws TException {
+    try (MetaStoreClient c = msClientPool_.getClient()) {
+      return c.getHiveClient().getTableColumnStatistics(dbName, tblName, colNames);
+    }
+  }
+
   @Override
   public List<LocatedFileStatus> loadFileMetadata(Path dir) throws IOException {
     Preconditions.checkNotNull(dir);

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
index 6871f90..42d3b03 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
@@ -31,6 +31,7 @@ import org.apache.impala.analysis.LiteralExpr;
 import org.apache.impala.analysis.NullLiteral;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.FeCatalogUtils;
 import org.apache.impala.catalog.FeFsPartition;
 import org.apache.impala.catalog.FeFsTable;
@@ -375,6 +376,29 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
     partitionSpecs_ = b.build();
   }
 
+  /**
+   * Override base implementation to populate column stats for
+   * clustering columns based on the partition map.
+   */
+  @Override
+  protected void loadColumnStats() {
+    super.loadColumnStats();
+    // TODO(todd): this is called for all tables even if not necessary,
+    // which means we need to load all partition names, even if not
+    // necessary.
+    loadPartitionValueMap();
+    for (int i = 0; i < getNumClusteringCols(); i++) {
+      ColumnStats stats = getColumns().get(i).getStats();
+      int nonNullParts = partitionValueMap_.get(i).size();
+      int nullParts = nullPartitionIds_.get(i).size();
+      stats.setNumDistinctValues(nonNullParts + nullParts);
+      // TODO(todd): this calculation ends up setting the num_nulls stat
+      // to the number of partitions with null rows, not the number of rows.
+      // However, it maintains the existing behavior from HdfsTable.
+      stats.setNumNulls(nullParts);
+    }
+  }
+
   @Override
   public int parseSkipHeaderLineCount(StringBuilder error) {
     // TODO Auto-generated method stub

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
index a0a7bd7..ae697b5 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalTable.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import javax.annotation.concurrent.Immutable;
 
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.impala.analysis.TableName;
@@ -37,6 +38,7 @@ import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TTableStats;
+import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 
 import com.google.common.base.Preconditions;
@@ -52,6 +54,8 @@ import com.google.common.collect.Lists;
  * each catalog instance.
  */
 abstract class LocalTable implements FeTable {
+  private static final Logger LOG = Logger.getLogger(LocalTable.class);
+
   protected final LocalDb db_;
   /** The lower-case name of the table. */
   protected final String name_;
@@ -61,15 +65,22 @@ abstract class LocalTable implements FeTable {
     // In order to know which kind of table subclass to instantiate, we need
     // to eagerly grab and parse the top-level Table object from the HMS.
     SchemaInfo schemaInfo = SchemaInfo.load(db, tblName);
+    LocalTable t;
     if (HdfsFileFormat.isHdfsInputFormatClass(
         schemaInfo.msTable_.getSd().getInputFormat())) {
-      return new LocalFsTable(db, tblName, schemaInfo);
+      t = new LocalFsTable(db, tblName, schemaInfo);
+    } else {
+      throw new LocalCatalogException("Unknown table type for table " +
+          db.getName() + "." + tblName);
     }
 
-    throw new LocalCatalogException("Unknown table type for table " +
-        db.getName() + "." + tblName);
+    // TODO(todd): it would be preferable to only load stats for those columns
+    // referenced in a query, but there doesn't seem to be a convenient spot
+    // in between slot reference resolution and where the stats are needed.
+    // So, for now, we'll just load all the column stats up front.
+    t.loadColumnStats();
+    return t;
   }
-
   public LocalTable(LocalDb db, String tblName, SchemaInfo schemaInfo) {
     this.db_ = Preconditions.checkNotNull(db);
     this.name_ = Preconditions.checkNotNull(tblName);
@@ -179,6 +190,16 @@ abstract class LocalTable implements FeTable {
     return schemaInfo_.tableStats_;
   }
 
+  protected void loadColumnStats() {
+    try {
+      List<ColumnStatisticsObj> stats = db_.getCatalog().getMetaProvider()
+          .loadTableColumnStatistics(db_.getName(), getName(), getColumnNames());
+      FeCatalogUtils.injectColumnStats(stats, this);
+    } catch (TException e) {
+      LOG.warn("Could not load column statistics for: " + getFullName(), e);
+    }
+  }
+
   /**
    * The table schema, loaded from the HMS Table object. This is common
    * to all Table implementations and includes the column definitions and

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
index f59a7c9..9f433f8 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
@@ -23,6 +23,7 @@ import java.util.Map;
 
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -70,6 +71,12 @@ interface MetaProvider {
       throws MetaException, TException;
 
   /**
+   * Load statistics for the given columns from the given table.
+   */
+  List<ColumnStatisticsObj> loadTableColumnStatistics(String dbName,
+      String tblName, List<String> colNames) throws TException;
+
+  /**
    * Load file metadata and block locations for the files in the given
    * partition directory.
    */

http://git-wip-us.apache.org/repos/asf/impala/blob/9df9efc5/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
index a242b1c..100ba47 100644
--- a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.impala.catalog.CatalogTest;
+import org.apache.impala.catalog.ColumnStats;
 import org.apache.impala.catalog.FeCatalogUtils;
 import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.FeFsPartition;
@@ -149,4 +150,19 @@ public class LocalCatalogTest {
     }
     assertEquals(24, totalFds);
   }
+
+  @Test
+  public void testColumnStats() throws Exception {
+    FeFsTable t = (FeFsTable) catalog_.getTable("functional",  "alltypesagg");
+    // Verify expected stats for a partitioning column.
+    // 'days' has 10 non-NULL plus one NULL partition
+    ColumnStats stats = t.getColumn("day").getStats();
+    assertEquals(11, stats.getNumDistinctValues());
+    assertEquals(1, stats.getNumNulls());
+
+    // Verify expected stats for timestamp.
+    stats = t.getColumn("timestamp_col").getStats();
+    assertEquals(10210, stats.getNumDistinctValues());
+    assertEquals(-1, stats.getNumNulls());
+  }
 }


[3/3] impala git commit: IMPALA-7279: Fix flakiness in test_rows_availability

Posted by to...@apache.org.
IMPALA-7279: Fix flakiness in test_rows_availability

This patch fixes a flaky time string parsing method in
test_rows_availability that fails on strings with microsecond precision.

Change-Id: If7634869823d8cc4059048dd5d3c3a984744f3be
Reviewed-on: http://gerrit.cloudera.org:8080/10922
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: e8a669bf918e8d1c7ec158acdbacdaf6d1755549
Parents: 9df9efc
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Wed Jul 11 10:57:23 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Thu Jul 12 02:42:00 2018 +0000

----------------------------------------------------------------------
 tests/query_test/test_hash_join_timer.py   | 28 +++++-------------------
 tests/query_test/test_rows_availability.py | 29 +++----------------------
 tests/util/parse_util.py                   | 13 +++++++++++
 3 files changed, 22 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e8a669bf/tests/query_test/test_hash_join_timer.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_hash_join_timer.py b/tests/query_test/test_hash_join_timer.py
index c111c03..c1b0f0f 100644
--- a/tests/query_test/test_hash_join_timer.py
+++ b/tests/query_test/test_hash_join_timer.py
@@ -21,6 +21,7 @@ import re
 from tests.common.impala_cluster import ImpalaCluster
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.test_vector import ImpalaTestDimension
+from tests.util.parse_util import parse_duration_string_ms
 from tests.verifiers.metric_verifier import MetricVerifier
 
 
@@ -154,27 +155,10 @@ class TestHashJoinTimer(ImpalaTestSuite):
         "Unable to verify ExecSummary: {0}".format(profile)
 
   def __verify_join_time(self, duration, comment):
-    duration_ms = self.__parse_duration_ms(duration)
+    duration_ms = parse_duration_string_ms(duration)
     if (duration_ms > self.HASH_JOIN_UPPER_BOUND_MS):
-        assert False, "Hash join timing too high for %s: %s %s" %(comment, duration, duration_ms)
+      assert False, "Hash join timing too high for %s: %s %s" % (
+        comment, duration, duration_ms)
     if (duration_ms < self.HASH_JOIN_LOWER_BOUND_MS):
-        assert False, "Hash join timing too low for %s: %s %s" %(comment, duration, duration_ms)
-
-  def __parse_duration_ms(self, duration):
-    """Parses a duration string of the form 1h2h3m4s5.6ms into milliseconds."""
-    matches = re.findall(r'(?P<value>[0-9]+(\.[0-9]+)?)(?P<units>\D+)', duration)
-    assert matches, 'Failed to parse duration string %s' % duration
-    hours = 0
-    minutes = 0
-    seconds = 0
-    milliseconds = 0
-    for match in matches:
-      if (match[2] == 'h'):
-        hours = float(match[0])
-      elif (match[2] == 'm'):
-        minutes = float(match[0])
-      elif (match[2] == 's'):
-        seconds = float(match[0])
-      elif (match[2] == 'ms'):
-        milliseconds = float(match[0])
-    return hours * 60 * 60 * 1000 + minutes * 60 * 1000 + seconds * 1000 + milliseconds
+      assert False, "Hash join timing too low for %s: %s %s" % (
+        comment, duration, duration_ms)

http://git-wip-us.apache.org/repos/asf/impala/blob/e8a669bf/tests/query_test/test_rows_availability.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_rows_availability.py b/tests/query_test/test_rows_availability.py
index 06b4d65..0bd65df 100644
--- a/tests/query_test/test_rows_availability.py
+++ b/tests/query_test/test_rows_availability.py
@@ -19,6 +19,7 @@ import pytest
 import re
 from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.test_vector import ImpalaTestDimension
+from tests.util.parse_util import parse_duration_string_ms
 
 class TestRowsAvailability(ImpalaTestSuite):
   """Tests that the 'Rows available' timeline event is marked only after rows are
@@ -76,9 +77,9 @@ class TestRowsAvailability(ImpalaTestSuite):
     rows_avail_time_ms = None
     for line in profile.split("\n"):
       if "Ready to start on" in line:
-        start_time_ms = self.__parse_time_ms(self.__find_time(line))
+        start_time_ms = parse_duration_string_ms(self.__find_time(line))
       elif "Rows available:" in line:
-        rows_avail_time_ms = self.__parse_time_ms(self.__find_time(line))
+        rows_avail_time_ms = parse_duration_string_ms(self.__find_time(line))
 
     if start_time_ms is None:
       assert False, "Failed to find the 'Ready to start' timeline event in the " \
@@ -102,27 +103,3 @@ class TestRowsAvailability(ImpalaTestSuite):
     if match is None:
       assert False, "Failed to find time in runtime profile"
     return match.group(1)
-
-  @staticmethod
-  def __parse_time_ms(duration):
-    """Parses a duration string of the form 1h2h3m4s5.6ms7.8ns into milliseconds."""
-    matches = re.findall(r'([0-9]+h)?([0-9]+m)?([0-9]+s)?'\
-                         '([0-9]+(\.[0-9]+)?ms)?([0-9]+(\.[0-9]+)?ns)?',
-                         duration)
-    # Expect exactly two matches because all groups are optional in the regex.
-    if matches is None or len(matches) != 2:
-      assert False, 'Failed to parse duration string %s' % duration
-    hours = 0
-    minutes = 0
-    seconds = 0
-    milliseconds = 0
-    if matches[0][0]:
-      hours = int(matches[0][0][:-1])
-    if matches[0][1]:
-      minutes = int(matches[0][1][:-1])
-    if matches[0][2]:
-      seconds = int(matches[0][2][:-1])
-    if matches[0][3]:
-      # Truncate fractional milliseconds.
-      milliseconds = int(float(matches[0][3][:-2]))
-    return hours * 60 * 60 * 1000 + minutes * 60 * 1000 + seconds * 1000 + milliseconds

http://git-wip-us.apache.org/repos/asf/impala/blob/e8a669bf/tests/util/parse_util.py
----------------------------------------------------------------------
diff --git a/tests/util/parse_util.py b/tests/util/parse_util.py
index 6869489..592716b 100644
--- a/tests/util/parse_util.py
+++ b/tests/util/parse_util.py
@@ -70,3 +70,16 @@ def parse_mem_to_mb(mem, units):
   else:
     raise Exception('Unexpected memory unit "%s"' % units)
   return int(mem)
+
+def parse_duration_string_ms(duration):
+  """Parses a duration string of the form 1h2h3m4s5.6ms4.5us7.8ns into milliseconds."""
+  pattern = r'(?P<value>[0-9]+\.?[0-9]*?)(?P<units>\D+)'
+  matches = list(re.finditer(pattern, duration))
+  assert matches, 'Failed to parse duration string %s' % duration
+
+  times = {'h': 0, 'm': 0, 's': 0, 'ms': 0}
+  for match in matches:
+    parsed = match.groupdict()
+    times[parsed['units']] = float(parsed['value'])
+
+  return (times['h'] * 60 * 60 + times['m'] * 60 + times['s']) * 1000 + times['ms']