You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2016/08/30 15:48:20 UTC

[2/2] incubator-impala git commit: IMPALA-4001: qgen: add proof of concept tests for Query() objects

IMPALA-4001: qgen: add proof of concept tests for Query() objects

This patch adds a simple proof-of-concept test framework, and a few
tests, for the random query generator, specifically the portion of the
random query generator that is responsible for taking a Query object and
doing something with it. The two pieces of functionality I chose for
exhibition are

1. Writing the query into Impala SQL
2. Reporting characteristics in the SELECT clause (used internally)

In the interest of keeping the patch small, I have not added many tests,
nor have I chose to focus on more areas for test. On its own this is
fairly simple. As I add features to this portion of the query generator,
though, it will be more useful to test new functionality and also
regression test the framework.

Change-Id: I2ed1960430ae0af469986e33f88aecb6fa74e999
Reviewed-on: http://gerrit.cloudera.org:8080/4081
Reviewed-by: Michael Brown <mi...@cloudera.com>
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Internal Jenkins


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

Branch: refs/heads/master
Commit: 6bbb7fb3d24974a18c17c475b48d01c37013c204
Parents: e453086
Author: Michael Brown <mi...@cloudera.com>
Authored: Fri Aug 19 11:10:43 2016 -0700
Committer: Internal Jenkins <cl...@gerrit.cloudera.org>
Committed: Tue Aug 30 02:15:35 2016 +0000

----------------------------------------------------------------------
 tests/comparison/tests/README                   |  11 ++
 tests/comparison/tests/fake_query.py            | 115 +++++++++++++++++
 tests/comparison/tests/query_object_testdata.py | 126 +++++++++++++++++++
 tests/comparison/tests/test_query_objects.py    |  83 ++++++++++++
 4 files changed, 335 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/6bbb7fb3/tests/comparison/tests/README
