From a5d6327d83cc4beea06c42f96424f98520f35743 Mon Sep 17 00:00:00 2001 From: Udi Meiri Date: Mon, 15 Oct 2018 13:12:18 -0700 Subject: [PATCH 01/42] Upgrade BigQuery client from 0.25.0 to 1.6.0 This change only affects integration tests. --- .../complete/game/game_stats_it_test.py | 22 ++---- .../game/hourly_team_score_it_test.py | 21 ++--- .../complete/game/leader_board_it_test.py | 26 ++----- .../cookbook/bigquery_tornadoes_it_test.py | 3 +- .../gcp/big_query_query_to_table_it_test.py | 4 +- .../gcp/big_query_query_to_table_pipeline.py | 2 +- .../io/gcp/bigquery_io_read_it_test.py | 2 +- .../io/gcp/tests/bigquery_matcher.py | 15 +--- .../io/gcp/tests/bigquery_matcher_test.py | 62 ++++----------- sdks/python/apache_beam/io/gcp/tests/utils.py | 68 +++++++++++----- .../apache_beam/io/gcp/tests/utils_test.py | 78 ++++++------------- .../container/base_image_requirements.txt | 2 +- sdks/python/scripts/run_postcommit.sh | 3 +- sdks/python/setup.py | 2 +- 14 files changed, 121 insertions(+), 189 deletions(-) diff --git a/sdks/python/apache_beam/examples/complete/game/game_stats_it_test.py b/sdks/python/apache_beam/examples/complete/game/game_stats_it_test.py index 2fc19daa1af82..67b80c017bcf8 100644 --- a/sdks/python/apache_beam/examples/complete/game/game_stats_it_test.py +++ b/sdks/python/apache_beam/examples/complete/game/game_stats_it_test.py @@ -83,11 +83,8 @@ def setUp(self): self.input_topic.name) # Set up BigQuery environment - from google.cloud import bigquery - client = bigquery.Client() - unique_dataset_name = self.OUTPUT_DATASET + str(int(time.time())) - self.dataset = client.dataset(unique_dataset_name, project=self.project) - self.dataset.create() + self.dataset_ref = utils.create_bq_dataset(self.project, + self.OUTPUT_DATASET) self._test_timestamp = int(time.time() * 1000) @@ -105,17 +102,14 @@ def _cleanup_pubsub(self): test_utils.cleanup_subscriptions(self.sub_client, [self.input_sub]) test_utils.cleanup_topics(self.pub_client, [self.input_topic]) - def _cleanup_dataset(self): - self.dataset.delete() - @attr('IT') def test_game_stats_it(self): state_verifier = PipelineStateMatcher(PipelineState.RUNNING) success_condition = 'mean_duration=300 LIMIT 1' - sessions_query = ('SELECT mean_duration FROM [%s:%s.%s] ' + sessions_query = ('SELECT mean_duration FROM `%s.%s.%s` ' 'WHERE %s' % (self.project, - self.dataset.name, + self.dataset_ref.dataset_id, self.OUTPUT_TABLE_SESSIONS, success_condition)) bq_sessions_verifier = BigqueryMatcher(self.project, @@ -125,7 +119,7 @@ def test_game_stats_it(self): # TODO(mariagh): Add teams table verifier once game_stats.py is fixed. extra_opts = {'subscription': self.input_sub.name, - 'dataset': self.dataset.name, + 'dataset': self.dataset_ref.dataset_id, 'topic': self.input_topic.name, 'fixed_window_duration': 1, 'user_activity_window_duration': 1, @@ -137,11 +131,7 @@ def test_game_stats_it(self): # Register cleanup before pipeline execution. # Note that actual execution happens in reverse order. self.addCleanup(self._cleanup_pubsub) - self.addCleanup(self._cleanup_dataset) - self.addCleanup(utils.delete_bq_table, self.project, - self.dataset.name, self.OUTPUT_TABLE_SESSIONS) - self.addCleanup(utils.delete_bq_table, self.project, - self.dataset.name, self.OUTPUT_TABLE_TEAMS) + self.addCleanup(utils.delete_bq_dataset, self.project, self.dataset_ref) # Generate input data and inject to PubSub. self._inject_pubsub_game_events(self.input_topic, self.DEFAULT_INPUT_COUNT) diff --git a/sdks/python/apache_beam/examples/complete/game/hourly_team_score_it_test.py b/sdks/python/apache_beam/examples/complete/game/hourly_team_score_it_test.py index 584aa055d3d1d..568513201218b 100644 --- a/sdks/python/apache_beam/examples/complete/game/hourly_team_score_it_test.py +++ b/sdks/python/apache_beam/examples/complete/game/hourly_team_score_it_test.py @@ -33,7 +33,6 @@ from __future__ import absolute_import import logging -import time import unittest from hamcrest.core.core.allof import all_of @@ -60,20 +59,14 @@ def setUp(self): self.project = self.test_pipeline.get_option('project') # Set up BigQuery environment - from google.cloud import bigquery - client = bigquery.Client() - unique_dataset_name = self.OUTPUT_DATASET + str(int(time.time())) - self.dataset = client.dataset(unique_dataset_name, project=self.project) - self.dataset.create() - - def _cleanup_dataset(self): - self.dataset.delete() + self.dataset_ref = utils.create_bq_dataset(self.project, + self.OUTPUT_DATASET) @attr('IT') def test_hourly_team_score_it(self): state_verifier = PipelineStateMatcher(PipelineState.DONE) - query = ('SELECT COUNT(*) FROM [%s:%s.%s]' % (self.project, - self.dataset.name, + query = ('SELECT COUNT(*) FROM `%s.%s.%s`' % (self.project, + self.dataset_ref.dataset_id, self.OUTPUT_TABLE)) bigquery_verifier = BigqueryMatcher(self.project, @@ -81,16 +74,14 @@ def test_hourly_team_score_it(self): self.DEFAULT_EXPECTED_CHECKSUM) extra_opts = {'input': self.DEFAULT_INPUT_FILE, - 'dataset': self.dataset.name, + 'dataset': self.dataset_ref.dataset_id, 'window_duration': 1, 'on_success_matcher': all_of(state_verifier, bigquery_verifier)} # Register clean up before pipeline execution # Note that actual execution happens in reverse order. - self.addCleanup(self._cleanup_dataset) - self.addCleanup(utils.delete_bq_table, self.project, - self.dataset.name, self.OUTPUT_TABLE) + self.addCleanup(utils.delete_bq_dataset, self.project, self.dataset_ref) # Get pipeline options from command argument: --test-pipeline-options, # and start pipeline job by calling pipeline main function. diff --git a/sdks/python/apache_beam/examples/complete/game/leader_board_it_test.py b/sdks/python/apache_beam/examples/complete/game/leader_board_it_test.py index e0e309b126530..27a170fcb67a4 100644 --- a/sdks/python/apache_beam/examples/complete/game/leader_board_it_test.py +++ b/sdks/python/apache_beam/examples/complete/game/leader_board_it_test.py @@ -85,11 +85,8 @@ def setUp(self): self.input_topic.name) # Set up BigQuery environment - from google.cloud import bigquery - client = bigquery.Client() - unique_dataset_name = self.OUTPUT_DATASET + str(int(time.time())) - self.dataset = client.dataset(unique_dataset_name, project=self.project) - self.dataset.create() + self.dataset_ref = utils.create_bq_dataset(self.project, + self.OUTPUT_DATASET) self._test_timestamp = int(time.time() * 1000) @@ -107,26 +104,23 @@ def _cleanup_pubsub(self): test_utils.cleanup_subscriptions(self.sub_client, [self.input_sub]) test_utils.cleanup_topics(self.pub_client, [self.input_topic]) - def _cleanup_dataset(self): - self.dataset.delete() - @attr('IT') def test_leader_board_it(self): state_verifier = PipelineStateMatcher(PipelineState.RUNNING) success_condition = 'total_score=5000 LIMIT 1' - users_query = ('SELECT total_score FROM [%s:%s.%s] ' + users_query = ('SELECT total_score FROM `%s.%s.%s` ' 'WHERE %s' % (self.project, - self.dataset.name, + self.dataset_ref.dataset_id, self.OUTPUT_TABLE_USERS, success_condition)) bq_users_verifier = BigqueryMatcher(self.project, users_query, self.DEFAULT_EXPECTED_CHECKSUM) - teams_query = ('SELECT total_score FROM [%s:%s.%s] ' + teams_query = ('SELECT total_score FROM `%s.%s.%s` ' 'WHERE %s' % (self.project, - self.dataset.name, + self.dataset_ref.dataset_id, self.OUTPUT_TABLE_TEAMS, success_condition)) bq_teams_verifier = BigqueryMatcher(self.project, @@ -134,7 +128,7 @@ def test_leader_board_it(self): self.DEFAULT_EXPECTED_CHECKSUM) extra_opts = {'subscription': self.input_sub.name, - 'dataset': self.dataset.name, + 'dataset': self.dataset_ref.dataset_id, 'topic': self.input_topic.name, 'team_window_duration': 1, 'wait_until_finish_duration': @@ -146,11 +140,7 @@ def test_leader_board_it(self): # Register cleanup before pipeline execution. # Note that actual execution happens in reverse order. self.addCleanup(self._cleanup_pubsub) - self.addCleanup(self._cleanup_dataset) - self.addCleanup(utils.delete_bq_table, self.project, - self.dataset.name, self.OUTPUT_TABLE_USERS) - self.addCleanup(utils.delete_bq_table, self.project, - self.dataset.name, self.OUTPUT_TABLE_TEAMS) + self.addCleanup(utils.delete_bq_dataset, self.project, self.dataset_ref) # Generate input data and inject to PubSub. self._inject_pubsub_game_events(self.input_topic, self.DEFAULT_INPUT_COUNT) diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py index b7e90839ffaa4..21e9c48f302cf 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py @@ -53,7 +53,7 @@ def test_bigquery_tornadoes_it(self): dataset = 'BigQueryTornadoesIT' table = 'monthly_tornadoes_%s' % int(round(time.time() * 1000)) output_table = '.'.join([dataset, table]) - query = 'SELECT month, tornado_count FROM [%s]' % output_table + query = 'SELECT month, tornado_count FROM `%s`' % output_table pipeline_verifiers = [PipelineStateMatcher(), BigqueryMatcher( @@ -64,6 +64,7 @@ def test_bigquery_tornadoes_it(self): 'on_success_matcher': all_of(*pipeline_verifiers)} # Register cleanup before pipeline execution. + # Note that actual execution happens in reverse order. self.addCleanup(utils.delete_bq_table, project, dataset, table) # Get pipeline options from command argument: --test-pipeline-options, diff --git a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py index c0962c73e658d..adf3a0fcac0ec 100644 --- a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py +++ b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py @@ -47,7 +47,7 @@ NEW_TYPES_OUTPUT_SCHEMA = ( '{"fields": [{"name": "bytes","type": "BYTES"},' '{"name": "date","type": "DATE"},{"name": "time","type": "TIME"}]}') -NEW_TYPES_OUTPUT_VERIFY_QUERY = ('SELECT date FROM [%s];') +NEW_TYPES_OUTPUT_VERIFY_QUERY = ('SELECT date FROM `%s`;') # There are problems with query time and bytes with current version of bigquery. NEW_TYPES_OUTPUT_EXPECTED = [ (datetime.date(2000, 1, 1),), @@ -61,7 +61,7 @@ NEW_TYPES_QUERY = ( 'SELECT bytes, date, time FROM [%s.%s]') DIALECT_OUTPUT_SCHEMA = ('{"fields": [{"name": "fruit","type": "STRING"}]}') -DIALECT_OUTPUT_VERIFY_QUERY = ('SELECT fruit from [%s];') +DIALECT_OUTPUT_VERIFY_QUERY = ('SELECT fruit from `%s`;') DIALECT_OUTPUT_EXPECTED = [(u'apple',), (u'orange',)] diff --git a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_pipeline.py b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_pipeline.py index 65a4941f59ab5..1e549c6763008 100644 --- a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_pipeline.py +++ b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_pipeline.py @@ -62,7 +62,7 @@ def run_bq_pipeline(argv=None): known_args.output, schema=table_schema, create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, - write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE))) + write_disposition=beam.io.BigQueryDisposition.WRITE_EMPTY))) result = p.run() result.wait_until_finish() diff --git a/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py index 8851a143971e5..72e1bb7fc8be8 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py @@ -54,7 +54,7 @@ def run_bigquery_io_read_pipeline(self, input_size): **extra_opts)) @attr('IT') - def bigquery_read_1M_python(self): + def test_bigquery_read_1M_python(self): self.run_bigquery_io_read_pipeline('1M') diff --git a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py index c33f0db49983a..c3069a359cc29 100644 --- a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py +++ b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py @@ -94,19 +94,8 @@ def _matches(self, _): retry_filter=retry_on_http_and_value_error) def _query_with_retry(self, bigquery_client): """Run Bigquery query with retry if got error http response""" - query = bigquery_client.run_sync_query(self.query) - query.run() - - # Fetch query data one page at a time. - page_token = None - results = [] - while True: - for row in query.fetch_data(page_token=page_token): - results.append(row) - if results: - break - - return results + query_job = bigquery_client.query(self.query) + return [row.values() for row in query_job] def describe_to(self, description): description \ diff --git a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py index e6ae9a06dc872..f2714623b2366 100644 --- a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py +++ b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher_test.py @@ -22,9 +22,8 @@ import logging import unittest +import mock from hamcrest import assert_that as hc_assert_that -from mock import Mock -from mock import patch from apache_beam.io.gcp.tests import bigquery_matcher as bq_verifier from apache_beam.testing.test_utils import patch_retry @@ -32,26 +31,30 @@ # Protect against environments where bigquery library is not available. # pylint: disable=wrong-import-order, wrong-import-position try: + # TODO: fix usage from google.cloud import bigquery from google.cloud.exceptions import NotFound except ImportError: bigquery = None + NotFound = None # pylint: enable=wrong-import-order, wrong-import-position @unittest.skipIf(bigquery is None, 'Bigquery dependencies are not installed.') +@mock.patch.object(bigquery, 'Client') class BigqueryMatcherTest(unittest.TestCase): def setUp(self): - self._mock_result = Mock() + self._mock_result = mock.Mock() patch_retry(self, bq_verifier) - @patch.object(bigquery, 'Client') def test_bigquery_matcher_success(self, mock_bigquery): - mock_query = Mock() - mock_client = mock_bigquery.return_value - mock_client.run_sync_query.return_value = mock_query - mock_query.fetch_data.return_value = ([], None, None) + mock_query_result = [mock.Mock(), mock.Mock(), mock.Mock()] + mock_query_result[0].values.return_value = [] + mock_query_result[1].values.return_value = None + mock_query_result[2].values.return_value = None + + mock_bigquery.return_value.query.return_value = mock_query_result matcher = bq_verifier.BigqueryMatcher( 'mock_project', @@ -59,51 +62,16 @@ def test_bigquery_matcher_success(self, mock_bigquery): '59f9d6bdee30d67ea73b8aded121c3a0280f9cd8') hc_assert_that(self._mock_result, matcher) - @patch.object(bigquery, 'Client') - def test_bigquery_matcher_query_run_error(self, mock_bigquery): - mock_query = Mock() - mock_client = mock_bigquery.return_value - mock_client.run_sync_query.return_value = mock_query - mock_query.run.side_effect = ValueError('job is already running') - - matcher = bq_verifier.BigqueryMatcher('mock_project', - 'mock_query', - 'mock_checksum') - with self.assertRaises(ValueError): - hc_assert_that(self._mock_result, matcher) - self.assertTrue(mock_query.run.called) - self.assertEqual(bq_verifier.MAX_RETRIES + 1, mock_query.run.call_count) - - @patch.object(bigquery, 'Client') - def test_bigquery_matcher_fetch_data_error(self, mock_bigquery): - mock_query = Mock() - mock_client = mock_bigquery.return_value - mock_client.run_sync_query.return_value = mock_query - mock_query.fetch_data.side_effect = ValueError('query job not executed') - - matcher = bq_verifier.BigqueryMatcher('mock_project', - 'mock_query', - 'mock_checksum') - with self.assertRaises(ValueError): - hc_assert_that(self._mock_result, matcher) - self.assertTrue(mock_query.fetch_data.called) - self.assertEqual(bq_verifier.MAX_RETRIES + 1, - mock_query.fetch_data.call_count) - - @patch.object(bigquery, 'Client') - def test_bigquery_matcher_query_responds_error_code(self, mock_bigquery): - mock_query = Mock() - mock_client = mock_bigquery.return_value - mock_client.run_sync_query.return_value = mock_query - mock_query.run.side_effect = NotFound('table is not found') + def test_bigquery_matcher_query_error_retry(self, mock_bigquery): + mock_query = mock_bigquery.return_value.query + mock_query.side_effect = NotFound('table not found') matcher = bq_verifier.BigqueryMatcher('mock_project', 'mock_query', 'mock_checksum') with self.assertRaises(NotFound): hc_assert_that(self._mock_result, matcher) - self.assertTrue(mock_query.run.called) - self.assertEqual(bq_verifier.MAX_RETRIES + 1, mock_query.run.call_count) + self.assertEqual(bq_verifier.MAX_RETRIES + 1, mock_query.call_count) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/io/gcp/tests/utils.py b/sdks/python/apache_beam/io/gcp/tests/utils.py index 81fc4736c0469..60987f1f8062e 100644 --- a/sdks/python/apache_beam/io/gcp/tests/utils.py +++ b/sdks/python/apache_beam/io/gcp/tests/utils.py @@ -21,14 +21,17 @@ from __future__ import absolute_import import logging +import time from apache_beam.utils import retry # Protect against environments where bigquery library is not available. try: from google.cloud import bigquery + from google.cloud.exceptions import NotFound except ImportError: bigquery = None + NotFound = None class GcpTestIOError(retry.PermanentException): @@ -40,26 +43,53 @@ class GcpTestIOError(retry.PermanentException): @retry.with_exponential_backoff( num_retries=3, retry_filter=retry.retry_on_server_errors_filter) -def delete_bq_table(project, dataset, table): - """Delete a Biqquery table. +def create_bq_dataset(project, dataset_base_name): + """Creates an empty BigQuery dataset. + + Args: + project: Project to work in. + dataset_base_name: Prefix for dataset id. + + Returns: + A ``google.cloud.bigquery.dataset.DatasetReference`` object pointing to the + new dataset. + """ + client = bigquery.Client(project=project) + unique_dataset_name = dataset_base_name + str(int(time.time())) + dataset_ref = client.dataset(unique_dataset_name, project=project) + dataset = bigquery.Dataset(dataset_ref) + client.create_dataset(dataset) + return dataset_ref + + +@retry.with_exponential_backoff( + num_retries=3, + retry_filter=retry.retry_on_server_errors_filter) +def delete_bq_dataset(project, dataset_ref): + """Deletes a BigQuery dataset and its contents. + + Args: + project: Project to work in. + dataset_ref: A ``google.cloud.bigquery.dataset.DatasetReference`` object + pointing to the dataset to delete. + """ + client = bigquery.Client(project=project) + client.delete_dataset(dataset_ref, delete_contents=True) + + +def delete_bq_table(project, dataset_id, table_id): + """Delete a BiqQuery table. Args: project: Name of the project. - dataset: Name of the dataset where table is. - table: Name of the table. + dataset_id: Name of the dataset where table is. + table_id: Name of the table. """ - logging.info('Clean up a Bigquery table with project: %s, dataset: %s, ' - 'table: %s.', project, dataset, table) - bq_dataset = bigquery.Client(project=project).dataset(dataset) - if not bq_dataset.exists(): - raise GcpTestIOError('Failed to cleanup. Bigquery dataset %s doesn\'t ' - 'exist in project %s.' % (dataset, project)) - bq_table = bq_dataset.table(table) - if not bq_table.exists(): - raise GcpTestIOError('Failed to cleanup. Bigquery table %s doesn\'t ' - 'exist in project %s, dataset %s.' % - (table, project, dataset)) - bq_table.delete() - if bq_table.exists(): - raise RuntimeError('Failed to cleanup. Bigquery table %s still exists ' - 'after cleanup.' % table) + logging.info('Clean up a BigQuery table with project: %s, dataset: %s, ' + 'table: %s.', project, dataset_id, table_id) + client = bigquery.Client(project=project) + table_ref = client.dataset(dataset_id).table(table_id) + try: + client.delete_table(table_ref) + except NotFound: + raise GcpTestIOError('BigQuery table does not exist: %s' % table_ref) diff --git a/sdks/python/apache_beam/io/gcp/tests/utils_test.py b/sdks/python/apache_beam/io/gcp/tests/utils_test.py index 4ea65a9d86be1..8af749740b965 100644 --- a/sdks/python/apache_beam/io/gcp/tests/utils_test.py +++ b/sdks/python/apache_beam/io/gcp/tests/utils_test.py @@ -22,8 +22,7 @@ import logging import unittest -from mock import Mock -from mock import patch +import mock from apache_beam.io.gcp.tests import utils from apache_beam.testing.test_utils import patch_retry @@ -31,74 +30,47 @@ # Protect against environments where bigquery library is not available. try: from google.cloud import bigquery + from google.cloud.exceptions import NotFound except ImportError: bigquery = None + NotFound = None @unittest.skipIf(bigquery is None, 'Bigquery dependencies are not installed.') +@mock.patch.object(bigquery, 'Client') class UtilsTest(unittest.TestCase): def setUp(self): - self._mock_result = Mock() patch_retry(self, utils) - @patch.object(bigquery, 'Client') - def test_delete_table_succeeds(self, mock_client): - mock_dataset = Mock() - mock_client.return_value.dataset = mock_dataset - mock_dataset.return_value.exists.return_value = True + @mock.patch.object(bigquery, 'Dataset') + def test_create_bq_dataset(self, mock_dataset, mock_client): + mock_client.dataset.return_value = 'dataset_ref' + mock_dataset.return_value = 'dataset_obj' + + utils.create_bq_dataset('project', 'dataset_base_name') + mock_client.return_value.create_dataset.assert_called_with('dataset_obj') - mock_table = Mock() - mock_dataset.return_value.table = mock_table - mock_table.return_value.exists.side_effect = [True, False] + def test_delete_bq_dataset(self, mock_client): + utils.delete_bq_dataset('project', 'dataset_ref') + mock_client.return_value.delete_dataset.assert_called_with( + 'dataset_ref', delete_contents=mock.ANY) + + def test_delete_table_succeeds(self, mock_client): + mock_client.return_value.dataset.return_value.table.return_value = ( + 'table_ref') utils.delete_bq_table('unused_project', 'unused_dataset', 'unused_table') + mock_client.return_value.delete_table.assert_called_with('table_ref') - @patch.object(bigquery, 'Client') - def test_delete_table_fails_dataset_not_exist(self, mock_client): - mock_dataset = Mock() - mock_client.return_value.dataset = mock_dataset - mock_dataset.return_value.exists.return_value = False - - with self.assertRaisesRegexp( - Exception, r'^Failed to cleanup. Bigquery dataset unused_dataset ' - r'doesn\'t exist'): - utils.delete_bq_table('unused_project', - 'unused_dataset', - 'unused_table') - - @patch.object(bigquery, 'Client') - def test_delete_table_fails_table_not_exist(self, mock_client): - mock_dataset = Mock() - mock_client.return_value.dataset = mock_dataset - mock_dataset.return_value.exists.return_value = True - - mock_table = Mock() - mock_dataset.return_value.table = mock_table - mock_table.return_value.exists.return_value = False - - with self.assertRaisesRegexp(Exception, - r'^Failed to cleanup. Bigquery table ' - 'unused_table doesn\'t exist'): - utils.delete_bq_table('unused_project', - 'unused_dataset', - 'unused_table') - - @patch.object(bigquery, 'Client') - def test_delete_table_fails_service_error(self, mock_client): - mock_dataset = Mock() - mock_client.return_value.dataset = mock_dataset - mock_dataset.return_value.exists.return_value = True - - mock_table = Mock() - mock_dataset.return_value.table = mock_table - mock_table.return_value.exists.return_value = True + def test_delete_table_fails_not_found(self, mock_client): + mock_client.return_value.dataset.return_value.table.return_value = ( + 'table_ref') + mock_client.return_value.delete_table.side_effect = NotFound('test') - with self.assertRaisesRegexp(Exception, - r'^Failed to cleanup. Bigquery table ' - 'unused_table still exists'): + with self.assertRaisesRegexp(Exception, r'does not exist:.*table_ref'): utils.delete_bq_table('unused_project', 'unused_dataset', 'unused_table') diff --git a/sdks/python/container/base_image_requirements.txt b/sdks/python/container/base_image_requirements.txt index 8e352a664df32..162fb932784d6 100644 --- a/sdks/python/container/base_image_requirements.txt +++ b/sdks/python/container/base_image_requirements.txt @@ -48,7 +48,7 @@ nose==1.3.7 google-apitools==0.5.20 googledatastore==7.0.1 google-cloud-pubsub==0.35.4 -google-cloud-bigquery==0.25.0 +google-cloud-bigquery==1.6.0 proto-google-cloud-datastore-v1==0.90.4 # Optional packages diff --git a/sdks/python/scripts/run_postcommit.sh b/sdks/python/scripts/run_postcommit.sh index be133769c52e9..2bd1ca41889c6 100755 --- a/sdks/python/scripts/run_postcommit.sh +++ b/sdks/python/scripts/run_postcommit.sh @@ -105,7 +105,8 @@ apache_beam.io.gcp.pubsub_integration_test:PubSubIntegrationTest" TESTS="--tests=\ apache_beam.examples.wordcount_it_test:WordCountIT.test_wordcount_it,\ apache_beam.io.gcp.pubsub_integration_test:PubSubIntegrationTest,\ -apache_beam.io.gcp.big_query_query_to_table_it_test:BigQueryQueryToTableIT" +apache_beam.io.gcp.big_query_query_to_table_it_test:BigQueryQueryToTableIT,\ +apache_beam.io.gcp.bigquery_io_read_it_test" fi fi diff --git a/sdks/python/setup.py b/sdks/python/setup.py index a3db7903fca36..4d5e227163c95 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -143,7 +143,7 @@ def get_version(): 'googledatastore==7.0.1; python_version < "3.0"', 'google-cloud-pubsub==0.35.4', # GCP packages required by tests - 'google-cloud-bigquery==0.25.0', + 'google-cloud-bigquery>=1.6.0,<1.7.0', ] if sys.version_info[0] == 2: From ea485be6a519c7297918c1e5ac1c37be40f47a1b Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 17 Oct 2018 14:57:06 -0700 Subject: [PATCH 02/42] [BEAM-5741] Make 'Contact Us' link more visible from contributor guide. We received some great feedback from a new contributor on our Contributor Guide. Amongst other things, they mentioned it was difficult to find a place to reach out for help from the Contributor Guide. The content is there on the website, but it wasn't very visible. This PR adds additional links to correct for this. --- website/src/_includes/section-menu/contribute.html | 1 + website/src/contribute/index.md | 10 ++++++++++ 2 files changed, 11 insertions(+) diff --git a/website/src/_includes/section-menu/contribute.html b/website/src/_includes/section-menu/contribute.html index b9bbba3855ed0..add23de4e9338 100644 --- a/website/src/_includes/section-menu/contribute.html +++ b/website/src/_includes/section-menu/contribute.html @@ -52,3 +52,4 @@
  • Release guide
  • +
  • Contact Us
  • \ No newline at end of file diff --git a/website/src/contribute/index.md b/website/src/contribute/index.md index ef4a485b33441..226b302bbd2a1 100644 --- a/website/src/contribute/index.md +++ b/website/src/contribute/index.md @@ -54,6 +54,11 @@ Most importantly, if you have an idea of how to contribute, then do it! For a list of open starter tasks, check [https://s.apache.org/beam-starter-tasks](https://s.apache.org/beam-starter-tasks). +## Finding Help + +If you find any issues with this guide or have questions that aren't answered, please +[reach out to the Beam community]({{ site.baseurl }}/community/contact-us). + ## Permissions For the [Beam issue tracker (JIRA)](https://issues.apache.org/jira/projects/BEAM/issues), @@ -407,3 +412,8 @@ healthy. A pull request becomes stale after its author fails to respond to actionable comments for 60 days. Author of a closed pull request is welcome to reopen the same pull request again in the future. The associated JIRAs will be unassigned from the author but will stay open. + +---- + +If you didn't find the information you were looking for in this guide, please +[reach out to the Beam community]({{ site.baseurl }}/community/contact-us). From e2b6cd89b9d168f4b028daef2f5294e983d5ab24 Mon Sep 17 00:00:00 2001 From: Michael Luckey <25622840+adude3141@users.noreply.github.com> Date: Sat, 20 Oct 2018 18:11:49 +0200 Subject: [PATCH 03/42] remove javacc bad option warning 'grammar_encoding' --- sdks/java/extensions/sql/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 8eaf09cc05b3b..1b008bc49b1e5 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -123,7 +123,7 @@ compileJavacc { dependsOn generateFmppSources inputDirectory = file(generateFmppJavaccRoot) outputDirectory = file(generatedJavaccPackageDir) - arguments = [grammar_encoding: "UTF-8", static: "false", lookahead: "2"] + arguments = [static: "false", lookahead: "2"] } // Help IntelliJ find the fmpp bits From 5751d4933ea2d4ad02b6ef5eb63cc6f31818bdcf Mon Sep 17 00:00:00 2001 From: akedin Date: Fri, 19 Oct 2018 15:37:06 -0700 Subject: [PATCH 04/42] [SQL] Move builtin aggregations creation to a map of factories --- .../transform/BeamAggregationTransforms.java | 82 +++++++---------- .../transform/BeamBuiltinAggregations.java | 91 +++++++++++-------- .../sql/impl/transform/agg/CovarianceFn.java | 10 ++ .../sql/impl/transform/agg/VarianceFn.java | 10 ++ 4 files changed, 103 insertions(+), 90 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index 0a143690a9def..d19c6d9055141 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -17,10 +17,13 @@ */ package org.apache.beam.sdk.extensions.sql.impl.transform; +import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.Row.toRow; +import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; @@ -29,6 +32,8 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.function.Function; import java.util.stream.IntStream; import org.apache.beam.sdk.coders.BigDecimalCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -156,60 +161,37 @@ public AggregationAdaptor( sourceFieldExps.add(refIndex); } - Schema.Field field = CalciteUtils.toField(aggName, call.type); - Schema.TypeName fieldTypeName = field.getType().getTypeName(); + Schema.Field field = CalciteUtils.toField(aggName, call.getType()); fields.add(field); + aggregators.add( + createAggregator(call, call.getAggregation().getName(), field.getType().getTypeName())); + } + finalSchema = fields.build().stream().collect(toSchema()); + } + + private CombineFn createAggregator( + AggregateCall call, String aggregatorName, Schema.TypeName fieldTypeName) { - switch (call.getAggregation().getName()) { - case "COUNT": - aggregators.add(Count.combineFn()); - break; - case "MAX": - aggregators.add(BeamBuiltinAggregations.createMax(call.type.getSqlTypeName())); - break; - case "MIN": - aggregators.add(BeamBuiltinAggregations.createMin(call.type.getSqlTypeName())); - break; - case "SUM": - case "$SUM0": - aggregators.add(BeamBuiltinAggregations.createSum(call.type.getSqlTypeName())); - break; - case "AVG": - aggregators.add(BeamBuiltinAggregations.createAvg(call.type.getSqlTypeName())); - break; - case "VAR_POP": - aggregators.add( - VarianceFn.newPopulation(BigDecimalConverter.forSqlType(fieldTypeName))); - break; - case "VAR_SAMP": - aggregators.add(VarianceFn.newSample(BigDecimalConverter.forSqlType(fieldTypeName))); - break; - case "COVAR_POP": - aggregators.add( - CovarianceFn.newPopulation(BigDecimalConverter.forSqlType(fieldTypeName))); - break; - case "COVAR_SAMP": - aggregators.add(CovarianceFn.newSample(BigDecimalConverter.forSqlType(fieldTypeName))); - break; - default: - if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { - // handle UDAF. - SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation(); - UdafImpl fn = (UdafImpl) udaf.function; - try { - aggregators.add(fn.getCombineFn()); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } else { - throw new UnsupportedOperationException( - String.format( - "Aggregator [%s] is not supported", call.getAggregation().getName())); - } - break; + Function> aggregatorFactory = + BUILTIN_AGGREGATOR_FACTORIES.get(aggregatorName); + + if (aggregatorFactory != null) { + return aggregatorFactory.apply(fieldTypeName); + } + + if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { + // handle UDAF. + SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation(); + UdafImpl fn = (UdafImpl) udaf.function; + try { + return fn.getCombineFn(); + } catch (Exception e) { + throw new IllegalStateException(e); } + } else { + throw new UnsupportedOperationException( + String.format("Aggregator [%s] is not supported", call.getAggregation().getName())); } - finalSchema = fields.build().stream().collect(toSchema()); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java index 6050350706df1..be2c32bbf999a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java @@ -17,23 +17,28 @@ */ package org.apache.beam.sdk.extensions.sql.impl.transform; +import com.google.common.collect.ImmutableMap; import java.math.BigDecimal; import java.math.MathContext; import java.math.RoundingMode; +import java.util.Map; +import java.util.function.Function; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.BigDecimalCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.CovarianceFn; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.VarianceFn; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; -import org.apache.calcite.sql.type.SqlTypeName; -import org.joda.time.ReadableInstant; /** * Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG/VAR_POP/VAR_SAMP. @@ -41,29 +46,40 @@ *

    TODO: Consider making the interface in terms of (1-column) rows. reuvenlax */ class BeamBuiltinAggregations { + + static final Map>> + BUILTIN_AGGREGATOR_FACTORIES = + ImmutableMap.>>builder() + .put("COUNT", typeName -> Count.combineFn()) + .put("MAX", BeamBuiltinAggregations::createMax) + .put("MIN", BeamBuiltinAggregations::createMin) + .put("SUM", BeamBuiltinAggregations::createSum) + .put("$SUM0", BeamBuiltinAggregations::createSum) + .put("AVG", BeamBuiltinAggregations::createAvg) + .put("VAR_POP", VarianceFn::newPopulation) + .put("VAR_SAMP", VarianceFn::newSample) + .put("COVAR_POP", CovarianceFn::newPopulation) + .put("COVAR_SAMP", CovarianceFn::newSample) + .build(); + private static MathContext mc = new MathContext(10, RoundingMode.HALF_UP); /** {@link CombineFn} for MAX based on {@link Max} and {@link Combine.BinaryCombineFn}. */ - public static CombineFn createMax(SqlTypeName fieldType) { + static CombineFn createMax(Schema.TypeName fieldType) { switch (fieldType) { case BOOLEAN: - return new CustMax(); - case INTEGER: + case INT16: + case BYTE: + case FLOAT: + case DATETIME: + case DECIMAL: + return new CustMax<>(); + case INT32: return Max.ofIntegers(); - case SMALLINT: - return new CustMax(); - case TINYINT: - return new CustMax(); - case BIGINT: + case INT64: return Max.ofLongs(); - case FLOAT: - return new CustMax(); case DOUBLE: return Max.ofDoubles(); - case TIMESTAMP: - return new CustMax(); - case DECIMAL: - return new CustMax(); default: throw new UnsupportedOperationException( String.format("[%s] is not support in MAX", fieldType)); @@ -71,26 +87,21 @@ public static CombineFn createMax(SqlTypeName fieldType) { } /** {@link CombineFn} for MIN based on {@link Min} and {@link Combine.BinaryCombineFn}. */ - public static CombineFn createMin(SqlTypeName fieldType) { + static CombineFn createMin(Schema.TypeName fieldType) { switch (fieldType) { case BOOLEAN: - return new CustMin(); - case INTEGER: + case BYTE: + case INT16: + case FLOAT: + case DATETIME: + case DECIMAL: + return new CustMin(); + case INT32: return Min.ofIntegers(); - case SMALLINT: - return new CustMin(); - case TINYINT: - return new CustMin(); - case BIGINT: + case INT64: return Min.ofLongs(); - case FLOAT: - return new CustMin(); case DOUBLE: return Min.ofDoubles(); - case TIMESTAMP: - return new CustMin(); - case DECIMAL: - return new CustMin(); default: throw new UnsupportedOperationException( String.format("[%s] is not support in MIN", fieldType)); @@ -98,15 +109,15 @@ public static CombineFn createMin(SqlTypeName fieldType) { } /** {@link CombineFn} for Sum based on {@link Sum} and {@link Combine.BinaryCombineFn}. */ - public static CombineFn createSum(SqlTypeName fieldType) { + static CombineFn createSum(Schema.TypeName fieldType) { switch (fieldType) { - case INTEGER: + case INT32: return Sum.ofIntegers(); - case SMALLINT: + case INT16: return new ShortSum(); - case TINYINT: + case BYTE: return new ByteSum(); - case BIGINT: + case INT64: return Sum.ofLongs(); case FLOAT: return new FloatSum(); @@ -121,15 +132,15 @@ public static CombineFn createSum(SqlTypeName fieldType) { } /** {@link CombineFn} for AVG. */ - public static CombineFn createAvg(SqlTypeName fieldType) { + static CombineFn createAvg(Schema.TypeName fieldType) { switch (fieldType) { - case INTEGER: + case INT32: return new IntegerAvg(); - case SMALLINT: + case INT16: return new ShortAvg(); - case TINYINT: + case BYTE: return new ByteAvg(); - case BIGINT: + case INT64: return new LongAvg(); case FLOAT: return new FloatAvg(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java index a9f884fd77f8c..cf92e0772afa0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java @@ -26,6 +26,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.extensions.sql.impl.utils.BigDecimalConverter; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.KV; @@ -51,12 +53,20 @@ public class CovarianceFn private boolean isSample; // flag to determine return value should be Covariance Pop or Sample private SerializableFunction decimalConverter; + public static CovarianceFn newPopulation(Schema.TypeName typeName) { + return newPopulation(BigDecimalConverter.forSqlType(typeName)); + } + public static CovarianceFn newPopulation( SerializableFunction decimalConverter) { return new CovarianceFn<>(POP, decimalConverter); } + public static CovarianceFn newSample(Schema.TypeName typeName) { + return newSample(BigDecimalConverter.forSqlType(typeName)); + } + public static CovarianceFn newSample( SerializableFunction decimalConverter) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java index d1ec5c55d54b2..f54795b34b2a1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java @@ -26,6 +26,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.extensions.sql.impl.utils.BigDecimalConverter; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.calcite.runtime.SqlFunctions; @@ -74,12 +76,20 @@ public class VarianceFn extends Combine.CombineFn decimalConverter; + public static VarianceFn newPopulation(Schema.TypeName typeName) { + return newPopulation(BigDecimalConverter.forSqlType(typeName)); + } + public static VarianceFn newPopulation( SerializableFunction decimalConverter) { return new VarianceFn<>(POP, decimalConverter); } + public static VarianceFn newSample(Schema.TypeName typeName) { + return newSample(BigDecimalConverter.forSqlType(typeName)); + } + public static VarianceFn newSample( SerializableFunction decimalConverter) { From 2649207825792ea9182e75ddb1f273ef2d4963d2 Mon Sep 17 00:00:00 2001 From: akedin Date: Fri, 19 Oct 2018 20:21:00 -0700 Subject: [PATCH 05/42] [SQL] Simplify AggregationRel --- .../sql/impl/rel/BeamAggregationRel.java | 88 +++++++++++-------- .../transform/BeamAggregationTransforms.java | 18 +--- .../sql/BeamSqlDslAggregationTest.java | 63 +++++++++++++ .../BeamAggregationTransformTest.java | 11 ++- 4 files changed, 123 insertions(+), 57 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index 36b7225532efb..acd9622faf086 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel; import static com.google.common.base.Preconditions.checkArgument; +import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED; @@ -113,10 +114,56 @@ public RelWriter explainTerms(RelWriter pw) { @Override public PTransform, PCollection> buildPTransform() { - return new Transform(); + return new Transform( + windowFn, windowFieldIndex, input, groupSet, getNamedAggCalls(), getRowType()); } - private class Transform extends PTransform, PCollection> { + private static class Transform extends PTransform, PCollection> { + + private final List keyFieldsIds; + private Schema outputSchema; + private Schema keySchema; + private SchemaCoder keyCoder; + private WindowFn windowFn; + private int windowFieldIndex; + private List> namedAggCalls; + private Schema inputSchema; + private SchemaCoder aggCoder; + + private Transform( + WindowFn windowFn, + int windowFieldIndex, + RelNode input, + ImmutableBitSet groupSet, + List> namedAggCalls, + RelDataType rowType) { + + this.windowFn = windowFn; + this.windowFieldIndex = windowFieldIndex; + this.namedAggCalls = namedAggCalls; + + this.inputSchema = CalciteUtils.toSchema(input.getRowType()); + this.outputSchema = CalciteUtils.toSchema(rowType); + + this.keySchema = + groupSet + .asList() + .stream() + .filter(i -> i != windowFieldIndex) + .map(inputSchema::getField) + .collect(toSchema()); + + this.keyFieldsIds = + groupSet.asList().stream().filter(i -> i != windowFieldIndex).collect(toList()); + + this.keyCoder = SchemaCoder.of(keySchema); + this.aggCoder = + SchemaCoder.of( + namedAggCalls + .stream() + .map(agg -> CalciteUtils.toField(agg.right, agg.left.getType())) + .collect(toSchema())); + } @Override public PCollection expand(PCollectionList pinput) { @@ -141,26 +188,21 @@ public PCollection expand(PCollectionList pinput) { validateWindowIsSupported(windowedStream); - Schema keySchema = exKeyFieldsSchema(input.getRowType()); - SchemaCoder keyCoder = SchemaCoder.of(keySchema); PCollection> exCombineByStream = windowedStream .apply( "exCombineBy", WithKeys.of( new BeamAggregationTransforms.AggregationGroupByKeyFn( - keySchema, windowFieldIndex, groupSet))) + keySchema, keyFieldsIds))) .setCoder(KvCoder.of(keyCoder, upstream.getCoder())); - SchemaCoder aggCoder = SchemaCoder.of(exAggFieldsSchema()); - PCollection> aggregatedStream = exCombineByStream .apply( "combineBy", Combine.perKey( - new BeamAggregationTransforms.AggregationAdaptor( - getNamedAggCalls(), CalciteUtils.toSchema(input.getRowType())))) + new BeamAggregationTransforms.AggregationAdaptor(namedAggCalls, inputSchema))) .setCoder(KvCoder.of(keyCoder, aggCoder)); PCollection mergedStream = @@ -168,8 +210,8 @@ public PCollection expand(PCollectionList pinput) { "mergeRecord", ParDo.of( new BeamAggregationTransforms.MergeAggregationRecord( - CalciteUtils.toSchema(getRowType()), windowFieldIndex))); - mergedStream.setRowSchema(CalciteUtils.toSchema(getRowType())); + outputSchema, windowFieldIndex))); + mergedStream.setRowSchema(outputSchema); return mergedStream; } @@ -196,30 +238,6 @@ private void validateWindowIsSupported(PCollection upstream) { + "See GroupByKey section in Beam Programming Guide"); } } - - /** Type of sub-rowrecord used as Group-By keys. */ - private Schema exKeyFieldsSchema(RelDataType relDataType) { - Schema inputSchema = CalciteUtils.toSchema(relDataType); - return groupSet - .asList() - .stream() - .filter(i -> i != windowFieldIndex) - .map(i -> newRowField(inputSchema, i)) - .collect(toSchema()); - } - - private Schema.Field newRowField(Schema schema, int i) { - return schema.getField(i); - } - - /** Type of sub-rowrecord, that represents the list of aggregation fields. */ - private Schema exAggFieldsSchema() { - return getNamedAggCalls().stream().map(this::newRowField).collect(toSchema()); - } - - private Schema.Field newRowField(Pair namedAggCall) { - return CalciteUtils.toField(namedAggCall.right, namedAggCall.left.getType()); - } } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index d19c6d9055141..643d789f00b04 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -21,9 +21,7 @@ import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.Row.toRow; -import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; @@ -32,7 +30,6 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.function.Function; import java.util.stream.IntStream; import org.apache.beam.sdk.coders.BigDecimalCoder; @@ -45,13 +42,9 @@ import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; -import org.apache.beam.sdk.extensions.sql.impl.transform.agg.CovarianceFn; -import org.apache.beam.sdk.extensions.sql.impl.transform.agg.VarianceFn; -import org.apache.beam.sdk.extensions.sql.impl.utils.BigDecimalConverter; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -60,7 +53,6 @@ import org.apache.beam.sdk.values.Row; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; -import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.Pair; import org.joda.time.Instant; @@ -98,14 +90,9 @@ public static class AggregationGroupByKeyFn implements SerializableFunction groupByKeys; - public AggregationGroupByKeyFn(Schema keySchema, int windowFieldIdx, ImmutableBitSet groupSet) { + public AggregationGroupByKeyFn(Schema keySchema, List groupByKeys) { this.keySchema = keySchema; - this.groupByKeys = new ArrayList<>(); - for (int i : groupSet.asList()) { - if (i != windowFieldIdx) { - groupByKeys.add(i); - } - } + this.groupByKeys = groupByKeys; } @Override @@ -119,7 +106,6 @@ public static class WindowTimestampFn implements SerializableFunction input = + pipeline.apply( + Create.of( + Row.withSchema(inputSchema) + .addValues("project1") + .addValue(1) + // .addValue(1) + .build(), + Row.withSchema(inputSchema) + .addValues("project1") + .addValue(2) + // .addValue(1) + .build(), + Row.withSchema(inputSchema) + .addValues("project2") + .addValue(4) + // .addValue(1) + .build(), + Row.withSchema(inputSchema) + .addValues("project1") + .addValue(13) + // .addValue(1) + .build(), + Row.withSchema(inputSchema) + .addValues("project3") + .addValue(1) + // .addValue(1) + .build()) + .withSchema( + inputSchema, + SerializableFunctions.identity(), + SerializableFunctions.identity())); + + Schema resultSchema = + Schema.builder() + // .addInt32Field("f_string") + .addInt32Field("f_int") + // .addInt32Field("f_anotherint") + .build(); + + PCollection result = + input.apply(SqlTransform.query("SELECT sum(f_int) FROM PCOLLECTION GROUP BY f_string")); + + PAssert.that(result) + .containsInAnyOrder( + Row.withSchema(resultSchema).addValues(16).build(), + Row.withSchema(resultSchema).addValues(4).build(), + Row.withSchema(resultSchema).addValues(1).build()); + + pipeline.run(); + } + private List rowsWithSingleIntField(String fieldName, List values) { return TestUtils.rowsBuilderOf(Schema.builder().addInt32Field(fieldName).build()) .addRows(values) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java index 28a8dab3df726..783ccbc5352f0 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java @@ -50,8 +50,8 @@ import org.apache.calcite.sql.fun.SqlSumAggFunction; import org.apache.calcite.sql.type.BasicSqlType; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.Pair; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -95,6 +95,7 @@ public class BeamAggregationTransformTest extends BeamTransformBaseTest { * * @throws ParseException */ + @Ignore @Test public void testCountPerElementBasic() throws ParseException { setupEnvironment(); @@ -106,11 +107,9 @@ public void testCountPerElementBasic() throws ParseException { // 1. extract fields in group-by key part PCollection> exGroupByStream = input - .apply( - "exGroupBy", - WithKeys.of( - new BeamAggregationTransforms.AggregationGroupByKeyFn( - keySchema, -1, ImmutableBitSet.of(0)))) + .apply("exGroupBy", WithKeys.of((Row row) -> row)) + // new BeamAggregationTransforms.AggregationGroupByKeyFn( + // keySchema, -1, ImmutableBitSet.of(0)))) .setCoder(KvCoder.of(keyCoder, inRecordCoder)); // 2. apply a GroupByKey. From 5fbc2b958a989a0ad00a8a2a068aa3cea56bd110 Mon Sep 17 00:00:00 2001 From: akedin Date: Fri, 19 Oct 2018 22:46:15 -0700 Subject: [PATCH 06/42] [SQL] Add AggregationCall wrapper --- .../sql/impl/rel/BeamAggregationRel.java | 25 +- .../transform/BeamAggregationTransforms.java | 82 ++- .../BeamAggregationTransformTest.java | 581 ------------------ 3 files changed, 69 insertions(+), 619 deletions(-) delete mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index acd9622faf086..89eefa40f1899 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -55,7 +55,6 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.Pair; import org.joda.time.Duration; /** {@link BeamRelNode} to replace a {@link Aggregate} node. */ @@ -115,7 +114,15 @@ public RelWriter explainTerms(RelWriter pw) { @Override public PTransform, PCollection> buildPTransform() { return new Transform( - windowFn, windowFieldIndex, input, groupSet, getNamedAggCalls(), getRowType()); + windowFn, + windowFieldIndex, + input, + groupSet, + getNamedAggCalls() + .stream() + .map(BeamAggregationTransforms.AggregationCall::of) + .collect(toList()), + getRowType()); } private static class Transform extends PTransform, PCollection> { @@ -126,7 +133,7 @@ private static class Transform extends PTransform, PCollect private SchemaCoder keyCoder; private WindowFn windowFn; private int windowFieldIndex; - private List> namedAggCalls; + private List aggregationCalls; private Schema inputSchema; private SchemaCoder aggCoder; @@ -135,12 +142,12 @@ private Transform( int windowFieldIndex, RelNode input, ImmutableBitSet groupSet, - List> namedAggCalls, + List aggregationCalls, RelDataType rowType) { this.windowFn = windowFn; this.windowFieldIndex = windowFieldIndex; - this.namedAggCalls = namedAggCalls; + this.aggregationCalls = aggregationCalls; this.inputSchema = CalciteUtils.toSchema(input.getRowType()); this.outputSchema = CalciteUtils.toSchema(rowType); @@ -159,10 +166,7 @@ private Transform( this.keyCoder = SchemaCoder.of(keySchema); this.aggCoder = SchemaCoder.of( - namedAggCalls - .stream() - .map(agg -> CalciteUtils.toField(agg.right, agg.left.getType())) - .collect(toSchema())); + aggregationCalls.stream().map(aggCall -> aggCall.field()).collect(toSchema())); } @Override @@ -202,7 +206,8 @@ public PCollection expand(PCollectionList pinput) { .apply( "combineBy", Combine.perKey( - new BeamAggregationTransforms.AggregationAdaptor(namedAggCalls, inputSchema))) + new BeamAggregationTransforms.AggregationAdaptor( + aggregationCalls, inputSchema))) .setCoder(KvCoder.of(keyCoder, aggCoder)); PCollection mergedStream = diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index 643d789f00b04..15be622e0a06a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -21,6 +21,7 @@ import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.Row.toRow; +import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; @@ -32,6 +33,7 @@ import java.util.List; import java.util.function.Function; import java.util.stream.IntStream; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.BigDecimalCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -115,6 +117,42 @@ public Instant apply(Row input) { } } + /** Wrapper for aggregation function call information. */ + @AutoValue + public abstract static class AggregationCall implements Serializable { + public abstract String functionName(); + + public abstract Schema.Field field(); + + public abstract List args(); + + public abstract @Nullable CombineFn udafCombineFn(); + + public static AggregationCall of(Pair callWithAlias) { + AggregateCall call = callWithAlias.getKey(); + String alias = callWithAlias.getValue(); + + return new AutoValue_BeamAggregationTransforms_AggregationCall( + call.getAggregation().getName(), + CalciteUtils.toField(alias, call.getType()), + call.getArgList(), + getUdafCombineFn(call)); + } + + private static @Nullable CombineFn getUdafCombineFn(AggregateCall call) { + if (!(call.getAggregation() instanceof SqlUserDefinedAggFunction)) { + return null; + } + + try { + return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) + .getCombineFn(); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + } + /** An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}. */ public static class AggregationAdaptor extends CombineFn { private List aggregators; @@ -122,62 +160,50 @@ public static class AggregationAdaptor extends CombineFn> aggregationCalls, Schema sourceSchema) { + public AggregationAdaptor(List aggregationCalls, Schema sourceSchema) { this.aggregators = new ArrayList<>(); this.sourceFieldExps = new ArrayList<>(); this.sourceSchema = sourceSchema; ImmutableList.Builder fields = ImmutableList.builder(); - for (Pair aggCall : aggregationCalls) { - AggregateCall call = aggCall.left; - String aggName = aggCall.right; + for (AggregationCall aggCall : aggregationCalls) { - if (call.getArgList().size() == 2) { + if (aggCall.args().size() == 2) { /* * handle the case of aggregation function has two parameters and use KV pair to bundle * two corresponding expressions. */ - int refIndexKey = call.getArgList().get(0); - int refIndexValue = call.getArgList().get(1); + int refIndexKey = aggCall.args().get(0); + int refIndexValue = aggCall.args().get(1); sourceFieldExps.add(KV.of(refIndexKey, refIndexValue)); } else { - Integer refIndex = call.getArgList().size() > 0 ? call.getArgList().get(0) : 0; + Integer refIndex = aggCall.args().size() > 0 ? aggCall.args().get(0) : 0; sourceFieldExps.add(refIndex); } - Schema.Field field = CalciteUtils.toField(aggName, call.getType()); - fields.add(field); - aggregators.add( - createAggregator(call, call.getAggregation().getName(), field.getType().getTypeName())); + fields.add(aggCall.field()); + aggregators.add(createAggregator(aggCall, aggCall.field().getType().getTypeName())); } finalSchema = fields.build().stream().collect(toSchema()); } private CombineFn createAggregator( - AggregateCall call, String aggregatorName, Schema.TypeName fieldTypeName) { + AggregationCall aggCall, Schema.TypeName fieldTypeName) { + + if (aggCall.udafCombineFn() != null) { + return aggCall.udafCombineFn(); + } Function> aggregatorFactory = - BUILTIN_AGGREGATOR_FACTORIES.get(aggregatorName); + BUILTIN_AGGREGATOR_FACTORIES.get(aggCall.functionName()); if (aggregatorFactory != null) { return aggregatorFactory.apply(fieldTypeName); } - if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { - // handle UDAF. - SqlUserDefinedAggFunction udaf = (SqlUserDefinedAggFunction) call.getAggregation(); - UdafImpl fn = (UdafImpl) udaf.function; - try { - return fn.getCombineFn(); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } else { - throw new UnsupportedOperationException( - String.format("Aggregator [%s] is not supported", call.getAggregation().getName())); - } + throw new UnsupportedOperationException( + String.format("Aggregator [%s] is not supported", aggCall.functionName())); } @Override diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java deleted file mode 100644 index 783ccbc5352f0..0000000000000 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamAggregationTransformTest.java +++ /dev/null @@ -1,581 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.extensions.sql.impl.schema.transform; - -import com.google.common.collect.Lists; -import java.text.ParseException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunctions; -import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.Row; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.fun.SqlAvgAggFunction; -import org.apache.calcite.sql.fun.SqlCountAggFunction; -import org.apache.calcite.sql.fun.SqlMinMaxAggFunction; -import org.apache.calcite.sql.fun.SqlSumAggFunction; -import org.apache.calcite.sql.type.BasicSqlType; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.util.Pair; -import org.junit.Ignore; -import org.junit.Rule; -import org.junit.Test; - -/** Unit tests for {@link BeamAggregationTransforms}. */ -public class BeamAggregationTransformTest extends BeamTransformBaseTest { - - @Rule public TestPipeline p = TestPipeline.create(); - - private List> aggCalls; - - private Schema keyType; - private Schema aggPartType; - private Schema outputType; - - private Coder inRecordCoder; - private Coder keyCoder; - private Coder aggCoder; - - /** - * This step equals to below query. - * - *

    -   * SELECT `f_int`
    -   * , COUNT(*) AS `size`
    -   * , SUM(`f_long`) AS `sum1`, AVG(`f_long`) AS `avg1`
    -   * , MAX(`f_long`) AS `max1`, MIN(`f_long`) AS `min1`
    -   * , SUM(`f_short`) AS `sum2`, AVG(`f_short`) AS `avg2`
    -   * , MAX(`f_short`) AS `max2`, MIN(`f_short`) AS `min2`
    -   * , SUM(`f_byte`) AS `sum3`, AVG(`f_byte`) AS `avg3`
    -   * , MAX(`f_byte`) AS `max3`, MIN(`f_byte`) AS `min3`
    -   * , SUM(`f_float`) AS `sum4`, AVG(`f_float`) AS `avg4`
    -   * , MAX(`f_float`) AS `max4`, MIN(`f_float`) AS `min4`
    -   * , SUM(`f_double`) AS `sum5`, AVG(`f_double`) AS `avg5`
    -   * , MAX(`f_double`) AS `max5`, MIN(`f_double`) AS `min5`
    -   * , MAX(`f_timestamp`) AS `max7`, MIN(`f_timestamp`) AS `min7`
    -   * ,SUM(`f_int2`) AS `sum8`, AVG(`f_int2`) AS `avg8`
    -   * , MAX(`f_int2`) AS `max8`, MIN(`f_int2`) AS `min8`
    -   * FROM TABLE_NAME
    -   * GROUP BY `f_int`
    -   * 
    - * - * @throws ParseException - */ - @Ignore - @Test - public void testCountPerElementBasic() throws ParseException { - setupEnvironment(); - - PCollection input = p.apply(Create.of(inputRows)); - - Schema keySchema = - Schema.builder().addFields(Lists.newArrayList(inputSchema.getField(0))).build(); - // 1. extract fields in group-by key part - PCollection> exGroupByStream = - input - .apply("exGroupBy", WithKeys.of((Row row) -> row)) - // new BeamAggregationTransforms.AggregationGroupByKeyFn( - // keySchema, -1, ImmutableBitSet.of(0)))) - .setCoder(KvCoder.of(keyCoder, inRecordCoder)); - - // 2. apply a GroupByKey. - PCollection>> groupedStream = - exGroupByStream - .apply("groupBy", GroupByKey.create()) - .setCoder(KvCoder.of(keyCoder, IterableCoder.of(inRecordCoder))); - - // 3. run aggregation functions - PCollection> aggregatedStream = - groupedStream - .apply( - "aggregation", - Combine.groupedValues( - new BeamAggregationTransforms.AggregationAdaptor(aggCalls, inputSchema))) - .setCoder(KvCoder.of(keyCoder, aggCoder)); - - // 4. flat KV to a single record - PCollection mergedStream = - aggregatedStream.apply( - "mergeRecord", - ParDo.of(new BeamAggregationTransforms.MergeAggregationRecord(outputType, -1))); - mergedStream.setRowSchema(outputType); - - // assert function BeamAggregationTransform.AggregationGroupByKeyFn - PAssert.that(exGroupByStream).containsInAnyOrder(prepareResultOfAggregationGroupByKeyFn()); - - // assert BeamAggregationTransform.AggregationCombineFn - PAssert.that(aggregatedStream).containsInAnyOrder(prepareResultOfAggregationCombineFn()); - - // assert BeamAggregationTransform.MergeAggregationRecord - PAssert.that(mergedStream).containsInAnyOrder(prepareResultOfMergeAggregationRow()); - - p.run(); - } - - private void setupEnvironment() { - prepareAggregationCalls(); - prepareTypeAndCoder(); - } - - /** create list of all {@link AggregateCall}. */ - @SuppressWarnings("deprecation") - private void prepareAggregationCalls() { - // aggregations for all data type - aggCalls = new ArrayList<>(); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlCountAggFunction("COUNT"), - false, - Arrays.asList(), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT), - "count"), - "count")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT)), - false, - Arrays.asList(1), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT), - "sum1"), - "sum1")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(1), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT), - "avg1"), - "avg1")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(1), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT), - "max1"), - "max1")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(1), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.BIGINT), - "min1"), - "min1")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT)), - false, - Arrays.asList(2), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT), - "sum2"), - "sum2")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(2), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT), - "avg2"), - "avg2")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(2), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT), - "max2"), - "max2")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(2), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.SMALLINT), - "min2"), - "min2")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT)), - false, - Arrays.asList(3), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT), - "sum3"), - "sum3")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(3), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT), - "avg3"), - "avg3")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(3), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT), - "max3"), - "max3")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(3), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TINYINT), - "min3"), - "min3")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT)), - false, - Arrays.asList(4), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT), - "sum4"), - "sum4")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(4), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT), - "avg4"), - "avg4")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(4), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT), - "max4"), - "max4")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(4), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.FLOAT), - "min4"), - "min4")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE)), - false, - Arrays.asList(5), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE), - "sum5"), - "sum5")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(5), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE), - "avg5"), - "avg5")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(5), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE), - "max5"), - "max5")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(5), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.DOUBLE), - "min5"), - "min5")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(7), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP), - "max7"), - "max7")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(7), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.TIMESTAMP), - "min7"), - "min7")); - - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlSumAggFunction( - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER)), - false, - Arrays.asList(8), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER), - "sum8"), - "sum8")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlAvgAggFunction(SqlKind.AVG), - false, - Arrays.asList(8), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER), - "avg8"), - "avg8")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MAX), - false, - Arrays.asList(8), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER), - "max8"), - "max8")); - aggCalls.add( - Pair.of( - new AggregateCall( - new SqlMinMaxAggFunction(SqlKind.MIN), - false, - Arrays.asList(8), - new BasicSqlType(RelDataTypeSystem.DEFAULT, SqlTypeName.INTEGER), - "min8"), - "min8")); - } - - /** Coders used in aggregation steps. */ - private void prepareTypeAndCoder() { - inRecordCoder = - SchemaCoder.of( - inputSchema, SerializableFunctions.identity(), SerializableFunctions.identity()); - - keyType = Schema.builder().addInt32Field("f_int").build(); - - keyCoder = - SchemaCoder.of(keyType, SerializableFunctions.identity(), SerializableFunctions.identity()); - - aggPartType = - Schema.builder() - .addInt64Field("count") - .addInt64Field("sum1") - .addInt64Field("avg1") - .addInt64Field("max1") - .addInt64Field("min1") - .addInt16Field("sum2") - .addInt16Field("avg2") - .addInt16Field("max2") - .addInt16Field("min2") - .addByteField("sum3") - .addByteField("avg3") - .addByteField("max3") - .addByteField("min3") - .addFloatField("sum4") - .addFloatField("avg4") - .addFloatField("max4") - .addFloatField("min4") - .addDoubleField("sum5") - .addDoubleField("avg5") - .addDoubleField("max5") - .addDoubleField("min5") - .addDateTimeField("max7") - .addDateTimeField("min7") - .addInt32Field("sum8") - .addInt32Field("avg8") - .addInt32Field("max8") - .addInt32Field("min8") - .build(); - - aggCoder = - SchemaCoder.of( - aggPartType, SerializableFunctions.identity(), SerializableFunctions.identity()); - - outputType = prepareFinalSchema(); - } - - /** expected results after {@link BeamAggregationTransforms.AggregationGroupByKeyFn}. */ - private List> prepareResultOfAggregationGroupByKeyFn() { - return IntStream.range(0, 4) - .mapToObj( - i -> - KV.of( - Row.withSchema(keyType).addValues(inputRows.get(i).getInt32(0)).build(), - inputRows.get(i))) - .collect(Collectors.toList()); - } - - /** expected results. */ - private List> prepareResultOfAggregationCombineFn() throws ParseException { - return Arrays.asList( - KV.of( - Row.withSchema(keyType).addValues(inputRows.get(0).getInt32(0)).build(), - Row.withSchema(aggPartType) - .addValues( - 4L, - 10000L, - 2500L, - 4000L, - 1000L, - (short) 10, - (short) 2, - (short) 4, - (short) 1, - (byte) 10, - (byte) 2, - (byte) 4, - (byte) 1, - 10.0F, - 2.5F, - 4.0F, - 1.0F, - 10.0, - 2.5, - 4.0, - 1.0, - FORMAT.parseDateTime("2017-01-01 02:04:03"), - FORMAT.parseDateTime("2017-01-01 01:01:03"), - 10, - 2, - 4, - 1) - .build())); - } - - /** Row type of final output row. */ - private Schema prepareFinalSchema() { - return Schema.builder() - .addInt32Field("f_int") - .addInt64Field("count") - .addInt64Field("sum1") - .addInt64Field("avg1") - .addInt64Field("max1") - .addInt64Field("min1") - .addInt16Field("sum2") - .addInt16Field("avg2") - .addInt16Field("max2") - .addInt16Field("min2") - .addByteField("sum3") - .addByteField("avg3") - .addByteField("max3") - .addByteField("min3") - .addFloatField("sum4") - .addFloatField("avg4") - .addFloatField("max4") - .addFloatField("min4") - .addDoubleField("sum5") - .addDoubleField("avg5") - .addDoubleField("max5") - .addDoubleField("min5") - .addDateTimeField("max7") - .addDateTimeField("min7") - .addInt32Field("sum8") - .addInt32Field("avg8") - .addInt32Field("max8") - .addInt32Field("min8") - .build(); - } - - /** expected results after {@link BeamAggregationTransforms.MergeAggregationRecord}. */ - private Row prepareResultOfMergeAggregationRow() throws ParseException { - return Row.withSchema(outputType) - .addValues( - 1, - 4L, - 10000L, - 2500L, - 4000L, - 1000L, - (short) 10, - (short) 2, - (short) 4, - (short) 1, - (byte) 10, - (byte) 2, - (byte) 4, - (byte) 1, - 10.0F, - 2.5F, - 4.0F, - 1.0F, - 10.0, - 2.5, - 4.0, - 1.0, - FORMAT.parseDateTime("2017-01-01 02:04:03"), - FORMAT.parseDateTime("2017-01-01 01:01:03"), - 10, - 2, - 4, - 1) - .build(); - } -} From a998596eb8d26922e5de200a88d94dcfa8e787cf Mon Sep 17 00:00:00 2001 From: akedin Date: Fri, 19 Oct 2018 22:59:58 -0700 Subject: [PATCH 07/42] [SQL] Inline aggregation rel helper transforms --- .../sql/impl/rel/BeamAggregationRel.java | 7 ++-- .../transform/BeamAggregationTransforms.java | 32 ------------------- 2 files changed, 3 insertions(+), 36 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index 89eefa40f1899..e17bc8af6cd2e 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -21,6 +21,7 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED; +import static org.apache.beam.sdk.values.Row.toRow; import java.util.List; import org.apache.beam.sdk.coders.KvCoder; @@ -183,8 +184,7 @@ public PCollection expand(PCollectionList pinput) { upstream .apply( "assignEventTimestamp", - WithTimestamps.of( - new BeamAggregationTransforms.WindowTimestampFn(windowFieldIndex)) + WithTimestamps.of(row -> row.getDateTime(windowFieldIndex).toInstant()) .withAllowedTimestampSkew(new Duration(Long.MAX_VALUE))) .setCoder(upstream.getCoder()); windowedStream = upstream.apply(Window.into(windowFn)); @@ -197,8 +197,7 @@ public PCollection expand(PCollectionList pinput) { .apply( "exCombineBy", WithKeys.of( - new BeamAggregationTransforms.AggregationGroupByKeyFn( - keySchema, keyFieldsIds))) + row -> keyFieldsIds.stream().map(row::getValue).collect(toRow(keySchema)))) .setCoder(KvCoder.of(keyCoder, upstream.getCoder())); PCollection> aggregatedStream = diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index 15be622e0a06a..3daf3d238b292 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -48,7 +48,6 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.values.KV; @@ -56,7 +55,6 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; import org.apache.calcite.util.Pair; -import org.joda.time.Instant; /** Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation. */ public class BeamAggregationTransforms implements Serializable { @@ -87,36 +85,6 @@ public void processElement(ProcessContext c, BoundedWindow window) { } } - /** extract group-by fields. */ - public static class AggregationGroupByKeyFn implements SerializableFunction { - private Schema keySchema; - private List groupByKeys; - - public AggregationGroupByKeyFn(Schema keySchema, List groupByKeys) { - this.keySchema = keySchema; - this.groupByKeys = groupByKeys; - } - - @Override - public Row apply(Row input) { - return groupByKeys.stream().map(input::getValue).collect(toRow(keySchema)); - } - } - - /** Assign event timestamp. */ - public static class WindowTimestampFn implements SerializableFunction { - private int windowFieldIdx = -1; - - public WindowTimestampFn(int windowFieldIdx) { - this.windowFieldIdx = windowFieldIdx; - } - - @Override - public Instant apply(Row input) { - return new Instant(input.getDateTime(windowFieldIdx)); - } - } - /** Wrapper for aggregation function call information. */ @AutoValue public abstract static class AggregationCall implements Serializable { From 8f7608eb84dc553a3990bb864b2d530dc1b0a0dc Mon Sep 17 00:00:00 2001 From: akedin Date: Sat, 20 Oct 2018 14:18:02 -0700 Subject: [PATCH 08/42] [SQL] Move CombineFn creation to AggregationCall constructor --- .../transform/BeamAggregationTransforms.java | 89 ++++++++++++------- 1 file changed, 59 insertions(+), 30 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index 3daf3d238b292..a88792b684618 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -85,7 +85,10 @@ public void processElement(ProcessContext c, BoundedWindow window) { } } - /** Wrapper for aggregation function call information. */ + /** + * Wrapper for aggregation function call. This is needed to avoid dealing with non-serializable + * Calcite classes. + */ @AutoValue public abstract static class AggregationCall implements Serializable { public abstract String functionName(); @@ -94,24 +97,39 @@ public abstract static class AggregationCall implements Serializable { public abstract List args(); - public abstract @Nullable CombineFn udafCombineFn(); + public abstract @Nullable CombineFn combineFn(); + + public abstract Builder toBuilder(); + + public static Builder builder() { + return new AutoValue_BeamAggregationTransforms_AggregationCall.Builder(); + } public static AggregationCall of(Pair callWithAlias) { AggregateCall call = callWithAlias.getKey(); - String alias = callWithAlias.getValue(); + Schema.Field field = CalciteUtils.toField(callWithAlias.getValue(), call.getType()); + String functionName = call.getAggregation().getName(); - return new AutoValue_BeamAggregationTransforms_AggregationCall( - call.getAggregation().getName(), - CalciteUtils.toField(alias, call.getType()), - call.getArgList(), - getUdafCombineFn(call)); + Builder builder = + builder() + .setFunctionName(functionName) + .setArgs(call.getArgList()) + .setField(field) + .setCombineFn(createCombineFn(call, field, functionName)); + + return builder.build(); } - private static @Nullable CombineFn getUdafCombineFn(AggregateCall call) { - if (!(call.getAggregation() instanceof SqlUserDefinedAggFunction)) { - return null; + private static CombineFn createCombineFn( + AggregateCall call, Schema.Field field, String functionName) { + if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { + return getUdafCombineFn(call); } + return createAggregator(functionName, field.getType().getTypeName()); + } + + private static @Nullable CombineFn getUdafCombineFn(AggregateCall call) { try { return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) .getCombineFn(); @@ -119,6 +137,34 @@ public static AggregationCall of(Pair callWithAlias) { throw new IllegalStateException(e); } } + + private static CombineFn createAggregator( + String functionName, Schema.TypeName fieldTypeName) { + + Function> aggregatorFactory = + BUILTIN_AGGREGATOR_FACTORIES.get(functionName); + + if (aggregatorFactory != null) { + return aggregatorFactory.apply(fieldTypeName); + } + + throw new UnsupportedOperationException( + String.format("Aggregator [%s] is not supported", functionName)); + } + + /** Builder for AggregationCall. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setFunctionName(String functionName); + + public abstract Builder setField(Schema.Field field); + + public abstract Builder setArgs(List args); + + public abstract Builder setCombineFn(@Nullable CombineFn combineFn); + + public abstract AggregationCall build(); + } } /** An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}. */ @@ -132,6 +178,7 @@ public AggregationAdaptor(List aggregationCalls, Schema sourceS this.aggregators = new ArrayList<>(); this.sourceFieldExps = new ArrayList<>(); this.sourceSchema = sourceSchema; + ImmutableList.Builder fields = ImmutableList.builder(); for (AggregationCall aggCall : aggregationCalls) { @@ -151,29 +198,11 @@ public AggregationAdaptor(List aggregationCalls, Schema sourceS } fields.add(aggCall.field()); - aggregators.add(createAggregator(aggCall, aggCall.field().getType().getTypeName())); + aggregators.add(aggCall.combineFn()); } finalSchema = fields.build().stream().collect(toSchema()); } - private CombineFn createAggregator( - AggregationCall aggCall, Schema.TypeName fieldTypeName) { - - if (aggCall.udafCombineFn() != null) { - return aggCall.udafCombineFn(); - } - - Function> aggregatorFactory = - BUILTIN_AGGREGATOR_FACTORIES.get(aggCall.functionName()); - - if (aggregatorFactory != null) { - return aggregatorFactory.apply(fieldTypeName); - } - - throw new UnsupportedOperationException( - String.format("Aggregator [%s] is not supported", aggCall.functionName())); - } - @Override public AggregationAccumulator createAccumulator() { AggregationAccumulator initialAccu = new AggregationAccumulator(); From 6ea49ad140951297d7ff0f47cd80942dbbe93fd7 Mon Sep 17 00:00:00 2001 From: akedin Date: Sat, 20 Oct 2018 22:24:14 -0700 Subject: [PATCH 09/42] [SQL] Split and rename Aggregation CombineFn wrappers --- .../sql/impl/rel/BeamAggregationRel.java | 33 +-- .../transform/BeamAggregationTransforms.java | 258 +++--------------- .../transform/BeamBuiltinAggregations.java | 4 +- .../agg/AggregationCombineFnAdapter.java | 177 ++++++++++++ 4 files changed, 238 insertions(+), 234 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index e17bc8af6cd2e..e98605354a4f0 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -26,6 +26,7 @@ import java.util.List; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; @@ -54,7 +55,6 @@ import org.apache.calcite.rel.RelWriter; import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableBitSet; import org.joda.time.Duration; @@ -114,16 +114,16 @@ public RelWriter explainTerms(RelWriter pw) { @Override public PTransform, PCollection> buildPTransform() { - return new Transform( - windowFn, - windowFieldIndex, - input, - groupSet, + Schema inputSchema = CalciteUtils.toSchema(getInput().getRowType()); + Schema outputSchema = CalciteUtils.toSchema(getRowType()); + List aggregationAdapters = getNamedAggCalls() .stream() - .map(BeamAggregationTransforms.AggregationCall::of) - .collect(toList()), - getRowType()); + .map(aggCall -> AggregationCombineFnAdapter.of(aggCall, inputSchema)) + .collect(toList()); + + return new Transform( + windowFn, windowFieldIndex, inputSchema, getGroupSet(), aggregationAdapters, outputSchema); } private static class Transform extends PTransform, PCollection> { @@ -134,24 +134,22 @@ private static class Transform extends PTransform, PCollect private SchemaCoder keyCoder; private WindowFn windowFn; private int windowFieldIndex; - private List aggregationCalls; - private Schema inputSchema; + private List aggregationCalls; private SchemaCoder aggCoder; private Transform( WindowFn windowFn, int windowFieldIndex, - RelNode input, + Schema inputSchema, ImmutableBitSet groupSet, - List aggregationCalls, - RelDataType rowType) { + List aggregationCalls, + Schema outputSchema) { this.windowFn = windowFn; this.windowFieldIndex = windowFieldIndex; this.aggregationCalls = aggregationCalls; - this.inputSchema = CalciteUtils.toSchema(input.getRowType()); - this.outputSchema = CalciteUtils.toSchema(rowType); + this.outputSchema = outputSchema; this.keySchema = groupSet @@ -205,8 +203,7 @@ public PCollection expand(PCollectionList pinput) { .apply( "combineBy", Combine.perKey( - new BeamAggregationTransforms.AggregationAdaptor( - aggregationCalls, inputSchema))) + new BeamAggregationTransforms.MultiAggregationCombineFn(aggregationCalls))) .setCoder(KvCoder.of(keyCoder, aggCoder)); PCollection mergedStream = diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java index a88792b684618..ff85ccfe96f9f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java @@ -17,12 +17,9 @@ */ package org.apache.beam.sdk.extensions.sql.impl.transform; -import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.apache.beam.sdk.values.Row.toRow; -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; @@ -31,20 +28,16 @@ import java.math.BigDecimal; import java.util.ArrayList; import java.util.List; -import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.IntStream; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.BigDecimalCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; -import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; @@ -52,9 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; -import org.apache.calcite.util.Pair; /** Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation. */ public class BeamAggregationTransforms implements Serializable { @@ -85,237 +75,77 @@ public void processElement(ProcessContext c, BoundedWindow window) { } } - /** - * Wrapper for aggregation function call. This is needed to avoid dealing with non-serializable - * Calcite classes. - */ - @AutoValue - public abstract static class AggregationCall implements Serializable { - public abstract String functionName(); - - public abstract Schema.Field field(); - - public abstract List args(); - - public abstract @Nullable CombineFn combineFn(); - - public abstract Builder toBuilder(); - - public static Builder builder() { - return new AutoValue_BeamAggregationTransforms_AggregationCall.Builder(); - } - - public static AggregationCall of(Pair callWithAlias) { - AggregateCall call = callWithAlias.getKey(); - Schema.Field field = CalciteUtils.toField(callWithAlias.getValue(), call.getType()); - String functionName = call.getAggregation().getName(); - - Builder builder = - builder() - .setFunctionName(functionName) - .setArgs(call.getArgList()) - .setField(field) - .setCombineFn(createCombineFn(call, field, functionName)); - - return builder.build(); - } - - private static CombineFn createCombineFn( - AggregateCall call, Schema.Field field, String functionName) { - if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { - return getUdafCombineFn(call); - } - - return createAggregator(functionName, field.getType().getTypeName()); - } - - private static @Nullable CombineFn getUdafCombineFn(AggregateCall call) { - try { - return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) - .getCombineFn(); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - private static CombineFn createAggregator( - String functionName, Schema.TypeName fieldTypeName) { - - Function> aggregatorFactory = - BUILTIN_AGGREGATOR_FACTORIES.get(functionName); - - if (aggregatorFactory != null) { - return aggregatorFactory.apply(fieldTypeName); - } - - throw new UnsupportedOperationException( - String.format("Aggregator [%s] is not supported", functionName)); - } - - /** Builder for AggregationCall. */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setFunctionName(String functionName); - - public abstract Builder setField(Schema.Field field); - - public abstract Builder setArgs(List args); - - public abstract Builder setCombineFn(@Nullable CombineFn combineFn); - - public abstract AggregationCall build(); - } - } - /** An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}. */ - public static class AggregationAdaptor extends CombineFn { - private List aggregators; - private List sourceFieldExps; - private Schema sourceSchema; + public static class MultiAggregationCombineFn extends CombineFn, Row> { + private List aggCombineFns; private Schema finalSchema; - public AggregationAdaptor(List aggregationCalls, Schema sourceSchema) { - this.aggregators = new ArrayList<>(); - this.sourceFieldExps = new ArrayList<>(); - this.sourceSchema = sourceSchema; - - ImmutableList.Builder fields = ImmutableList.builder(); - - for (AggregationCall aggCall : aggregationCalls) { - - if (aggCall.args().size() == 2) { - /* - * handle the case of aggregation function has two parameters and use KV pair to bundle - * two corresponding expressions. - */ - int refIndexKey = aggCall.args().get(0); - int refIndexValue = aggCall.args().get(1); - - sourceFieldExps.add(KV.of(refIndexKey, refIndexValue)); - } else { - Integer refIndex = aggCall.args().size() > 0 ? aggCall.args().get(0) : 0; - sourceFieldExps.add(refIndex); - } - - fields.add(aggCall.field()); - aggregators.add(aggCall.combineFn()); - } - finalSchema = fields.build().stream().collect(toSchema()); + public MultiAggregationCombineFn(List aggCombineFns) { + this.aggCombineFns = aggCombineFns; + finalSchema = + this.aggCombineFns.stream().map(AggregationCombineFnAdapter::field).collect(toSchema()); } @Override - public AggregationAccumulator createAccumulator() { - AggregationAccumulator initialAccu = new AggregationAccumulator(); - for (CombineFn agg : aggregators) { - initialAccu.accumulatorElements.add(agg.createAccumulator()); - } - return initialAccu; + public List createAccumulator() { + return aggCombineFns + .stream() + .map(AggregationCombineFnAdapter::createAccumulator) + .collect(Collectors.toList()); } @Override - public AggregationAccumulator addInput(AggregationAccumulator accumulator, Row input) { - AggregationAccumulator deltaAcc = new AggregationAccumulator(); - for (int idx = 0; idx < aggregators.size(); ++idx) { - CombineFn aggregator = aggregators.get(idx); - Object element = accumulator.accumulatorElements.get(idx); + public List addInput(List accumulators, Row input) { + List deltaAcc = new ArrayList<>(); - if (sourceFieldExps.get(idx) instanceof Integer) { - Object value = input.getValue((Integer) sourceFieldExps.get(idx)); + for (int idx = 0; idx < aggCombineFns.size(); ++idx) { + AggregationCombineFnAdapter aggregator = aggCombineFns.get(idx); + Object aggregatorAccumulator = accumulators.get(idx); - // every aggregator ignores null values, e.g., COUNT(NULL) is always zero - if (value != null) { - Object delta = aggregator.addInput(element, value); - deltaAcc.accumulatorElements.add(delta); - } else { - deltaAcc.accumulatorElements.add(element); - } - } else if (sourceFieldExps.get(idx) instanceof KV) { - /* - * If source expression is type of KV pair, we bundle the value of two expressions into KV - * pair and pass it to aggregator's addInput method. - */ - KV exp = (KV) sourceFieldExps.get(idx); - - Object key = input.getValue(exp.getKey()); - - Object value = input.getValue(exp.getValue()); - - // ignore aggregator if either key or value is null, e.g., COVAR_SAMP(x, NULL) is null - if (key != null && value != null) { - deltaAcc.accumulatorElements.add(aggregator.addInput(element, KV.of(key, value))); - } else { - deltaAcc.accumulatorElements.add(element); - } - } + Object newAccumulator = aggregator.addInput(aggregatorAccumulator, input); + deltaAcc.add(newAccumulator); } return deltaAcc; } @Override - public AggregationAccumulator mergeAccumulators(Iterable accumulators) { - AggregationAccumulator deltaAcc = new AggregationAccumulator(); - for (int idx = 0; idx < aggregators.size(); ++idx) { + public List mergeAccumulators(Iterable> accumulators) { + List deltaAcc = new ArrayList<>(); + for (int idx = 0; idx < aggCombineFns.size(); ++idx) { List accs = new ArrayList<>(); - for (AggregationAccumulator accumulator : accumulators) { - accs.add(accumulator.accumulatorElements.get(idx)); + for (List accumulator : accumulators) { + accs.add(accumulator.get(idx)); } - deltaAcc.accumulatorElements.add(aggregators.get(idx).mergeAccumulators(accs)); + deltaAcc.add(aggCombineFns.get(idx).mergeAccumulators(accs)); } return deltaAcc; } @Override - public Row extractOutput(AggregationAccumulator accumulator) { - return IntStream.range(0, aggregators.size()) - .mapToObj(idx -> getAggregatorOutput(accumulator, idx)) + public Row extractOutput(List accumulator) { + return IntStream.range(0, aggCombineFns.size()) + .mapToObj(idx -> aggCombineFns.get(idx).extractOutput(accumulator.get(idx))) .collect(toRow(finalSchema)); } - private Object getAggregatorOutput(AggregationAccumulator accumulator, int idx) { - return aggregators.get(idx).extractOutput(accumulator.accumulatorElements.get(idx)); - } - @Override - public Coder getAccumulatorCoder( - CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) + throws CannotProvideCoderException { // TODO: Doing this here is wrong. registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of()); - List aggAccuCoderList = new ArrayList<>(); - for (int idx = 0; idx < aggregators.size(); ++idx) { - if (sourceFieldExps.get(idx) instanceof Integer) { - int srcFieldIndex = (Integer) sourceFieldExps.get(idx); - Coder srcFieldCoder = - RowCoder.coderForFieldType(sourceSchema.getField(srcFieldIndex).getType()); - aggAccuCoderList.add(aggregators.get(idx).getAccumulatorCoder(registry, srcFieldCoder)); - } else if (sourceFieldExps.get(idx) instanceof KV) { - // extract coder of two expressions separately. - KV exp = (KV) sourceFieldExps.get(idx); - int srcFieldIndexKey = exp.getKey(); - int srcFieldIndexValue = exp.getValue(); - - Coder srcFieldCoderKey = - RowCoder.coderForFieldType(sourceSchema.getField(srcFieldIndexKey).getType()); - Coder srcFieldCoderValue = - RowCoder.coderForFieldType(sourceSchema.getField(srcFieldIndexValue).getType()); + List aggAccuCoderList = new ArrayList<>(); - aggAccuCoderList.add( - aggregators - .get(idx) - .getAccumulatorCoder(registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue))); - } + for (AggregationCombineFnAdapter aggCombineFn : aggCombineFns) { + aggAccuCoderList.add(aggCombineFn.getAccumulatorCoder(registry, inputCoder)); } + return new AggregationAccumulatorCoder(aggAccuCoderList); } } - /** A class to holder varied accumulator objects. */ - public static class AggregationAccumulator { - private List accumulatorElements = new ArrayList<>(); - } - - /** Coder for {@link AggregationAccumulator}. */ - public static class AggregationAccumulatorCoder extends CustomCoder { + /** Coder for accumulators. */ + public static class AggregationAccumulatorCoder extends CustomCoder> { private VarIntCoder sizeCoder = VarIntCoder.of(); private List elementCoders; @@ -324,19 +154,19 @@ public AggregationAccumulatorCoder(List elementCoders) { } @Override - public void encode(AggregationAccumulator value, OutputStream outStream) throws IOException { - sizeCoder.encode(value.accumulatorElements.size(), outStream); - for (int idx = 0; idx < value.accumulatorElements.size(); ++idx) { - elementCoders.get(idx).encode(value.accumulatorElements.get(idx), outStream); + public void encode(List value, OutputStream outStream) throws IOException { + sizeCoder.encode(value.size(), outStream); + for (int idx = 0; idx < value.size(); ++idx) { + elementCoders.get(idx).encode(value.get(idx), outStream); } } @Override - public AggregationAccumulator decode(InputStream inStream) throws CoderException, IOException { - AggregationAccumulator accu = new AggregationAccumulator(); + public List decode(InputStream inStream) throws CoderException, IOException { + List accu = new ArrayList<>(); int size = sizeCoder.decode(inStream); for (int idx = 0; idx < size; ++idx) { - accu.accumulatorElements.add(elementCoders.get(idx).decode(inStream)); + accu.add(elementCoders.get(idx).decode(inStream)); } return accu; } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java index be2c32bbf999a..0172ebf7fe336 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java @@ -45,9 +45,9 @@ * *

    TODO: Consider making the interface in terms of (1-column) rows. reuvenlax */ -class BeamBuiltinAggregations { +public class BeamBuiltinAggregations { - static final Map>> + public static final Map>> BUILTIN_AGGREGATOR_FACTORIES = ImmutableMap.>>builder() .put("COUNT", typeName -> Count.combineFn()) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java new file mode 100644 index 0000000000000..e698265a00618 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -0,0 +1,177 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.impl.transform.agg; + +import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; + +import com.google.auto.value.AutoValue; +import java.util.List; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; +import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; +import org.apache.calcite.util.Pair; + +/** + * Wrapper for aggregation function call. This is needed to avoid dealing with non-serializable + * Calcite classes. + */ +@AutoValue +public abstract class AggregationCombineFnAdapter extends Combine.CombineFn { + + public abstract Schema.Field field(); + + public abstract List args(); + + public abstract @Nullable Combine.CombineFn combineFn(); + + public abstract Schema sourceSchema(); + + public static Builder builder() { + return new AutoValue_AggregationCombineFnAdapter.Builder(); + } + + public static AggregationCombineFnAdapter of( + Pair callWithAlias, Schema inputSchema) { + AggregateCall call = callWithAlias.getKey(); + Schema.Field field = CalciteUtils.toField(callWithAlias.getValue(), call.getType()); + String functionName = call.getAggregation().getName(); + + Builder builder = + builder() + .setArgs(call.getArgList()) + .setField(field) + .setSourceSchema(inputSchema) + .setCombineFn(createCombineFn(call, field, functionName)); + + return builder.build(); + } + + private static Combine.CombineFn createCombineFn( + AggregateCall call, Schema.Field field, String functionName) { + if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { + return getUdafCombineFn(call); + } + + return createAggregator(functionName, field.getType().getTypeName()); + } + + private static @Nullable Combine.CombineFn getUdafCombineFn(AggregateCall call) { + try { + return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) + .getCombineFn(); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + private static Combine.CombineFn createAggregator( + String functionName, Schema.TypeName fieldTypeName) { + + Function> aggregatorFactory = + BUILTIN_AGGREGATOR_FACTORIES.get(functionName); + + if (aggregatorFactory != null) { + return aggregatorFactory.apply(fieldTypeName); + } + + throw new UnsupportedOperationException( + String.format("Aggregator [%s] is not supported", functionName)); + } + + /** Builder for AggregationCombineFnAdapter. */ + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setSourceSchema(Schema sourceSchema); + + public abstract Builder setField(Schema.Field field); + + public abstract Builder setArgs(List args); + + public abstract Builder setCombineFn(@Nullable Combine.CombineFn combineFn); + + public abstract AggregationCombineFnAdapter build(); + } + + @Override + public Object createAccumulator() { + return combineFn().createAccumulator(); + } + + @Override + public Object addInput(Object accumulator, Row input) { + if (args().size() == 2) { + // Aggregation function takes KV + Object key = input.getValue(args().get(0)); + Object value = input.getValue(args().get(1)); + + return (key == null || value == null) + ? accumulator + : combineFn().addInput(accumulator, KV.of(key, value)); + } else { + // Aggregation function takes single value + int fieldIndex = args().size() == 0 ? 0 : args().get(0); + Object value = input.getValue(fieldIndex); + return (value == null) ? accumulator : combineFn().addInput(accumulator, value); + } + } + + @Override + public Object mergeAccumulators(Iterable accumulators) { + return combineFn().mergeAccumulators(accumulators); + } + + @Override + public Object extractOutput(Object accumulator) { + return combineFn().extractOutput(accumulator); + } + + @Override + public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) + throws CannotProvideCoderException { + + if (args().size() == 2) { + int keyIndex = args().get(0); + int valueIndex = args().get(1); + + Coder srcFieldCoderKey = + RowCoder.coderForFieldType(sourceSchema().getField(keyIndex).getType()); + Coder srcFieldCoderValue = + RowCoder.coderForFieldType(sourceSchema().getField(valueIndex).getType()); + + return combineFn() + .getAccumulatorCoder(registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); + } else { + int fieldIndex = args().size() == 0 ? 0 : args().get(0); + Coder srcFieldCoder = + RowCoder.coderForFieldType(sourceSchema().getField(fieldIndex).getType()); + return combineFn().getAccumulatorCoder(registry, srcFieldCoder); + } + } +} From d6cfc06c7ba296a8b5fcb3d0fc4ce62009a6f598 Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 10:38:55 -0700 Subject: [PATCH 10/42] [SQL] Make AggregationCombineFnAdapter non-AutoValue --- .../agg/AggregationCombineFnAdapter.java | 90 ++++++++----------- 1 file changed, 37 insertions(+), 53 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index e698265a00618..b2a03f2f3629a 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -19,10 +19,8 @@ import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; -import com.google.auto.value.AutoValue; import java.util.List; import java.util.function.Function; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -42,19 +40,26 @@ * Wrapper for aggregation function call. This is needed to avoid dealing with non-serializable * Calcite classes. */ -@AutoValue -public abstract class AggregationCombineFnAdapter extends Combine.CombineFn { +public class AggregationCombineFnAdapter extends Combine.CombineFn { - public abstract Schema.Field field(); + protected Schema.Field field; - public abstract List args(); + protected List args; - public abstract @Nullable Combine.CombineFn combineFn(); + protected Combine.CombineFn combineFn; - public abstract Schema sourceSchema(); + protected Schema sourceSchema; - public static Builder builder() { - return new AutoValue_AggregationCombineFnAdapter.Builder(); + public Schema.Field field() { + return field; + } + + public AggregationCombineFnAdapter( + Schema.Field field, List args, Combine.CombineFn combineFn, Schema sourceSchema) { + this.field = field; + this.args = args; + this.combineFn = combineFn; + this.sourceSchema = sourceSchema; } public static AggregationCombineFnAdapter of( @@ -63,14 +68,8 @@ public static AggregationCombineFnAdapter of( Schema.Field field = CalciteUtils.toField(callWithAlias.getValue(), call.getType()); String functionName = call.getAggregation().getName(); - Builder builder = - builder() - .setArgs(call.getArgList()) - .setField(field) - .setSourceSchema(inputSchema) - .setCombineFn(createCombineFn(call, field, functionName)); - - return builder.build(); + return new AggregationCombineFnAdapter( + field, call.getArgList(), createCombineFn(call, field, functionName), inputSchema); } private static Combine.CombineFn createCombineFn( @@ -82,7 +81,7 @@ public static AggregationCombineFnAdapter of( return createAggregator(functionName, field.getType().getTypeName()); } - private static @Nullable Combine.CombineFn getUdafCombineFn(AggregateCall call) { + private static Combine.CombineFn getUdafCombineFn(AggregateCall call) { try { return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) .getCombineFn(); @@ -105,73 +104,58 @@ public static AggregationCombineFnAdapter of( String.format("Aggregator [%s] is not supported", functionName)); } - /** Builder for AggregationCombineFnAdapter. */ - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setSourceSchema(Schema sourceSchema); - - public abstract Builder setField(Schema.Field field); - - public abstract Builder setArgs(List args); - - public abstract Builder setCombineFn(@Nullable Combine.CombineFn combineFn); - - public abstract AggregationCombineFnAdapter build(); - } - @Override public Object createAccumulator() { - return combineFn().createAccumulator(); + return combineFn.createAccumulator(); } @Override public Object addInput(Object accumulator, Row input) { - if (args().size() == 2) { + if (args.size() == 2) { // Aggregation function takes KV - Object key = input.getValue(args().get(0)); - Object value = input.getValue(args().get(1)); + Object key = input.getValue(args.get(0)); + Object value = input.getValue(args.get(1)); return (key == null || value == null) ? accumulator - : combineFn().addInput(accumulator, KV.of(key, value)); + : combineFn.addInput(accumulator, KV.of(key, value)); } else { // Aggregation function takes single value - int fieldIndex = args().size() == 0 ? 0 : args().get(0); + int fieldIndex = args.size() == 0 ? 0 : args.get(0); Object value = input.getValue(fieldIndex); - return (value == null) ? accumulator : combineFn().addInput(accumulator, value); + return (value == null) ? accumulator : combineFn.addInput(accumulator, value); } } @Override public Object mergeAccumulators(Iterable accumulators) { - return combineFn().mergeAccumulators(accumulators); + return combineFn.mergeAccumulators(accumulators); } @Override public Object extractOutput(Object accumulator) { - return combineFn().extractOutput(accumulator); + return combineFn.extractOutput(accumulator); } @Override public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { - if (args().size() == 2) { - int keyIndex = args().get(0); - int valueIndex = args().get(1); + if (args.size() == 2) { + int keyIndex = args.get(0); + int valueIndex = args.get(1); Coder srcFieldCoderKey = - RowCoder.coderForFieldType(sourceSchema().getField(keyIndex).getType()); + RowCoder.coderForFieldType(sourceSchema.getField(keyIndex).getType()); Coder srcFieldCoderValue = - RowCoder.coderForFieldType(sourceSchema().getField(valueIndex).getType()); + RowCoder.coderForFieldType(sourceSchema.getField(valueIndex).getType()); - return combineFn() - .getAccumulatorCoder(registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); + return combineFn.getAccumulatorCoder( + registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); } else { - int fieldIndex = args().size() == 0 ? 0 : args().get(0); - Coder srcFieldCoder = - RowCoder.coderForFieldType(sourceSchema().getField(fieldIndex).getType()); - return combineFn().getAccumulatorCoder(registry, srcFieldCoder); + int fieldIndex = args.size() == 0 ? 0 : args.get(0); + Coder srcFieldCoder = RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); + return combineFn.getAccumulatorCoder(registry, srcFieldCoder); } } } From bb82e44b5501e1ff5f76f406f4c54ed2fad1a576 Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 10:48:58 -0700 Subject: [PATCH 11/42] [SQL] Convert ifs to guard statements in AggregationCombineFnAdapter --- .../agg/AggregationCombineFnAdapter.java | 58 +++++++++++-------- 1 file changed, 33 insertions(+), 25 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index b2a03f2f3629a..ee320caf0e5bd 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -78,7 +78,7 @@ public static AggregationCombineFnAdapter of( return getUdafCombineFn(call); } - return createAggregator(functionName, field.getType().getTypeName()); + return createBuiltinCombineFn(functionName, field.getType().getTypeName()); } private static Combine.CombineFn getUdafCombineFn(AggregateCall call) { @@ -90,7 +90,7 @@ public static AggregationCombineFnAdapter of( } } - private static Combine.CombineFn createAggregator( + private static Combine.CombineFn createBuiltinCombineFn( String functionName, Schema.TypeName fieldTypeName) { Function> aggregatorFactory = @@ -111,20 +111,24 @@ public Object createAccumulator() { @Override public Object addInput(Object accumulator, Row input) { - if (args.size() == 2) { - // Aggregation function takes KV - Object key = input.getValue(args.get(0)); - Object value = input.getValue(args.get(1)); - - return (key == null || value == null) - ? accumulator - : combineFn.addInput(accumulator, KV.of(key, value)); - } else { - // Aggregation function takes single value - int fieldIndex = args.size() == 0 ? 0 : args.get(0); - Object value = input.getValue(fieldIndex); + + if (args.size() == 0) { + Object value = input.getValue(0); return (value == null) ? accumulator : combineFn.addInput(accumulator, value); } + + if (args.size() == 1) { + Object value = input.getValue(args.get(0)); + return (value == null) ? accumulator : combineFn.addInput(accumulator, value); + } + + // Aggregation function takes KV + Object key = input.getValue(args.get(0)); + Object value = input.getValue(args.get(1)); + + return (key == null || value == null) + ? accumulator + : combineFn.addInput(accumulator, KV.of(key, value)); } @Override @@ -141,21 +145,25 @@ public Object extractOutput(Object accumulator) { public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { - if (args.size() == 2) { - int keyIndex = args.get(0); - int valueIndex = args.get(1); - - Coder srcFieldCoderKey = - RowCoder.coderForFieldType(sourceSchema.getField(keyIndex).getType()); - Coder srcFieldCoderValue = - RowCoder.coderForFieldType(sourceSchema.getField(valueIndex).getType()); + if (args.size() == 0) { + Coder srcFieldCoder = RowCoder.coderForFieldType(sourceSchema.getField(0).getType()); + return combineFn.getAccumulatorCoder(registry, srcFieldCoder); + } - return combineFn.getAccumulatorCoder( - registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); - } else { + if (args.size() == 1) { int fieldIndex = args.size() == 0 ? 0 : args.get(0); Coder srcFieldCoder = RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); return combineFn.getAccumulatorCoder(registry, srcFieldCoder); } + + int keyIndex = args.get(0); + int valueIndex = args.get(1); + + Coder srcFieldCoderKey = RowCoder.coderForFieldType(sourceSchema.getField(keyIndex).getType()); + Coder srcFieldCoderValue = + RowCoder.coderForFieldType(sourceSchema.getField(valueIndex).getType()); + + return combineFn.getAccumulatorCoder( + registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); } } From 3fbdd02a77798581c846a773195e9330e700f00c Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 11:02:24 -0700 Subject: [PATCH 12/42] [SQL] Convert Covariance to accept rows instead of KVs --- .../agg/AggregationCombineFnAdapter.java | 31 +++++++++---------- .../sql/impl/transform/agg/CovarianceFn.java | 16 ++++++---- 2 files changed, 25 insertions(+), 22 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index ee320caf0e5bd..ec31daf18b18c 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -17,20 +17,20 @@ */ package org.apache.beam.sdk.extensions.sql.impl.transform.agg; +import static java.util.stream.Collectors.toList; import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; +import static org.apache.beam.sdk.schemas.Schema.toSchema; import java.util.List; import java.util.function.Function; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; @@ -122,13 +122,17 @@ public Object addInput(Object accumulator, Row input) { return (value == null) ? accumulator : combineFn.addInput(accumulator, value); } - // Aggregation function takes KV - Object key = input.getValue(args.get(0)); - Object value = input.getValue(args.get(1)); + List argsValues = args.stream().map(input::getValue).collect(toList()); - return (key == null || value == null) - ? accumulator - : combineFn.addInput(accumulator, KV.of(key, value)); + if (argsValues.contains(null)) { + return accumulator; + } + + Schema argsSchema = + args.stream().map(fieldIndex -> input.getSchema().getField(fieldIndex)).collect(toSchema()); + + return combineFn.addInput( + accumulator, Row.withSchema(argsSchema).addValues(argsValues).build()); } @Override @@ -156,14 +160,9 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder inpu return combineFn.getAccumulatorCoder(registry, srcFieldCoder); } - int keyIndex = args.get(0); - int valueIndex = args.get(1); - - Coder srcFieldCoderKey = RowCoder.coderForFieldType(sourceSchema.getField(keyIndex).getType()); - Coder srcFieldCoderValue = - RowCoder.coderForFieldType(sourceSchema.getField(valueIndex).getType()); + Schema argsSchema = + args.stream().map(fieldIndex -> sourceSchema.getField(fieldIndex)).collect(toSchema()); - return combineFn.getAccumulatorCoder( - registry, KvCoder.of(srcFieldCoderKey, srcFieldCoderValue)); + return combineFn.getAccumulatorCoder(registry, RowCoder.of(argsSchema)); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java index cf92e0772afa0..f3c2c8dc9cc31 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.sql.impl.transform.agg; +import static com.google.common.base.Preconditions.checkArgument; + import java.math.BigDecimal; import java.math.MathContext; import java.math.RoundingMode; @@ -30,7 +32,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; import org.apache.calcite.runtime.SqlFunctions; /** @@ -43,7 +45,7 @@ */ @Internal public class CovarianceFn - extends Combine.CombineFn, CovarianceAccumulator, T> { + extends Combine.CombineFn { static final MathContext MATH_CTX = new MathContext(10, RoundingMode.HALF_UP); @@ -84,15 +86,17 @@ public CovarianceAccumulator createAccumulator() { } @Override - public CovarianceAccumulator addInput(CovarianceAccumulator currentVariance, KV rawInput) { + public CovarianceAccumulator addInput(CovarianceAccumulator currentVariance, Row rawInput) { if (rawInput == null) { return currentVariance; } + checkArgument(rawInput.getFieldCount() > 1); + return currentVariance.combineWith( CovarianceAccumulator.ofSingleElement( - SqlFunctions.toBigDecimal(rawInput.getKey()), - SqlFunctions.toBigDecimal(rawInput.getValue()))); + SqlFunctions.toBigDecimal((Object) rawInput.getValue(0)), + SqlFunctions.toBigDecimal((Object) rawInput.getValue(1)))); } @Override @@ -103,7 +107,7 @@ public CovarianceAccumulator mergeAccumulators(Iterable c @Override public Coder getAccumulatorCoder( - CoderRegistry registry, Coder> inputCoder) { + CoderRegistry registry, Coder inputCoder) { return SerializableCoder.of(CovarianceAccumulator.class); } From 3292271b152b1733cb726a7ff575701df47e61bb Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 11:26:34 -0700 Subject: [PATCH 13/42] [SQL] Split Args Adapters from AggregationCombineFnAdapter --- .../agg/AggregationCombineFnAdapter.java | 146 ++++++++++++++---- 1 file changed, 112 insertions(+), 34 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index ec31daf18b18c..41090c0e3b379 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -21,8 +21,10 @@ import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; import static org.apache.beam.sdk.schemas.Schema.toSchema; +import java.io.Serializable; import java.util.List; import java.util.function.Function; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -30,6 +32,7 @@ import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.values.Row; import org.apache.calcite.rel.core.AggregateCall; @@ -50,16 +53,23 @@ public class AggregationCombineFnAdapter extends Combine.CombineFn args, Combine.CombineFn combineFn, Schema sourceSchema) { + Schema.Field field, + List args, + Combine.CombineFn combineFn, + Schema sourceSchema, + ArgsAdapter argsAdapter) { this.field = field; this.args = args; this.combineFn = combineFn; this.sourceSchema = sourceSchema; + this.argsAdapter = argsAdapter; } public static AggregationCombineFnAdapter of( @@ -69,7 +79,21 @@ public static AggregationCombineFnAdapter of( String functionName = call.getAggregation().getName(); return new AggregationCombineFnAdapter( - field, call.getArgList(), createCombineFn(call, field, functionName), inputSchema); + field, + call.getArgList(), + createCombineFn(call, field, functionName), + inputSchema, + createArgsAdapter(call.getArgList(), inputSchema)); + } + + private static ArgsAdapter createArgsAdapter(List argList, Schema inputSchema) { + if (argList.size() == 0) { + return new ZeroArgsAdapter(inputSchema); + } else if (argList.size() == 1) { + return new SingleArgAdapter(inputSchema, argList); + } else { + return new MultiArgsAdapter(inputSchema, argList); + } } private static Combine.CombineFn createCombineFn( @@ -111,28 +135,8 @@ public Object createAccumulator() { @Override public Object addInput(Object accumulator, Row input) { - - if (args.size() == 0) { - Object value = input.getValue(0); - return (value == null) ? accumulator : combineFn.addInput(accumulator, value); - } - - if (args.size() == 1) { - Object value = input.getValue(args.get(0)); - return (value == null) ? accumulator : combineFn.addInput(accumulator, value); - } - - List argsValues = args.stream().map(input::getValue).collect(toList()); - - if (argsValues.contains(null)) { - return accumulator; - } - - Schema argsSchema = - args.stream().map(fieldIndex -> input.getSchema().getField(fieldIndex)).collect(toSchema()); - - return combineFn.addInput( - accumulator, Row.withSchema(argsSchema).addValues(argsValues).build()); + Object argsValues = argsAdapter.getArgsValues(input); + return (argsValues == null) ? accumulator : combineFn.addInput(accumulator, argsValues); } @Override @@ -149,20 +153,94 @@ public Object extractOutput(Object accumulator) { public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { - if (args.size() == 0) { - Coder srcFieldCoder = RowCoder.coderForFieldType(sourceSchema.getField(0).getType()); - return combineFn.getAccumulatorCoder(registry, srcFieldCoder); + return combineFn.getAccumulatorCoder(registry, argsAdapter.getArgsValuesCoder()); + } + + static class ZeroArgsAdapter implements ArgsAdapter { + Schema sourceSchema; + + public ZeroArgsAdapter(Schema sourceSchema) { + this.sourceSchema = sourceSchema; + } + + @Nullable + @Override + public Object getArgsValues(Row input) { + return input.getValue(0); + } + + @Override + public Coder getArgsValuesCoder() { + return RowCoder.coderForFieldType(sourceSchema.getField(0).getType()); } + } + + static class SingleArgAdapter implements ArgsAdapter { + Schema sourceSchema; + List argsIndicies; + + public SingleArgAdapter(Schema sourceSchema, List argsIndicies) { + this.sourceSchema = sourceSchema; + this.argsIndicies = argsIndicies; + } + + @Nullable + @Override + public Object getArgsValues(Row input) { + return input.getValue(argsIndicies.get(0)); + } + + @Override + public Coder getArgsValuesCoder() { + int fieldIndex = argsIndicies.get(0); + return RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); + } + } + + static class MultiArgsAdapter implements ArgsAdapter { + Schema sourceSchema; + List argsIndicies; - if (args.size() == 1) { - int fieldIndex = args.size() == 0 ? 0 : args.get(0); - Coder srcFieldCoder = RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); - return combineFn.getAccumulatorCoder(registry, srcFieldCoder); + public MultiArgsAdapter(Schema sourceSchema, List argsIndicies) { + this.sourceSchema = sourceSchema; + this.argsIndicies = argsIndicies; } - Schema argsSchema = - args.stream().map(fieldIndex -> sourceSchema.getField(fieldIndex)).collect(toSchema()); + @Nullable + @Override + public Object getArgsValues(Row input) { + + List argsValues = argsIndicies.stream().map(input::getValue).collect(toList()); + + if (argsValues.contains(null)) { + return null; + } + + Schema argsSchema = + argsIndicies + .stream() + .map(fieldIndex -> input.getSchema().getField(fieldIndex)) + .collect(toSchema()); + + return Row.withSchema(argsSchema).addValues(argsValues).build(); + } + + @Override + public Coder getArgsValuesCoder() { + Schema argsSchema = + argsIndicies + .stream() + .map(fieldIndex -> sourceSchema.getField(fieldIndex)) + .collect(toSchema()); + + return SchemaCoder.of(argsSchema); + } + } + + interface ArgsAdapter extends Serializable { + @Nullable + Object getArgsValues(Row input); - return combineFn.getAccumulatorCoder(registry, RowCoder.of(argsSchema)); + Coder getArgsValuesCoder(); } } From 19ec3d20700d6457b8629607dbbb465504a66749 Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 12:09:37 -0700 Subject: [PATCH 14/42] [SQL] Extract MultipleAggregationFn from BeamAggregationTransforms --- .../sql/impl/rel/BeamAggregationRel.java | 36 +++- .../transform/BeamAggregationTransforms.java | 174 ----------------- .../transform/MultipleAggregationsFn.java | 175 ++++++++++++++++++ .../agg/AggregationCombineFnAdapter.java | 10 +- 4 files changed, 208 insertions(+), 187 deletions(-) delete mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/MultipleAggregationsFn.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index e98605354a4f0..abb10a8de92fb 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -23,19 +23,22 @@ import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED; import static org.apache.beam.sdk.values.Row.toRow; +import com.google.common.collect.Lists; import java.util.List; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.extensions.sql.impl.transform.BeamAggregationTransforms; +import org.apache.beam.sdk.extensions.sql.impl.transform.MultipleAggregationsFn; import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.WithTimestamps; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -201,17 +204,12 @@ public PCollection expand(PCollectionList pinput) { PCollection> aggregatedStream = exCombineByStream .apply( - "combineBy", - Combine.perKey( - new BeamAggregationTransforms.MultiAggregationCombineFn(aggregationCalls))) + "combineBy", Combine.perKey(MultipleAggregationsFn.combineFns(aggregationCalls))) .setCoder(KvCoder.of(keyCoder, aggCoder)); PCollection mergedStream = aggregatedStream.apply( - "mergeRecord", - ParDo.of( - new BeamAggregationTransforms.MergeAggregationRecord( - outputSchema, windowFieldIndex))); + "mergeRecord", ParDo.of(mergeRecord(outputSchema, windowFieldIndex))); mergedStream.setRowSchema(outputSchema); return mergedStream; @@ -239,6 +237,28 @@ private void validateWindowIsSupported(PCollection upstream) { + "See GroupByKey section in Beam Programming Guide"); } } + + public static DoFn, Row> mergeRecord( + Schema outputSchema, int windowStartFieldIndex) { + + return new DoFn, Row>() { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) { + KV kvRow = c.element(); + List fieldValues = + Lists.newArrayListWithCapacity( + kvRow.getKey().getValues().size() + kvRow.getValue().getValues().size()); + fieldValues.addAll(kvRow.getKey().getValues()); + fieldValues.addAll(kvRow.getValue().getValues()); + + if (windowStartFieldIndex != -1) { + fieldValues.add(windowStartFieldIndex, ((IntervalWindow) window).start()); + } + + c.output(Row.withSchema(outputSchema).addValues(fieldValues).build()); + } + }; + } } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java deleted file mode 100644 index ff85ccfe96f9f..0000000000000 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamAggregationTransforms.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.sdk.extensions.sql.impl.transform; - -import static org.apache.beam.sdk.schemas.Schema.toSchema; -import static org.apache.beam.sdk.values.Row.toRow; - -import com.google.common.collect.Lists; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.math.BigDecimal; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.beam.sdk.coders.BigDecimalCoder; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter; -import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.Row; - -/** Collections of {@code PTransform} and {@code DoFn} used to perform GROUP-BY operation. */ -public class BeamAggregationTransforms implements Serializable { - /** Merge KV to single record. */ - public static class MergeAggregationRecord extends DoFn, Row> { - private Schema outSchema; - private int windowStartFieldIdx; - - public MergeAggregationRecord(Schema outSchema, int windowStartFieldIdx) { - this.outSchema = outSchema; - this.windowStartFieldIdx = windowStartFieldIdx; - } - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) { - KV kvRow = c.element(); - List fieldValues = - Lists.newArrayListWithCapacity( - kvRow.getKey().getValues().size() + kvRow.getValue().getValues().size()); - fieldValues.addAll(kvRow.getKey().getValues()); - fieldValues.addAll(kvRow.getValue().getValues()); - - if (windowStartFieldIdx != -1) { - fieldValues.add(windowStartFieldIdx, ((IntervalWindow) window).start()); - } - - c.output(Row.withSchema(outSchema).addValues(fieldValues).build()); - } - } - - /** An adaptor class to invoke Calcite UDAF instances in Beam {@code CombineFn}. */ - public static class MultiAggregationCombineFn extends CombineFn, Row> { - private List aggCombineFns; - private Schema finalSchema; - - public MultiAggregationCombineFn(List aggCombineFns) { - this.aggCombineFns = aggCombineFns; - finalSchema = - this.aggCombineFns.stream().map(AggregationCombineFnAdapter::field).collect(toSchema()); - } - - @Override - public List createAccumulator() { - return aggCombineFns - .stream() - .map(AggregationCombineFnAdapter::createAccumulator) - .collect(Collectors.toList()); - } - - @Override - public List addInput(List accumulators, Row input) { - List deltaAcc = new ArrayList<>(); - - for (int idx = 0; idx < aggCombineFns.size(); ++idx) { - AggregationCombineFnAdapter aggregator = aggCombineFns.get(idx); - Object aggregatorAccumulator = accumulators.get(idx); - - Object newAccumulator = aggregator.addInput(aggregatorAccumulator, input); - deltaAcc.add(newAccumulator); - } - return deltaAcc; - } - - @Override - public List mergeAccumulators(Iterable> accumulators) { - List deltaAcc = new ArrayList<>(); - for (int idx = 0; idx < aggCombineFns.size(); ++idx) { - List accs = new ArrayList<>(); - for (List accumulator : accumulators) { - accs.add(accumulator.get(idx)); - } - deltaAcc.add(aggCombineFns.get(idx).mergeAccumulators(accs)); - } - return deltaAcc; - } - - @Override - public Row extractOutput(List accumulator) { - return IntStream.range(0, aggCombineFns.size()) - .mapToObj(idx -> aggCombineFns.get(idx).extractOutput(accumulator.get(idx))) - .collect(toRow(finalSchema)); - } - - @Override - public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) - throws CannotProvideCoderException { - // TODO: Doing this here is wrong. - registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of()); - - List aggAccuCoderList = new ArrayList<>(); - - for (AggregationCombineFnAdapter aggCombineFn : aggCombineFns) { - aggAccuCoderList.add(aggCombineFn.getAccumulatorCoder(registry, inputCoder)); - } - - return new AggregationAccumulatorCoder(aggAccuCoderList); - } - } - - /** Coder for accumulators. */ - public static class AggregationAccumulatorCoder extends CustomCoder> { - private VarIntCoder sizeCoder = VarIntCoder.of(); - private List elementCoders; - - public AggregationAccumulatorCoder(List elementCoders) { - this.elementCoders = elementCoders; - } - - @Override - public void encode(List value, OutputStream outStream) throws IOException { - sizeCoder.encode(value.size(), outStream); - for (int idx = 0; idx < value.size(); ++idx) { - elementCoders.get(idx).encode(value.get(idx), outStream); - } - } - - @Override - public List decode(InputStream inStream) throws CoderException, IOException { - List accu = new ArrayList<>(); - int size = sizeCoder.decode(inStream); - for (int idx = 0; idx < size; ++idx) { - accu.add(elementCoders.get(idx).decode(inStream)); - } - return accu; - } - } -} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/MultipleAggregationsFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/MultipleAggregationsFn.java new file mode 100644 index 0000000000000..0b8d8961410d4 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/MultipleAggregationsFn.java @@ -0,0 +1,175 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.impl.transform; + +import static org.apache.beam.sdk.schemas.Schema.toSchema; +import static org.apache.beam.sdk.values.Row.toRow; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.beam.sdk.coders.BigDecimalCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationCombineFnAdapter; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.values.Row; + +/** + * Wrapper for multiple aggregations. + * + *

    Maintains the accumulators and output schema. Delegates the aggregation to the combiners for + * each separate aggregation call. + * + *

    Output schema is the schema corresponding to the list of all aggregation calls. + */ +public class MultipleAggregationsFn extends CombineFn, Row> { + private List aggCombineFns; + private Schema outputSchema; + + /** + * Returns an instance of {@link MultipleAggregationsFn}. + * + * @param aggCombineFns is the list of aggregation {@link CombineFn CombineFns} that perform the + * actual aggregations. + */ + public static MultipleAggregationsFn combineFns(List aggCombineFns) { + return new MultipleAggregationsFn(aggCombineFns); + } + + private MultipleAggregationsFn(List aggCombineFns) { + this.aggCombineFns = aggCombineFns; + outputSchema = + this.aggCombineFns.stream().map(AggregationCombineFnAdapter::field).collect(toSchema()); + } + + /** + * Accumulator for this {@link CombineFn} is a list of accumulators of the underlying delegate + * {@link CombineFn CombineFns}. + */ + @Override + public List createAccumulator() { + return aggCombineFns + .stream() + .map(AggregationCombineFnAdapter::createAccumulator) + .collect(Collectors.toList()); + } + + /** For each delegate {@link CombineFn} we use the corresponding accumulator from the list. */ + @Override + public List addInput(List accumulators, Row input) { + List newAcc = new ArrayList<>(); + + for (int idx = 0; idx < aggCombineFns.size(); ++idx) { + AggregationCombineFnAdapter aggregator = aggCombineFns.get(idx); + Object aggregatorAccumulator = accumulators.get(idx); + + Object newAccumulator = aggregator.addInput(aggregatorAccumulator, input); + newAcc.add(newAccumulator); + } + return newAcc; + } + + /** + * Collect all accumulators for the corresponding {@link CombineFn} into a list, and pass the list + * for merging to the delegate. + */ + @Override + public List mergeAccumulators(Iterable> accumulators) { + List newAcc = new ArrayList<>(); + for (int idx = 0; idx < aggCombineFns.size(); ++idx) { + List accs = new ArrayList<>(); + for (List accumulator : accumulators) { + accs.add(accumulator.get(idx)); + } + newAcc.add(aggCombineFns.get(idx).mergeAccumulators(accs)); + } + return newAcc; + } + + /** + * Just extract all outputs from the delegate {@link CombineFn CombineFns} and assemble them into + * a row. + */ + @Override + public Row extractOutput(List accumulator) { + return IntStream.range(0, aggCombineFns.size()) + .mapToObj(idx -> aggCombineFns.get(idx).extractOutput(accumulator.get(idx))) + .collect(toRow(outputSchema)); + } + + /** + * Accumulator coder is a special {@link AggregationAccumulatorCoder coder} that encodes a list of + * all accumulators using accumulator coders from their {@link CombineFn CombineFns}. + */ + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) + throws CannotProvideCoderException { + // TODO: Doing this here is wrong. + registry.registerCoderForClass(BigDecimal.class, BigDecimalCoder.of()); + List aggAccuCoderList = new ArrayList<>(); + + for (AggregationCombineFnAdapter aggCombineFn : aggCombineFns) { + aggAccuCoderList.add(aggCombineFn.getAccumulatorCoder(registry, inputCoder)); + } + + return new AggregationAccumulatorCoder(aggAccuCoderList); + } + + /** + * Coder for accumulators. + * + *

    Takes in a list of accumulator coders, delegates encoding/decoding to them. + */ + static class AggregationAccumulatorCoder extends CustomCoder> { + private VarIntCoder sizeCoder = VarIntCoder.of(); + private List elementCoders; + + AggregationAccumulatorCoder(List elementCoders) { + this.elementCoders = elementCoders; + } + + @Override + public void encode(List value, OutputStream outStream) throws IOException { + sizeCoder.encode(value.size(), outStream); + for (int idx = 0; idx < value.size(); ++idx) { + elementCoders.get(idx).encode(value.get(idx), outStream); + } + } + + @Override + public List decode(InputStream inStream) throws CoderException, IOException { + List accu = new ArrayList<>(); + int size = sizeCoder.decode(inStream); + for (int idx = 0; idx < size; ++idx) { + accu.add(elementCoders.get(idx).decode(inStream)); + } + return accu; + } + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index 41090c0e3b379..840b9709b541f 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -45,15 +45,15 @@ */ public class AggregationCombineFnAdapter extends Combine.CombineFn { - protected Schema.Field field; + private Schema.Field field; - protected List args; + private List args; - protected Combine.CombineFn combineFn; + private Combine.CombineFn combineFn; - protected Schema sourceSchema; + private Schema sourceSchema; - protected ArgsAdapter argsAdapter; + private ArgsAdapter argsAdapter; public Schema.Field field() { return field; From 301b0c9fd1f78ddbbf341a8c0dc668ec9eaf84aa Mon Sep 17 00:00:00 2001 From: akedin Date: Sun, 21 Oct 2018 13:09:10 -0700 Subject: [PATCH 15/42] [SQL] Clean up, comment aggregation transforms --- .../sql/impl/rel/BeamAggregationRel.java | 64 +++--- .../transform/BeamBuiltinAggregations.java | 13 ++ .../transform/agg/AggregationArgsAdapter.java | 168 ++++++++++++++++ .../agg/AggregationCombineFnAdapter.java | 185 ++++-------------- .../sql/BeamSqlDslAggregationTest.java | 63 ------ 5 files changed, 263 insertions(+), 230 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationArgsAdapter.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java index abb10a8de92fb..68a9f537b2ee3 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java @@ -181,37 +181,58 @@ public PCollection expand(PCollectionList pinput) { PCollection upstream = pinput.get(0); PCollection windowedStream = upstream; if (windowFn != null) { - upstream = - upstream - .apply( - "assignEventTimestamp", - WithTimestamps.of(row -> row.getDateTime(windowFieldIndex).toInstant()) - .withAllowedTimestampSkew(new Duration(Long.MAX_VALUE))) - .setCoder(upstream.getCoder()); - windowedStream = upstream.apply(Window.into(windowFn)); + windowedStream = assignTimestampsAndWindow(upstream); } validateWindowIsSupported(windowedStream); - PCollection> exCombineByStream = - windowedStream - .apply( - "exCombineBy", - WithKeys.of( - row -> keyFieldsIds.stream().map(row::getValue).collect(toRow(keySchema)))) - .setCoder(KvCoder.of(keyCoder, upstream.getCoder())); + PCollection> exCombineByStream = extractGroupingKeys(upstream, windowedStream); + + PCollection> aggregatedStream = performAggregation(exCombineByStream); + + PCollection mergedStream = mergeRows(aggregatedStream); + + return mergedStream; + } - PCollection> aggregatedStream = - exCombineByStream + /** Extract timestamps from the windowFieldIndex, then window into windowFns. */ + private PCollection assignTimestampsAndWindow(PCollection upstream) { + PCollection windowedStream; + windowedStream = + upstream .apply( - "combineBy", Combine.perKey(MultipleAggregationsFn.combineFns(aggregationCalls))) - .setCoder(KvCoder.of(keyCoder, aggCoder)); + "assignEventTimestamp", + WithTimestamps.of(row -> row.getDateTime(windowFieldIndex).toInstant()) + .withAllowedTimestampSkew(new Duration(Long.MAX_VALUE))) + .setCoder(upstream.getCoder()) + .apply(Window.into(windowFn)); + return windowedStream; + } + + /** Extract non-windowing group-by fields, assign them as a key. */ + private PCollection> extractGroupingKeys( + PCollection upstream, PCollection windowedStream) { + return windowedStream + .apply( + "exCombineBy", + WithKeys.of( + row -> keyFieldsIds.stream().map(row::getValue).collect(toRow(keySchema)))) + .setCoder(KvCoder.of(keyCoder, upstream.getCoder())); + } + private PCollection> performAggregation( + PCollection> exCombineByStream) { + return exCombineByStream + .apply("combineBy", Combine.perKey(MultipleAggregationsFn.combineFns(aggregationCalls))) + .setCoder(KvCoder.of(keyCoder, aggCoder)); + } + + /** Merge the KVs back into whole rows. */ + private PCollection mergeRows(PCollection> aggregatedStream) { PCollection mergedStream = aggregatedStream.apply( "mergeRecord", ParDo.of(mergeRecord(outputSchema, windowFieldIndex))); mergedStream.setRowSchema(outputSchema); - return mergedStream; } @@ -238,8 +259,7 @@ private void validateWindowIsSupported(PCollection upstream) { } } - public static DoFn, Row> mergeRecord( - Schema outputSchema, int windowStartFieldIndex) { + static DoFn, Row> mergeRecord(Schema outputSchema, int windowStartFieldIndex) { return new DoFn, Row>() { @ProcessElement diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java index 0172ebf7fe336..d35d0c4f0d269 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java @@ -64,6 +64,19 @@ public class BeamBuiltinAggregations { private static MathContext mc = new MathContext(10, RoundingMode.HALF_UP); + public static CombineFn create(String functionName, Schema.TypeName fieldTypeName) { + + Function> aggregatorFactory = + BUILTIN_AGGREGATOR_FACTORIES.get(functionName); + + if (aggregatorFactory != null) { + return aggregatorFactory.apply(fieldTypeName); + } + + throw new UnsupportedOperationException( + String.format("Aggregator [%s] is not supported", functionName)); + } + /** {@link CombineFn} for MAX based on {@link Max} and {@link Combine.BinaryCombineFn}. */ static CombineFn createMax(Schema.TypeName fieldType) { switch (fieldType) { diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationArgsAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationArgsAdapter.java new file mode 100644 index 0000000000000..9f7fb91b697d1 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationArgsAdapter.java @@ -0,0 +1,168 @@ +/* + * 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. + */ +package org.apache.beam.sdk.extensions.sql.impl.transform.agg; + +import static java.util.stream.Collectors.toList; +import static org.apache.beam.sdk.schemas.Schema.toSchema; + +import java.io.Serializable; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.values.Row; + +/** + * Utility class to extract arguments from the input Row to match the expected input of the {@link + * CombineFn}. + */ +class AggregationArgsAdapter { + + /** + * Creates an args adapter based on the args list and input schema. + * + * @param argList indices of fields that are specified as arguments for an aggregation call. + * @param inputSchema input row that will be passed into the aggregation + */ + static ArgsAdapter of(List argList, Schema inputSchema) { + if (argList.size() == 0) { + return new ZeroArgsAdapter(inputSchema); + } else if (argList.size() == 1) { + return new SingleArgAdapter(inputSchema, argList); + } else { + return new MultiArgsAdapter(inputSchema, argList); + } + } + + /** + * If SQL aggregation call doesn't have actual arguments, we still pass the value first field to + * the delegate {@link CombineFn}. + * + *

    Note: this is not necessarily the correct thing to do. + */ + static class ZeroArgsAdapter implements ArgsAdapter { + Schema sourceSchema; + + ZeroArgsAdapter(Schema sourceSchema) { + this.sourceSchema = sourceSchema; + } + + /** Extracts the value from the first field of a row. */ + @Nullable + @Override + public Object getArgsValues(Row input) { + return input.getValue(0); + } + + /** Coder for the first field of a row. */ + @Override + public Coder getArgsValuesCoder() { + return RowCoder.coderForFieldType(sourceSchema.getField(0).getType()); + } + } + + /** + * If SQL aggregation call has a single argument (e.g. MAX), we extract its raw value to pass to + * the delegate {@link CombineFn}. + */ + static class SingleArgAdapter implements ArgsAdapter { + Schema sourceSchema; + List argsIndicies; + + public SingleArgAdapter(Schema sourceSchema, List argsIndicies) { + this.sourceSchema = sourceSchema; + this.argsIndicies = argsIndicies; + } + + /** + * Args indices contain a single element with the index of a field that SQL call specifies. Here + * we extract the value of that field from the input row. + */ + @Nullable + @Override + public Object getArgsValues(Row input) { + return input.getValue(argsIndicies.get(0)); + } + + /** Coder for the field of a row used as an argument. */ + @Override + public Coder getArgsValuesCoder() { + int fieldIndex = argsIndicies.get(0); + return RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); + } + } + + /** + * If SQL aggregation call has multiple arguments (e.g. COVAR_POP), we extract the fields + * specified in the arguments, then combine them into a row, and then pass into the delegate + * {@link CombineFn}. + */ + static class MultiArgsAdapter implements ArgsAdapter { + Schema sourceSchema; + List argsIndicies; + + MultiArgsAdapter(Schema sourceSchema, List argsIndicies) { + this.sourceSchema = sourceSchema; + this.argsIndicies = argsIndicies; + } + + /** + * Extract the sub-row with the fields specified in the arguments. If args values contain nulls, + * return null. + */ + @Nullable + @Override + public Object getArgsValues(Row input) { + List argsValues = argsIndicies.stream().map(input::getValue).collect(toList()); + + if (argsValues.contains(null)) { + return null; + } + + Schema argsSchema = + argsIndicies + .stream() + .map(fieldIndex -> input.getSchema().getField(fieldIndex)) + .collect(toSchema()); + + return Row.withSchema(argsSchema).addValues(argsValues).build(); + } + + /** Schema coder of the sub-row specified by the fields in the arguments list. */ + @Override + public Coder getArgsValuesCoder() { + Schema argsSchema = + argsIndicies + .stream() + .map(fieldIndex -> sourceSchema.getField(fieldIndex)) + .collect(toSchema()); + + return SchemaCoder.of(argsSchema); + } + } + + interface ArgsAdapter extends Serializable { + @Nullable + Object getArgsValues(Row input); + + Coder getArgsValuesCoder(); + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java index 840b9709b541f..a58a4680b56ee 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java @@ -17,61 +17,57 @@ */ package org.apache.beam.sdk.extensions.sql.impl.transform.agg; -import static java.util.stream.Collectors.toList; -import static org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations.BUILTIN_AGGREGATOR_FACTORIES; -import static org.apache.beam.sdk.schemas.Schema.toSchema; - -import java.io.Serializable; -import java.util.List; -import java.util.function.Function; -import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.RowCoder; import org.apache.beam.sdk.extensions.sql.impl.UdafImpl; +import org.apache.beam.sdk.extensions.sql.impl.transform.BeamBuiltinAggregations; +import org.apache.beam.sdk.extensions.sql.impl.transform.agg.AggregationArgsAdapter.ArgsAdapter; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.SchemaCoder; -import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.Row; import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction; import org.apache.calcite.util.Pair; /** - * Wrapper for aggregation function call. This is needed to avoid dealing with non-serializable - * Calcite classes. + * Wrapper {@link CombineFn} for aggregation function call. + * + *

    Delegates to the actual aggregation {@link CombineFn}, either built-in, or UDAF. + * + *

    Actual aggregation {@link CombineFn CombineFns} expect their specific arguments, not the full + * input row. This class uses {@link ArgsAdapter arg adapters} to extract and map the call arguments + * to the {@link CombineFn CombineFn's} inputs. */ -public class AggregationCombineFnAdapter extends Combine.CombineFn { +public class AggregationCombineFnAdapter extends CombineFn { + // Field for a function call private Schema.Field field; - private List args; - - private Combine.CombineFn combineFn; - - private Schema sourceSchema; + // Actual aggregation CombineFn + private CombineFn combineFn; + // Adapter to convert input Row to CombineFn's arguments private ArgsAdapter argsAdapter; + /** {@link Schema.Field} with this function call. */ public Schema.Field field() { return field; } - public AggregationCombineFnAdapter( - Schema.Field field, - List args, - Combine.CombineFn combineFn, - Schema sourceSchema, - ArgsAdapter argsAdapter) { + private AggregationCombineFnAdapter( + Schema.Field field, CombineFn combineFn, ArgsAdapter argsAdapter) { this.field = field; - this.args = args; this.combineFn = combineFn; - this.sourceSchema = sourceSchema; this.argsAdapter = argsAdapter; } + /** + * Creates an instance of {@link AggregationCombineFnAdapter}. + * + * @param callWithAlias Calcite's output, represents a function call paired with its field alias + */ public static AggregationCombineFnAdapter of( Pair callWithAlias, Schema inputSchema) { AggregateCall call = callWithAlias.getKey(); @@ -80,32 +76,21 @@ public static AggregationCombineFnAdapter of( return new AggregationCombineFnAdapter( field, - call.getArgList(), createCombineFn(call, field, functionName), - inputSchema, - createArgsAdapter(call.getArgList(), inputSchema)); - } - - private static ArgsAdapter createArgsAdapter(List argList, Schema inputSchema) { - if (argList.size() == 0) { - return new ZeroArgsAdapter(inputSchema); - } else if (argList.size() == 1) { - return new SingleArgAdapter(inputSchema, argList); - } else { - return new MultiArgsAdapter(inputSchema, argList); - } + AggregationArgsAdapter.of(call.getArgList(), inputSchema)); } - private static Combine.CombineFn createCombineFn( + /** Creates either a UDAF or a built-in {@link CombineFn}. */ + private static CombineFn createCombineFn( AggregateCall call, Schema.Field field, String functionName) { if (call.getAggregation() instanceof SqlUserDefinedAggFunction) { return getUdafCombineFn(call); } - return createBuiltinCombineFn(functionName, field.getType().getTypeName()); + return BeamBuiltinAggregations.create(functionName, field.getType().getTypeName()); } - private static Combine.CombineFn getUdafCombineFn(AggregateCall call) { + private static CombineFn getUdafCombineFn(AggregateCall call) { try { return ((UdafImpl) ((SqlUserDefinedAggFunction) call.getAggregation()).function) .getCombineFn(); @@ -114,25 +99,18 @@ private static ArgsAdapter createArgsAdapter(List argList, Schema input } } - private static Combine.CombineFn createBuiltinCombineFn( - String functionName, Schema.TypeName fieldTypeName) { - - Function> aggregatorFactory = - BUILTIN_AGGREGATOR_FACTORIES.get(functionName); - - if (aggregatorFactory != null) { - return aggregatorFactory.apply(fieldTypeName); - } - - throw new UnsupportedOperationException( - String.format("Aggregator [%s] is not supported", functionName)); - } - @Override public Object createAccumulator() { return combineFn.createAccumulator(); } + /** + * Calls the args adapter to extract the fields from the input row and pass them into the actual + * {@link CombineFn}. E.g. input of a MAX(f) is not a full row, but just a number. + * + *

    If argument is null, skip it and return the original accumulator. This is what SQL + * aggregations are supposed to do. + */ @Override public Object addInput(Object accumulator, Row input) { Object argsValues = argsAdapter.getArgsValues(input); @@ -149,98 +127,15 @@ public Object extractOutput(Object accumulator) { return combineFn.extractOutput(accumulator); } + /** + * {@link CombineFn#getAccumulatorCoder} is supposed to use input {@link Coder coder} to infer the + * {@link Coder coder} for the accumulator. Here we call the args adapter to get the input coder + * for the delegate {@link CombineFn}. + */ @Override public Coder getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { return combineFn.getAccumulatorCoder(registry, argsAdapter.getArgsValuesCoder()); } - - static class ZeroArgsAdapter implements ArgsAdapter { - Schema sourceSchema; - - public ZeroArgsAdapter(Schema sourceSchema) { - this.sourceSchema = sourceSchema; - } - - @Nullable - @Override - public Object getArgsValues(Row input) { - return input.getValue(0); - } - - @Override - public Coder getArgsValuesCoder() { - return RowCoder.coderForFieldType(sourceSchema.getField(0).getType()); - } - } - - static class SingleArgAdapter implements ArgsAdapter { - Schema sourceSchema; - List argsIndicies; - - public SingleArgAdapter(Schema sourceSchema, List argsIndicies) { - this.sourceSchema = sourceSchema; - this.argsIndicies = argsIndicies; - } - - @Nullable - @Override - public Object getArgsValues(Row input) { - return input.getValue(argsIndicies.get(0)); - } - - @Override - public Coder getArgsValuesCoder() { - int fieldIndex = argsIndicies.get(0); - return RowCoder.coderForFieldType(sourceSchema.getField(fieldIndex).getType()); - } - } - - static class MultiArgsAdapter implements ArgsAdapter { - Schema sourceSchema; - List argsIndicies; - - public MultiArgsAdapter(Schema sourceSchema, List argsIndicies) { - this.sourceSchema = sourceSchema; - this.argsIndicies = argsIndicies; - } - - @Nullable - @Override - public Object getArgsValues(Row input) { - - List argsValues = argsIndicies.stream().map(input::getValue).collect(toList()); - - if (argsValues.contains(null)) { - return null; - } - - Schema argsSchema = - argsIndicies - .stream() - .map(fieldIndex -> input.getSchema().getField(fieldIndex)) - .collect(toSchema()); - - return Row.withSchema(argsSchema).addValues(argsValues).build(); - } - - @Override - public Coder getArgsValuesCoder() { - Schema argsSchema = - argsIndicies - .stream() - .map(fieldIndex -> sourceSchema.getField(fieldIndex)) - .collect(toSchema()); - - return SchemaCoder.of(argsSchema); - } - } - - interface ArgsAdapter extends Serializable { - @Nullable - Object getArgsValues(Row input); - - Coder getArgsValuesCoder(); - } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java index f8713f2c1e748..7fd5718523c59 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java @@ -48,7 +48,6 @@ import org.joda.time.DateTime; import org.joda.time.Duration; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -642,68 +641,6 @@ public void testSupportsNonGlobalWindowWithCustomTrigger() { pipeline.run(); } - @Ignore - @Test - public void testSupportsBasicSum() throws Exception { - Schema inputSchema = - Schema.builder() - .addStringField("f_string") - .addInt32Field("f_int") - // .addInt32Field("f_anotherint") - .build(); - - PCollection input = - pipeline.apply( - Create.of( - Row.withSchema(inputSchema) - .addValues("project1") - .addValue(1) - // .addValue(1) - .build(), - Row.withSchema(inputSchema) - .addValues("project1") - .addValue(2) - // .addValue(1) - .build(), - Row.withSchema(inputSchema) - .addValues("project2") - .addValue(4) - // .addValue(1) - .build(), - Row.withSchema(inputSchema) - .addValues("project1") - .addValue(13) - // .addValue(1) - .build(), - Row.withSchema(inputSchema) - .addValues("project3") - .addValue(1) - // .addValue(1) - .build()) - .withSchema( - inputSchema, - SerializableFunctions.identity(), - SerializableFunctions.identity())); - - Schema resultSchema = - Schema.builder() - // .addInt32Field("f_string") - .addInt32Field("f_int") - // .addInt32Field("f_anotherint") - .build(); - - PCollection result = - input.apply(SqlTransform.query("SELECT sum(f_int) FROM PCOLLECTION GROUP BY f_string")); - - PAssert.that(result) - .containsInAnyOrder( - Row.withSchema(resultSchema).addValues(16).build(), - Row.withSchema(resultSchema).addValues(4).build(), - Row.withSchema(resultSchema).addValues(1).build()); - - pipeline.run(); - } - private List rowsWithSingleIntField(String fieldName, List values) { return TestUtils.rowsBuilderOf(Schema.builder().addInt32Field(fieldName).build()) .addRows(values) From aaf9c3a359f1af5c325ea3857d59bddb59094c2e Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Mon, 22 Oct 2018 13:42:12 -0700 Subject: [PATCH 16/42] Export Grafana testing dashboards and improve README --- .test-infra/metrics/README.md | 65 +- .test-infra/metrics/dashboards/dashboard.json | 297 ------- .../metrics/dashboards/post-commit_tests.json | 780 ++++++++++++++++++ .../metrics/dashboards/pre-commit_tests.json | 204 +++++ .../stability_critical_jobs_status.json | 331 ++++++++ 5 files changed, 1371 insertions(+), 306 deletions(-) delete mode 100644 .test-infra/metrics/dashboards/dashboard.json create mode 100644 .test-infra/metrics/dashboards/post-commit_tests.json create mode 100644 .test-infra/metrics/dashboards/pre-commit_tests.json create mode 100644 .test-infra/metrics/dashboards/stability_critical_jobs_status.json diff --git a/.test-infra/metrics/README.md b/.test-infra/metrics/README.md index d18c52c4c35c0..1da9ca27f3cb7 100644 --- a/.test-infra/metrics/README.md +++ b/.test-infra/metrics/README.md @@ -17,17 +17,23 @@ under the License. --> # BeamMonitoring -This folder contains files required to spin-up metrics dashboard for Beam. +This folder contains files required to deploy the Beam Community metrics stack +on your local machine. -## Utilized technologies -* [Grafana](https://grafana.com) as dashboarding engine. -* PostgreSQL as underlying DB. +This includes +* Python scripts for ingesting data from sources (Jenkins, JIRA, + GitHub) +* Postgres analytics database +* [Grafana](https://grafana.com) dashboarding UI -Approach utilized is to fetch data from corresponding system: Jenkins/Jira/GithubArchives/etc, put it into PostreSQL and fetch it to show in Grafana. +All components run within Docker containers. These are composed together via +docker-compose for local hosting, and Kubernetes for the production instance on +GCP. ## Local setup -Install docker +Docker Compose is used to host the full metrics stack on your local machine. + * install docker * https://docs.docker.com/install/#supported-platforms * install docker-compose @@ -76,8 +82,24 @@ docker-compose build docker-compose up ``` +After running these commands, you can access the services running on your local +machine: + +* Grafana: http://localhost:3000 +* Postgres DB: localhost:5432 + +If you're deploying for the first time on your machine, follow instructions on +how to manually [configure Grafana](#configuring-grafana). + +Grafana and Postgres containers persist data to Docker volumes, which will be +restored on subsequent runs. To start from a clean state, you must also wipe out +these volumes. (List volumes via `docker volume ls`) + ## Kubernetes setup +The cloud-hosted topology is composed via Kubernetes instead of docker-compose. +Follow the steps below to re-deploy the production setup. + 1. Configure gcloud & kubectl * https://cloud.google.com/kubernetes-engine/docs/quickstart 2. Configure PosgreSQL @@ -119,8 +141,33 @@ kubectl set image deployment/beamgrafana container= kubectl replace -f beamgrafana-deploy.yaml ``` - -## Useful Kubernetes commands and hints +## Configuring Grafana + +When you deploy a new Grafana instance, there is some one-time setup: + +1. Log-in at http://localhost:3000 with username `admin` and the value specified + for `GF_SECURITY_ADMIN_PASSWORD` in + [`docker-compose.yml`](docker-compose.yml). +1. Add Postgres as a data source: + 1. Click the 'Add data source' button. + 1. Fill out the following config: + * **Name**: BeamPSQL + * **Type**: PostgreSQL + * **Host** beampostgresql:5432 + * **Database**: beam\_metrics + * **User**: admin + * **Password**: `POSTGRES_PASSWORD` in + [`docker-compose.yml`](docker-compose.yml). + * **SSL Mode**: Disable +1. Restore dashboards from config + 1. In the Grafana sidebar, hover over the plus (+) and select 'Import' + 1. Select 'Upload .json File', and select the first exported JSON dashboard + file in [dashboards/](dashboards) + 1. Repeat for each of the remaining exported dashboards. + +## Appendix + +### Useful Kubernetes commands and hints ```sh # Get pods kubectl get pods @@ -135,7 +182,7 @@ kubectl log https://github.com/kubernetes/kubernetes/issues/35054 ``` -## Useful docker commands and hints +### Useful docker commands and hints * Connect from one container to another * `curl :` * Remove all containers/images/volumes diff --git a/.test-infra/metrics/dashboards/dashboard.json b/.test-infra/metrics/dashboards/dashboard.json deleted file mode 100644 index 2af9e18f4bfdc..0000000000000 --- a/.test-infra/metrics/dashboards/dashboard.json +++ /dev/null @@ -1,297 +0,0 @@ -{ - "annotations": { - "list": [ - { - "builtIn": 1, - "datasource": "-- Grafana --", - "enable": true, - "hide": true, - "iconColor": "rgba(0, 211, 255, 1)", - "limit": 100, - "name": "Annotations & Alerts", - "showIn": 0, - "type": "dashboard" - } - ] - }, - "editable": true, - "gnetId": null, - "graphTooltip": 0, - "id": 1, - "links": [], - "panels": [ - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "BeamPSQL", - "fill": 0, - "gridPos": { - "h": 9, - "w": 12, - "x": 0, - "y": 0 - }, - "id": 6, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": true, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "alias": "", - "format": "time_series", - "rawSql": "SELECT\n DATE_TRUNC('day', build_timestamp) as time,\n avg(\n case \n when build_result = 'SUCCESS' then 1\n else 0\n end) as value,\n job_name\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name like '%_PR')\nGROUP BY\n time, job_name\norder BY\n time\n", - "refId": "A" - } - ], - "thresholds": [ - { - "colorMode": "critical", - "fill": false, - "line": true, - "op": "lt", - "value": 0.85, - "yaxis": "left" - } - ], - "timeFrom": "14d", - "timeShift": null, - "title": "Greenness per day (in %)", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percentunit", - "label": "", - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "BeamPSQL", - "fill": 1, - "gridPos": { - "h": 9, - "w": 12, - "x": 12, - "y": 0 - }, - "id": 5, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "alias": "", - "format": "time_series", - "rawSql": "SELECT\n build_timestamp as time,\n build_duration as value,\n job_name\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name LIKE '%_PR')\nORDER BY\n time\n ", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeShift": null, - "title": "Job duration", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "ms", - "label": "", - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "columns": [], - "datasource": "BeamPSQL", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 24, - "x": 0, - "y": 9 - }, - "hideTimeOverride": false, - "id": 8, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "link": false, - "pattern": "Time", - "type": "date" - }, - { - "alias": "Build Url", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "link": true, - "linkUrl": "${__cell}", - "mappingType": 1, - "pattern": "build_url", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], - "targets": [ - { - "alias": "", - "format": "table", - "rawSql": "SELECT \n build_timestamp,\n job_name,\n build_url\nFROM jenkins_builds\nWHERE \n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name LIKE '%_PR')\n AND NOT (build_result = 'SUCCESS')\nORDER BY \n build_timestamp", - "refId": "A" - } - ], - "timeShift": null, - "title": "Failed builds", - "transform": "table", - "type": "table" - } - ], - "refresh": false, - "schemaVersion": 16, - "style": "dark", - "tags": [], - "templating": { - "list": [] - }, - "time": { - "from": "now-24h", - "to": "now" - }, - "timepicker": { - "hidden": false, - "refresh_intervals": [ - "1m", - "5m", - "15m", - "30m", - "1h", - "2h", - "1d" - ], - "time_options": [ - "5m", - "15m", - "1h", - "6h", - "12h", - "24h", - "2d", - "7d", - "30d" - ] - }, - "timezone": "", - "title": "Post-commit jobs", - "uid": "D81lW0pmk", - "version": 3 -} \ No newline at end of file diff --git a/.test-infra/metrics/dashboards/post-commit_tests.json b/.test-infra/metrics/dashboards/post-commit_tests.json new file mode 100644 index 0000000000000..fc838e58c510e --- /dev/null +++ b/.test-infra/metrics/dashboards/post-commit_tests.json @@ -0,0 +1,780 @@ +{ + "__inputs": [ + { + "name": "DS_BEAMPSQL", + "label": "BeamPSQL", + "description": "", + "type": "datasource", + "pluginId": "postgres", + "pluginName": "PostgreSQL" + } + ], + "__requires": [ + { + "type": "grafana", + "id": "grafana", + "name": "Grafana", + "version": "5.3.1" + }, + { + "type": "panel", + "id": "graph", + "name": "Graph", + "version": "5.0.0" + }, + { + "type": "datasource", + "id": "postgres", + "name": "PostgreSQL", + "version": "5.0.0" + }, + { + "type": "panel", + "id": "table", + "name": "Table", + "version": "5.0.0" + }, + { + "type": "panel", + "id": "text", + "name": "Text", + "version": "5.0.0" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "limit": 100, + "name": "Annotations & Alerts", + "showIn": 0, + "type": "dashboard" + } + ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": null, + "links": [], + "panels": [ + { + "content": "Three graphs show: average greenness per week, per day and average job runtime. These graphs show health of our project.\n\nTable shows list of job failed during selected time interval (You can change time interval on top-right corner of the dashboard). Please, triage failed jobs and update or create corresponding jira tickets.", + "gridPos": { + "h": 3, + "w": 11, + "x": 0, + "y": 0 + }, + "id": 11, + "links": [], + "mode": "markdown", + "title": "Dashboard guidelines", + "transparent": false, + "type": "text" + }, + { + "content": "[List existing jira tickets](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened)%20AND%20resolution%20%3D%20Unresolved%20AND%20component%20%3D%20test-failures%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC)\n\n[Create new Jira ticket](https://s.apache.org/beam-test-failure)", + "gridPos": { + "h": 3, + "w": 13, + "x": 11, + "y": 0 + }, + "id": 12, + "links": [], + "mode": "markdown", + "title": "Useful links", + "transparent": false, + "type": "text" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "fill": 0, + "gridPos": { + "h": 5, + "w": 11, + "x": 0, + "y": 3 + }, + "id": 6, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "", + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT\n DATE_TRUNC('week', build_timestamp) as time,\n avg(\n case \n when build_result = 'SUCCESS' then 1\n else 0\n end) as value,\n job_name\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name like '%_PR')\nGROUP BY\n time, job_name\norder BY\n job_name, time\n", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [ + { + "colorMode": "custom", + "fill": false, + "line": true, + "lineColor": "#3f6833", + "op": "lt", + "value": 0.85, + "yaxis": "left" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Greenness per Week (in %)", + "tooltip": { + "shared": true, + "sort": 1, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [], + "datasource": "${DS_BEAMPSQL}", + "fontSize": "100%", + "gridPos": { + "h": 18, + "w": 13, + "x": 11, + "y": 3 + }, + "hideTimeOverride": false, + "id": 8, + "links": [ + { + "includeVars": false, + "title": "Beam Jenkins", + "type": "absolute", + "url": "https://builds.apache.org/view/A-D/view/Beam/" + } + ], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "link": false, + "pattern": "Time", + "type": "date" + }, + { + "alias": "Build Url", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "link": true, + "linkTargetBlank": true, + "linkTooltip": "Link to Jenkins job.", + "linkUrl": "${__cell:raw}", + "mappingType": 1, + "pattern": "build_url", + "thresholds": [], + "type": "hidden", + "unit": "short" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "link": true, + "linkTargetBlank": true, + "linkTooltip": "Go to ${__cell_1}", + "linkUrl": "${__cell_2:raw}", + "mappingType": 1, + "pattern": "/.*/", + "preserveFormat": false, + "sanitize": true, + "thresholds": [], + "type": "string", + "unit": "short" + } + ], + "targets": [ + { + "alias": "", + "format": "table", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT \n build_timestamp,\n concat_ws(' ', job_name, build_id) as build,\n build_url\nFROM jenkins_builds\nWHERE \n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name LIKE '%_PR')\n AND NOT (build_result = 'SUCCESS')\nORDER BY \n build_timestamp", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "timeShift": null, + "title": "Failed builds", + "transform": "table", + "type": "table" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "fill": 0, + "gridPos": { + "h": 5, + "w": 11, + "x": 0, + "y": 8 + }, + "id": 9, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "", + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT\n DATE_TRUNC('day', build_timestamp) as time,\n avg(\n case \n when build_result = 'SUCCESS' then 1\n else 0\n end) as value,\n job_name\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name like '%_PR')\nGROUP BY\n time, job_name\norder BY\n job_name, time\n", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [ + { + "colorMode": "custom", + "fill": false, + "line": true, + "lineColor": "#3f6833", + "op": "lt", + "value": 0.85, + "yaxis": "left" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Greenness per day (in %)", + "tooltip": { + "shared": true, + "sort": 1, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "decimals": 1, + "fill": 1, + "gridPos": { + "h": 4, + "w": 11, + "x": 0, + "y": 13 + }, + "id": 5, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "", + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT\n build_timestamp as time,\n build_duration as value,\n job_name as metric\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND (job_name LIKE 'beam_PostCommit_%')\n AND NOT (job_name LIKE '%_PR')\nORDER BY\n job_name, time\n ", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Job duration", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ms", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "decimals": 0, + "fill": 0, + "gridPos": { + "h": 4, + "w": 11, + "x": 0, + "y": 17 + }, + "id": 14, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [ + { + "targetBlank": true, + "title": "Jira tickets", + "type": "absolute", + "url": "https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20resolution%20%3D%20Unresolved%20AND%20component%20%3D%20test-failures%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC" + } + ], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "format": "time_series", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with days as (select date_trunc('day', dd) as day from generate_series( $__timeFrom()::timestamp, $__timeTo()::timestamp, '1 day'::interval) as dd),\n knowndays as (SELECT days.day, count(*) as total_open\n FROM jira_issues, days\n WHERE jira_issues.created < days.day AND (jira_issues.resolutiondate > days.day OR jira_issues.resolutiondate is null)\n GROUP BY days.day\n ORDER BY days.day)\nselect days.day as time, greatest(knowndays.total_open, 0) as total_open\nfrom days left outer join knowndays\non days.day = knowndays.day", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + }, + { + "format": "time_series", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with days as (select date_trunc('day', dd) as day from generate_series( $__timeFrom()::timestamp, $__timeTo()::timestamp, '1 day'::interval) as dd),\n knowndays as (SELECT days.day, count(*) as currently_failing\n FROM jira_issues, days\n WHERE jira_issues.created < days.day AND (jira_issues.resolutiondate > days.day OR jira_issues.resolutiondate is null) AND (jira_issues.labels LIKE '%currently-failing%')\n GROUP BY days.day\n ORDER BY days.day)\nselect days.day as time, greatest(knowndays.currently_failing, 0) as currently_failing\nfrom days left outer join knowndays\non days.day = knowndays.day", + "refId": "D", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + }, + { + "format": "time_series", + "group": [], + "hide": false, + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with days as (select date_trunc('day', dd) as day from generate_series( $__timeFrom()::timestamp, $__timeTo()::timestamp, '1 day'::interval) as dd),\n knowndays as (SELECT days.day as day, count(*) as OpenedTickets\n FROM jira_issues, days\n WHERE date_trunc('day', jira_issues.created) = days.day\n GROUP BY day\n ORDER BY day\n )\nselect days.day as time, greatest(knowndays.OpenedTickets, 0) as opened_count\nfrom days left outer join knowndays\non days.day = knowndays.day\n", + "refId": "B", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + }, + { + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "with days as (select date_trunc('day', dd) as day, 0 as zcnt from generate_series( $__timeFrom()::timestamp, $__timeTo()::timestamp, '1 day'::interval) as dd),\n knowndays as (SELECT days.day as day, count(*) as ClosedTickets\n FROM jira_issues, days\n WHERE date_trunc('day', jira_issues.resolutiondate) = days.day\n GROUP BY day\n ORDER BY day\n )\nselect days.day as time, greatest(knowndays.ClosedTickets, days.zcnt) as closed_count\nfrom days left outer join knowndays\non days.day = knowndays.day\n", + "refId": "C", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [], + "timeFrom": null, + "timeShift": null, + "title": "Jira tickets", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "refresh": false, + "schemaVersion": 16, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-30d", + "to": "now" + }, + "timepicker": { + "hidden": false, + "refresh_intervals": [ + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "", + "title": "Post-commit Tests", + "uid": "D81lW0pmk", + "version": 33 +} \ No newline at end of file diff --git a/.test-infra/metrics/dashboards/pre-commit_tests.json b/.test-infra/metrics/dashboards/pre-commit_tests.json new file mode 100644 index 0000000000000..3c839edcb114d --- /dev/null +++ b/.test-infra/metrics/dashboards/pre-commit_tests.json @@ -0,0 +1,204 @@ +{ + "__inputs": [ + { + "name": "DS_BEAMPSQL", + "label": "BeamPSQL", + "description": "", + "type": "datasource", + "pluginId": "postgres", + "pluginName": "PostgreSQL" + } + ], + "__requires": [ + { + "type": "grafana", + "id": "grafana", + "name": "Grafana", + "version": "5.3.1" + }, + { + "type": "panel", + "id": "graph", + "name": "Graph", + "version": "5.0.0" + }, + { + "type": "datasource", + "id": "postgres", + "name": "PostgreSQL", + "version": "5.0.0" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": null, + "links": [], + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 4, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "", + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT\n build_timestamp as time,\n build_duration as value,\n job_name as metric\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND ((job_name LIKE 'beam_PreCommit_%_Commit')\n OR (job_name LIKE 'beam_PreCommit_%_Cron'))\nORDER BY\n time\n ", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [ + { + "colorMode": "critical", + "fill": false, + "line": true, + "op": "lt", + "value": 720000, + "yaxis": "left" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Job duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ms", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "schemaVersion": 16, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-7d", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "", + "title": "Pre-commit Tests", + "uid": "_TNndF2iz", + "version": 7 +} \ No newline at end of file diff --git a/.test-infra/metrics/dashboards/stability_critical_jobs_status.json b/.test-infra/metrics/dashboards/stability_critical_jobs_status.json new file mode 100644 index 0000000000000..c6c520ae3fe45 --- /dev/null +++ b/.test-infra/metrics/dashboards/stability_critical_jobs_status.json @@ -0,0 +1,331 @@ +{ + "__inputs": [ + { + "name": "DS_BEAMPSQL", + "label": "BeamPSQL", + "description": "", + "type": "datasource", + "pluginId": "postgres", + "pluginName": "PostgreSQL" + } + ], + "__requires": [ + { + "type": "grafana", + "id": "grafana", + "name": "Grafana", + "version": "5.3.1" + }, + { + "type": "panel", + "id": "graph", + "name": "Graph", + "version": "5.0.0" + }, + { + "type": "datasource", + "id": "postgres", + "name": "PostgreSQL", + "version": "5.0.0" + }, + { + "type": "panel", + "id": "table", + "name": "Table", + "version": "5.0.0" + }, + { + "type": "panel", + "id": "text", + "name": "Text", + "version": "5.0.0" + } + ], + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "gnetId": null, + "graphTooltip": 0, + "id": null, + "links": [], + "panels": [ + { + "content": "The graph shows: average greenness of critical post-commit tests jobs per week. This graph show health of our project.\n\nTable shows list of relevant jobs failures during selected time interval (You can change time interval on top-right corner of the dashboard). Please, triage failed jobs and update or create corresponding jira tickets. You can utilized provided links to help with this.", + "gridPos": { + "h": 3, + "w": 10, + "x": 0, + "y": 0 + }, + "id": 8, + "links": [], + "mode": "markdown", + "title": "Dashboard guidelines", + "transparent": false, + "type": "text" + }, + { + "columns": [], + "datasource": "${DS_BEAMPSQL}", + "fontSize": "100%", + "gridPos": { + "h": 13, + "w": 14, + "x": 10, + "y": 0 + }, + "hideTimeOverride": false, + "id": 4, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "link": false, + "pattern": "Time", + "type": "date" + }, + { + "alias": "Build Url", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "link": true, + "linkTargetBlank": true, + "linkTooltip": "Link to Jenkins job.", + "linkUrl": "${__cell:raw}", + "mappingType": 1, + "pattern": "build_url", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "alias": "", + "format": "table", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT \n build_timestamp,\n job_name,\n build_url\nFROM jenkins_builds\nWHERE \n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND ((job_name LIKE 'beam_PostCommit_Java_GradleBuild') \n OR (job_name LIKE 'beam_PostCommit_Go_GradleBuild') \n OR (job_name LIKE 'beam_PostCommit_Python_Verify')\n OR (job_name LIKE 'beam_PostCommit_Website_Publish'))\n AND NOT (job_name LIKE '%_PR')\n AND NOT (build_result = 'SUCCESS')\nORDER BY \n build_timestamp", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "timeShift": null, + "title": "Failed builds", + "transform": "table", + "type": "table" + }, + { + "content": "[List existing jira tickets](https://issues.apache.org/jira/issues/?jql=project%20%3D%20BEAM%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20Reopened)%20AND%20resolution%20%3D%20Unresolved%20AND%20component%20%3D%20test-failures%20ORDER%20BY%20priority%20DESC%2C%20updated%20DESC)\n\n[Create new Jira ticket](https://issues.apache.org/jira/secure/CreateIssueDetails!init.jspa?pid=12319527&issuetype=1&summary=%5BjobName%5D%5BTestName%5D%5BIsFlake%5D%20Failure%20summary&priority=3&components=12334203&description=%3CFailure%20summary%3E%0AFailing%20job%20url:%0AJob%20history%20url:%0ARelevant%20log:)", + "gridPos": { + "h": 3, + "w": 10, + "x": 0, + "y": 3 + }, + "id": 6, + "links": [], + "mode": "markdown", + "title": "Useful links", + "transparent": false, + "type": "text" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_BEAMPSQL}", + "fill": 0, + "gridPos": { + "h": 7, + "w": 10, + "x": 0, + "y": 6 + }, + "id": 2, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "alias": "", + "format": "time_series", + "group": [], + "metricColumn": "none", + "rawQuery": true, + "rawSql": "SELECT\n DATE_TRUNC('week', build_timestamp) as time,\n avg(\n case \n when build_result = 'SUCCESS' then 1\n else 0\n end) as value,\n job_name\nFROM\n jenkins_builds\nWHERE\n (build_timestamp BETWEEN $__timeFrom() AND $__timeTo())\n AND ((job_name LIKE 'beam_PostCommit_Java_GradleBuild') \n OR (job_name LIKE 'beam_PostCommit_Go_GradleBuild') \n OR (job_name LIKE 'beam_PostCommit_Python_Verify')\n OR (job_name LIKE 'beam_PostCommit_Website_Publish'))\n AND NOT (job_name like '%_PR')\nGROUP BY\n time, job_name\norder BY\n job_name, time\n", + "refId": "A", + "select": [ + [ + { + "params": [ + "value" + ], + "type": "column" + } + ] + ], + "timeColumn": "time", + "where": [ + { + "name": "$__timeFilter", + "params": [], + "type": "macro" + } + ] + } + ], + "thresholds": [ + { + "colorMode": "custom", + "fill": false, + "line": true, + "lineColor": "#3f6833", + "op": "lt", + "value": 0.7, + "yaxis": "left" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Greenness per Week (in %)", + "tooltip": { + "shared": true, + "sort": 1, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "schemaVersion": 16, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-30d", + "to": "now" + }, + "timepicker": { + "refresh_intervals": [ + "5s", + "10s", + "30s", + "1m", + "5m", + "15m", + "30m", + "1h", + "2h", + "1d" + ], + "time_options": [ + "5m", + "15m", + "1h", + "6h", + "12h", + "24h", + "2d", + "7d", + "30d" + ] + }, + "timezone": "", + "title": "Stability critical jobs status", + "uid": "McTAiu0ik", + "version": 8 +} \ No newline at end of file From 4e8ef85708008bf8371facd1453c1718f0118692 Mon Sep 17 00:00:00 2001 From: Micah Wylde Date: Mon, 22 Oct 2018 13:34:10 -0700 Subject: [PATCH 17/42] [BEAM-5797] Ensure bundle factory is always closed on dispose() --- .../wrappers/streaming/ExecutableStageDoFnOperator.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index ee49ef451db2d..2135d5a828526 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -162,13 +162,14 @@ private StateRequestHandler getStateRequestHandler(ExecutableStage executableSta @Override public void dispose() throws Exception { - // DoFnOperator generates another "bundle" for the final watermark - super.dispose(); // Remove the reference to stageContext and make stageContext available for garbage collection. try (@SuppressWarnings("unused") AutoCloseable bundleFactoryCloser = stageBundleFactory; @SuppressWarnings("unused") - AutoCloseable closable = stageContext) {} + AutoCloseable closable = stageContext) { + // DoFnOperator generates another "bundle" for the final watermark -- see BEAM-5816 for more context + super.dispose(); + } stageContext = null; } From ca92daa99a0736bfe5b6033c22e6d2a4691e02de Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 Oct 2018 21:06:05 -0700 Subject: [PATCH 18/42] [BEAM-5829] Convert tests to use DECIMAL for price field --- .../sql/impl/rel/BeamIntersectRelTest.java | 51 +++++++++++++++---- .../sql/impl/rel/BeamMinusRelTest.java | 41 ++++++++++++--- .../sql/impl/rel/BeamUnionRelTest.java | 25 ++++++--- 3 files changed, 93 insertions(+), 24 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java index 9ce389bdc33a2..72e8144f1505a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel; +import java.math.BigDecimal; import org.apache.beam.sdk.extensions.sql.TestUtils; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; import org.apache.beam.sdk.schemas.Schema; @@ -41,20 +42,41 @@ public static void prepare() { TestBoundedTable.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 1L, 1, 1.0, 2L, 2, 2.0, 4L, 4, 4.0)); + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0), + 4L, + 4, + new BigDecimal(4.0))); registerTable( "ORDER_DETAILS2", TestBoundedTable.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 2L, 2, 2.0, 3L, 3, 3.0)); + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0), + 3L, + 3, + new BigDecimal(3.0))); } @Test - public void testIntersect() throws Exception { + public void testIntersect() { String sql = ""; sql += "SELECT order_id, site_id, price " @@ -69,15 +91,15 @@ public void testIntersect() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 2L, 2, 2.0) + Schema.FieldType.DECIMAL, "price") + .addRows(1L, 1, new BigDecimal(1.0), 2L, 2, new BigDecimal(2.0)) .getRows()); pipeline.run().waitUntilFinish(); } @Test - public void testIntersectAll() throws Exception { + public void testIntersectAll() { String sql = ""; sql += "SELECT order_id, site_id, price " @@ -94,8 +116,17 @@ public void testIntersectAll() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 1L, 1, 1.0, 2L, 2, 2.0) + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0)) .getRows()); pipeline.run(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java index 02e84b98b4706..20b0e63b524e6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel; +import java.math.BigDecimal; import org.apache.beam.sdk.extensions.sql.TestUtils; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; import org.apache.beam.sdk.schemas.Schema; @@ -40,16 +41,40 @@ public static void prepare() { TestBoundedTable.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 1L, 1, 1.0, 2L, 2, 2.0, 4L, 4, 4.0, 4L, 4, 4.0)); + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0), + 4L, + 4, + new BigDecimal(4.0), + 4L, + 4, + new BigDecimal(4.0))); registerTable( "ORDER_DETAILS2", TestBoundedTable.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 2L, 2, 2.0, 3L, 3, 3.0)); + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0), + 3L, + 3, + new BigDecimal(3.0))); } @Test @@ -68,8 +93,8 @@ public void testExcept() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(4L, 4, 4.0) + Schema.FieldType.DECIMAL, "price") + .addRows(4L, 4, new BigDecimal(4.0)) .getRows()); pipeline.run(); @@ -93,8 +118,8 @@ public void testExceptAll() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(4L, 4, 4.0, 4L, 4, 4.0) + Schema.FieldType.DECIMAL, "price") + .addRows(4L, 4, new BigDecimal(4.0), 4L, 4, new BigDecimal(4.0)) .getRows()); pipeline.run(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java index 63e63e87f148a..55368ae6013e2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.sql.impl.rel; +import java.math.BigDecimal; import org.apache.beam.sdk.extensions.sql.TestUtils; import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable; import org.apache.beam.sdk.schemas.Schema; @@ -40,8 +41,8 @@ public static void prepare() { TestBoundedTable.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 2L, 2, 2.0)); + Schema.FieldType.DECIMAL, "price") + .addRows(1L, 1, new BigDecimal(1.0), 2L, 2, new BigDecimal(2.0))); } @Test @@ -60,8 +61,8 @@ public void testUnion() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 2L, 2, 2.0) + Schema.FieldType.DECIMAL, "price") + .addRows(1L, 1, new BigDecimal(1.0), 2L, 2, new BigDecimal(2.0)) .getRows()); pipeline.run(); } @@ -82,8 +83,20 @@ public void testUnionAll() throws Exception { TestUtils.RowsBuilder.of( Schema.FieldType.INT64, "order_id", Schema.FieldType.INT32, "site_id", - Schema.FieldType.DOUBLE, "price") - .addRows(1L, 1, 1.0, 1L, 1, 1.0, 2L, 2, 2.0, 2L, 2, 2.0) + Schema.FieldType.DECIMAL, "price") + .addRows( + 1L, + 1, + new BigDecimal(1.0), + 1L, + 1, + new BigDecimal(1.0), + 2L, + 2, + new BigDecimal(2.0), + 2L, + 2, + new BigDecimal(2.0)) .getRows()); pipeline.run(); } From 5f3ced20289c4fc5692efc9b2ed3cf46d1891eda Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 22 Oct 2018 21:31:31 -0700 Subject: [PATCH 19/42] [BEAM-5830] Use the word LANGUAGE instead of SDK on site --- website/src/_includes/header.html | 2 +- website/src/_includes/section-menu/sdks.html | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/website/src/_includes/header.html b/website/src/_includes/header.html index 37aadc9d5e78d..072c927bcd036 100644 --- a/website/src/_includes/header.html +++ b/website/src/_includes/header.html @@ -35,7 +35,7 @@ Documentation
  • - SDKS + Languages
  • RUNNERS diff --git a/website/src/_includes/section-menu/sdks.html b/website/src/_includes/section-menu/sdks.html index e9a661ab96f8e..7f723ea302a9f 100644 --- a/website/src/_includes/section-menu/sdks.html +++ b/website/src/_includes/section-menu/sdks.html @@ -10,7 +10,7 @@ limitations under the License. See accompanying LICENSE file. --> -
  • SDKS
  • +
  • Languages
  • Java @@ -42,7 +42,7 @@
  • - Go SDK + Go