----------------------------------------------------------------------
diff --git a/tests/comparison/tests/README b/tests/comparison/tests/README
new file mode 100644
index 0000000..2859bb2
--- /dev/null
+++ b/tests/comparison/tests/README
@@ -0,0 +1,11 @@
+Purpose
+
+This directory contains tests for the Random Query Generator.
+
+How-to
+
+To run the tests we assume you have an impala-python environment already
+set up. Then:
+
+$ cd "${IMPALA_HOME}"/tests/comparison/tests
+$ impala-py.test

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/6bbb7fb3/tests/comparison/tests/fake_query.py
----------------------------------------------------------------------
diff --git a/tests/comparison/tests/fake_query.py b/tests/comparison/tests/fake_query.py
new file mode 100644
index 0000000..1241b73
--- /dev/null
+++ b/tests/comparison/tests/fake_query.py
@@ -0,0 +1,115 @@
+# 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.
+
+# This module is used to make instantation of Query() objects a little easier when
+# building them for testing. In typical usage, Query objects and their attributes
+# (clauses, expressions, etc.) are instantiated with little data and built up over time.
+# That works for the query generator, because a lot of logical steps need to happen
+# before the Query building is completed. For our testing purposes, though, we need
+# completed, static Query() objects, and a way to build them up rather easily and
+# expressively.
+#
+# Thus we have lightweight functions that handle initialization and any attribute
+# setting as needed.
+#
+# TODO: As much as possible, it would be better to refactor our data structures to be
+# more testable. But we have a chicken and egg problem in that we have no tests. We have
+# chosen to leave the original datastructures alone, and after we build up some tests to
+# gain confidence, we can modify them to be more testable, and we can remove items from
+# here.
+
+from tests.comparison.common import Column, Table
+from tests.comparison.funcs import AnalyticFirstValue
+from tests.comparison.query import Query, SelectClause, SelectItem
+
+
+def FakeColumn(name, type_):
+  """
+  Return a Column, the creation of which allows the user not to have to specify the
+  first argument, which is the table to which the column belongs.
+
+  Typical use should be when creating a FakeTable, use FakeColumns as arguments.
+  """
+  return Column(None, name, type_)
+
+
+def FakeTable(name, fake_columns):
+  """
+  Return a Table consisting of one or more FakeColumns. Because Columns are added via
+  method, we support nesting here instead.
+  """
+  table = Table(name)
+  if not fake_columns:
+    raise Exception('You must supply at least one FakeColumn argument')
+  for fake_column in fake_columns:
+    table.add_col(fake_column)
+  return table
+
+
+def FakeSelectClause(*args):
+  """
+  Return a SelectClause from value expressions args. This abstracts away from the
+  user the need to explicitly make the value expression items SelectItems.
+  """
+  return SelectClause([SelectItem(_) for _ in args])
+
+
+def FakeQuery(
+    with_clause=None,
+    select_clause=None,
+    from_clause=None,
+    where_clause=None,
+    group_by_clause=None,
+    having_clause=None,
+    union_clause=None,
+    order_by_clause=None,
+    limit_clause=None
+):
+  """
+  Return a Query object constructed by the keyword args above. select_clause and
+  from_clause are required.
+  """
+  query = Query()
+  query.with_clause = with_clause
+  query.select_clause = select_clause
+  query.from_clause = from_clause
+  query.where_clause = where_clause
+  query.group_by_clause = group_by_clause
+  query.having_clause = having_clause
+  query.union_clause = union_clause
+  query.order_by_clause = order_by_clause
+  query.limit_clause = limit_clause
+  if select_clause is None or from_clause is None:
+    raise Exception('FakeQuery must at least contain a select_clause and a from_clause')
+  return query
+
+
+def FakeFirstValue(
+    val_expr,
+    partition_by_clause=None,
+    order_by_clause=None,
+    window_clause=None
+):
+  """
+  Return an AnalyticFirstValue object based on val_expr and optional clauses. The
+  clauses must be *Clause objects (see the funcs and query modules).
+  """
+  first_value = AnalyticFirstValue.create_from_args(val_expr)
+  first_value.partition_by_clause = partition_by_clause
+  first_value.order_by_clause = order_by_clause
+  first_value.window_clause = window_clause
+  return first_value

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/6bbb7fb3/tests/comparison/tests/query_object_testdata.py
----------------------------------------------------------------------
diff --git a/tests/comparison/tests/query_object_testdata.py b/tests/comparison/tests/query_object_testdata.py
new file mode 100644
index 0000000..c2b3d3e
--- /dev/null
+++ b/tests/comparison/tests/query_object_testdata.py
@@ -0,0 +1,126 @@
+# 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.
+
+from collections import namedtuple
+
+from fake_query import FakeColumn, FakeFirstValue, FakeQuery, FakeSelectClause, FakeTable
+from tests.comparison.db_types import Char, Int
+from tests.comparison.funcs import AggCount
+from tests.comparison.query import FromClause, OrderByClause
+
+
+QueryTest = namedtuple(
+    # A QueryTest object contains a Query and all data to verify about it as other
+    # attributes. This allows a new Query to be added without need to modify tests
+    # themselves. The various tests cherry-pick which test attributes they need to
+    # verify against the Query.
+    #
+    # If you add a new test, add a new attribute, or perhaps reuse one or more
+    # attributes.
+    #
+    # If you add a new test case, add a new item to QUERY_TEST_CASESs array.
+    #
+    # All attributes are required.
+    'QueryTest',
+    [
+        # string to represent readable pytest testid
+        'testid',
+        # Query object, formed via FakeQuery
+        'query',
+        # textual form of FakeQuery
+        'impala_query_string',
+        # hash representing various item counts (see SelectItem property methods)
+        'select_item_counts',
+    ]
+)
+
+
+# FakeTables must be declared for use by queries. Tables may be reused as needed for
+# multiple FakeQueries.
+SIMPLE_TABLE = FakeTable(
+    'fake_table',
+    [
+        FakeColumn('int_col', Int),
+        FakeColumn('char_col', Char),
+    ]
+)
+
+
+# All tests involving queries should be written to use this dataset.
+QUERY_TEST_CASES = [
+    QueryTest(
+        testid='select col from table',
+        query=FakeQuery(
+            select_clause=FakeSelectClause(*SIMPLE_TABLE.cols),
+            from_clause=FromClause(SIMPLE_TABLE),
+        ),
+        impala_query_string=(
+            'SELECT\n'
+            'fake_table.int_col,\n'
+            'TRIM(fake_table.char_col)\n'
+            'FROM fake_table'
+        ),
+        select_item_counts={
+            'items': 2,
+            'basic_items': 2,
+            'agg_items': 0,
+            'analytic_items': 0,
+        },
+    ),
+    QueryTest(
+        testid='select count()',
+        query=FakeQuery(
+            select_clause=FakeSelectClause(
+                AggCount.create_from_args(SIMPLE_TABLE.cols[0])),
+            from_clause=FromClause(SIMPLE_TABLE),
+        ),
+        impala_query_string=(
+            'SELECT\n'
+            'COUNT(fake_table.int_col)\n'
+            'FROM fake_table'
+        ),
+        select_item_counts={
+            'items': 1,
+            'basic_items': 0,
+            'agg_items': 1,
+            'analytic_items': 0,
+        },
+    ),
+    QueryTest(
+        testid='select first_value(col) over (order by col)',
+        query=FakeQuery(
+            select_clause=FakeSelectClause(
+                FakeFirstValue(
+                    SIMPLE_TABLE.cols[0],
+                    order_by_clause=OrderByClause([SIMPLE_TABLE.cols[0]])
+                ),
+            ),
+            from_clause=FromClause(SIMPLE_TABLE),
+        ),
+        impala_query_string=(
+            'SELECT\n'
+            'FIRST_VALUE(fake_table.int_col) OVER (ORDER BY fake_table.int_col ASC)\n'
+            'FROM fake_table'
+        ),
+        select_item_counts={
+            'items': 1,
+            'basic_items': 0,
+            'agg_items': 0,
+            'analytic_items': 1,
+        },
+    ),
+]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/6bbb7fb3/tests/comparison/tests/test_query_objects.py
----------------------------------------------------------------------
diff --git a/tests/comparison/tests/test_query_objects.py b/tests/comparison/tests/test_query_objects.py
new file mode 100644
index 0000000..b0794d2
--- /dev/null
+++ b/tests/comparison/tests/test_query_objects.py
@@ -0,0 +1,83 @@
+# 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.
+
+import pytest
+
+from tests.comparison.model_translator import SqlWriter
+
+from query_object_testdata import QUERY_TEST_CASES
+
+
+def _idfn(query_test):
+  return query_test.testid
+
+
+def verify_select_clause_items(query, expected_item_counts):
+  """
+  Verify that a well-formed Query() object's select_clause (SelectClause instance)
+  reports correct item counts. expected_item_counts should be a dictionary with keys
+  matching SelectItem property methods that report item counts and values for the
+  counts.
+  """
+  attrs_to_check = [
+      'items',
+      'basic_items',
+      'analytic_items',
+      'agg_items',
+  ]
+  select_clause = query.select_clause
+  for attr_name in attrs_to_check:
+    select_clause_attr = getattr(select_clause, attr_name)
+    expected_item_count_attr = expected_item_counts.get(attr_name)
+    actual_item_count = len(select_clause_attr)
+    assert len(select_clause_attr) == expected_item_count_attr, (
+        'item count mismatch for item "{item}": expected: {expected}; actual: '
+        '{actual}'.format(item=attr_name, expected=expected_item_count_attr,
+                          actual=actual_item_count))
+
+
+def verify_sql_matches(actual, expected, strip=True):
+  """
+  Assert that the actual and expected SQL queries match. Trailing white space is
+  stripped by default.
+  """
+  if strip:
+    actual = actual.strip()
+    expected = expected.strip()
+  assert actual == expected, 'actual SQL "{actual}" != expected SQL "{expected}"'.format(
+      actual=actual, expected=expected)
+
+
+@pytest.yield_fixture
+def sql_writer():
+  """
+  Return a SqlWriter object that is torn down at the end of each test.
+  """
+  # TODO: Later, we can parametrize on dialect, but for now, this is just PoC.
+  yield SqlWriter.create(dialect='IMPALA')
+
+
+@pytest.mark.parametrize('query_test', QUERY_TEST_CASES, ids=_idfn)
+def test_select_clause_items(query_test):
+  verify_select_clause_items(query_test.query, query_test.select_item_counts)
+
+
+@pytest.mark.parametrize('query_test', QUERY_TEST_CASES, ids=_idfn)
+def test_write_query(sql_writer, query_test):
+  verify_sql_matches(
+      sql_writer.write_query(query_test.query),
+      query_test.impala_query_string)