You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2019/11/06 16:47:23 UTC

[impala] branch master updated (4bffd12 -> 8b8a49e)

This is an automated email from the ASF dual-hosted git repository.

stakiar pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 4bffd12  IMPALA-8692 Gracefully fail complex type inserts
     new e8fda1f  IMPALA-9117, IMPALA-7726: Fixed a few unit tests for ABFS
     new 8b8a49e  IMPALA-8557: Add '.txt' to text files, remove '.' at end of filenames

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/exec/hdfs-table-sink.cc        |  6 ++++--
 be/src/exec/hdfs-text-table-writer.cc |  2 +-
 tests/common/skip.py                  |  2 ++
 tests/custom_cluster/test_lineage.py  |  4 +++-
 tests/custom_cluster/test_mt_dop.py   |  3 ++-
 tests/metadata/test_ddl.py            |  2 --
 tests/query_test/test_insert.py       | 32 +++++++++++++++++++++++++++++++-
 7 files changed, 43 insertions(+), 8 deletions(-)


[impala] 02/02: IMPALA-8557: Add '.txt' to text files, remove '.' at end of filenames

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stakiar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 8b8a49e617818e9bcf99b784b63587c95cebd622
Author: Sahil Takiar <ta...@gmail.com>
AuthorDate: Fri Nov 1 20:30:34 2019 +0000

    IMPALA-8557: Add '.txt' to text files, remove '.' at end of filenames
    
    Writes to text tables on ABFS are failing because HADOOP-15860 recently
    changed the ABFS behavior when writing files / folders that end with a
    '.'. ABFS explicitly does not allow files / folders that end with a dot.
    From the ABFS docs: "Avoid blob names that end with a dot (.), a forward
    slash (/), or a sequence or combination of the two."
    
    The behavior prior to HADOOP-15860 was to simply drop any trailing dots
    when writing files or folders, but that can lead to various issues
    because clients may try to read back a file that should exist on ABFS,
    but doesn't. HADOOP-15860 changed the behavior so that any attempt to
    write a file or folder with a trailing dot fails on ABFS.
    
    Impala writes all text files with a trailing dot due to some odd
    behavior in hdfs-table-sink.cc. The table sink writes files with
    a "file extension" which is dependent on the file type. For example,
    Parquet files have a file extension of ".parq". For some reason, text
    files had no file extension, so Impala would try to write text files of
    the following form:
    "244c5ee8ece6f759-8b1a1e3b00000000_45513034_data.0.".
    
    Several tables created during dataload, such as alltypes, already use
    the '.txt' extension for their files. These tables are not created via
    Impala's INSERT code path, they are copied into the table. However,
    there are several tables created during dataload, such as
    alltypesinsert, that are created via Impala. This patch will change
    the files in these tables so that they end in '.txt'.
    
    This patch adds the ".txt" extension to all written text files and
    modifies the hdfs-table-sink.cc so that it doesn't add a trailing dot to
    a filename if there is no file extension.
    
    Testing:
    * Ran core tests
    * Re-ran affected ABFS tests
    * Added test to validate that the correct file extension is used for
    Parquet and text tables
    * Manually validated that without the addition of the '.txt' file
    extension, files are not written with a trailing dot
    
    Change-Id: I2a9adacd45855cde86724e10f8a131e17ebf46f8
    Reviewed-on: http://gerrit.cloudera.org:8080/14621
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/hdfs-table-sink.cc        |  6 ++++--
 be/src/exec/hdfs-text-table-writer.cc |  2 +-
 tests/query_test/test_insert.py       | 30 ++++++++++++++++++++++++++++++
 3 files changed, 35 insertions(+), 3 deletions(-)

diff --git a/be/src/exec/hdfs-table-sink.cc b/be/src/exec/hdfs-table-sink.cc
index ea53d1a..3e695b4 100644
--- a/be/src/exec/hdfs-table-sink.cc
+++ b/be/src/exec/hdfs-table-sink.cc
@@ -340,7 +340,9 @@ Status HdfsTableSink::WriteClusteredRowBatch(RuntimeState* state, RowBatch* batc
 Status HdfsTableSink::CreateNewTmpFile(RuntimeState* state,
     OutputPartition* output_partition) {
   SCOPED_TIMER(ADD_TIMER(profile(), "TmpFileCreateTimer"));
-  string final_location = Substitute("$0.$1.$2",
+  string file_name_pattern =
+      output_partition->writer->file_extension().empty() ? "$0.$1" : "$0.$1.$2";
+  string final_location = Substitute(file_name_pattern,
       output_partition->final_hdfs_file_name_prefix, output_partition->num_files,
       output_partition->writer->file_extension());
 
@@ -351,7 +353,7 @@ Status HdfsTableSink::CreateNewTmpFile(RuntimeState* state,
   if (ShouldSkipStaging(state, output_partition)) {
     output_partition->current_file_name = final_location;
   } else {
-    output_partition->current_file_name = Substitute("$0.$1.$2",
+    output_partition->current_file_name = Substitute(file_name_pattern,
       output_partition->tmp_hdfs_file_name_prefix, output_partition->num_files,
       output_partition->writer->file_extension());
   }
diff --git a/be/src/exec/hdfs-text-table-writer.cc b/be/src/exec/hdfs-text-table-writer.cc
index f09b161..6b77852 100644
--- a/be/src/exec/hdfs-text-table-writer.cc
+++ b/be/src/exec/hdfs-text-table-writer.cc
@@ -62,7 +62,7 @@ void HdfsTextTableWriter::Close() {
 
 uint64_t HdfsTextTableWriter::default_block_size() const { return 0; }
 
-string HdfsTextTableWriter::file_extension() const { return ""; }
+string HdfsTextTableWriter::file_extension() const { return "txt"; }
 
 Status HdfsTextTableWriter::AppendRows(
     RowBatch* batch, const vector<int32_t>& row_group_indices, bool* new_file) {
diff --git a/tests/query_test/test_insert.py b/tests/query_test/test_insert.py
index 7807030..edbf853 100644
--- a/tests/query_test/test_insert.py
+++ b/tests/query_test/test_insert.py
@@ -309,3 +309,33 @@ class TestInsertNullQueries(ImpalaTestSuite):
   @pytest.mark.execute_serially
   def test_insert_null(self, vector):
     self.run_test_case('QueryTest/insert_null', vector)
+
+
+class TestInsertFileExtension(ImpalaTestSuite):
+  """Tests that files written to a table have the correct file extension. Asserts that
+  Parquet files end with .parq and text files end with .txt."""
+
+  @classmethod
+  def get_workload(self):
+    return 'functional-query'
+
+  @classmethod
+  def add_test_dimensions(cls):
+    cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension(
+        'table_format_and_file_extension',
+        *[('parquet', '.parq'), ('textfile', '.txt')]))
+
+  @classmethod
+  def setup_class(cls):
+    super(TestInsertFileExtension, cls).setup_class()
+
+  def test_file_extension(self, vector, unique_database):
+    table_format = vector.get_value('table_format_and_file_extension')[0]
+    file_extension = vector.get_value('table_format_and_file_extension')[1]
+    table_name = "{0}_table".format(table_format)
+    ctas_query = "create table {0}.{1} stored as {2} as select 1".format(
+        unique_database, table_name, table_format)
+    self.execute_query_expect_success(self.client, ctas_query)
+    for path in self.filesystem_client.ls("test-warehouse/{0}.db/{1}".format(
+        unique_database, table_name)):
+      if not path.startswith('_'): assert path.endswith(file_extension)


[impala] 01/02: IMPALA-9117, IMPALA-7726: Fixed a few unit tests for ABFS

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

stakiar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit e8fda1f224d3ad237183a53e238eee90188d82e2
Author: Sahil Takiar <ta...@gmail.com>
AuthorDate: Fri Nov 1 21:29:00 2019 +0000

    IMPALA-9117, IMPALA-7726: Fixed a few unit tests for ABFS
    
    This test makes the following changes / fixes when running Impala tests
    on ABFS:
    * Skips some tests in test_lineage.py that don't work on ABFS / ADLS
    (they were already skipped for S3)
    * Skips some tests in test_mt_dop.py; the test creates a directory that
    ends with a period (and ABFS does not support writing files or
    directories that end with a period)
    * Removes the ABFS skip flag SkipIfABFS.trash (IMPALA-7726: Drop with
    purge tests fail against ABFS due to trash misbehavior"); I removed
    these flags and looped the tests overnight with no failures, so it is
    likely whatever bug was causing this has now been fixed
    * Now that HADOOP-15860 has been resolved, and the agreed upon behavior
    for ABFS is that it will fail if a client tries to write a file /
    directory that ends with a period, I added a new entry to the SkipIfABFS
    class called file_or_folder_name_ends_with_period and applied it where
    necessary
    
    Testing:
    * Ran core tests on ABFS
    
    Change-Id: I18ae5b0f7de6aa7628a1efd780ff30a0cc3c5285
    Reviewed-on: http://gerrit.cloudera.org:8080/14636
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/common/skip.py                 | 2 ++
 tests/custom_cluster/test_lineage.py | 4 +++-
 tests/custom_cluster/test_mt_dop.py  | 3 ++-
 tests/metadata/test_ddl.py           | 2 --
 tests/query_test/test_insert.py      | 2 +-
 5 files changed, 8 insertions(+), 5 deletions(-)

diff --git a/tests/common/skip.py b/tests/common/skip.py
index 1426b86..5203de5 100644
--- a/tests/common/skip.py
+++ b/tests/common/skip.py
@@ -75,6 +75,8 @@ class SkipIfABFS:
       reason="HDFS encryption is not supported with ABFS")
   trash = pytest.mark.skipif(IS_ABFS,
       reason="Drop/purge not working as expected on ABFS, IMPALA-7726")
+  file_or_folder_name_ends_with_period = pytest.mark.skipif(IS_ABFS,
+      reason="ABFS does not support file / directories that end with a period")
 
   # These need test infra work to re-enable.
   udfs = pytest.mark.skipif(IS_ABFS, reason="udas/udfs not copied to ABFS")
diff --git a/tests/custom_cluster/test_lineage.py b/tests/custom_cluster/test_lineage.py
index 850ae65..73f785d 100644
--- a/tests/custom_cluster/test_lineage.py
+++ b/tests/custom_cluster/test_lineage.py
@@ -26,7 +26,7 @@ import tempfile
 import time
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
-from tests.common.skip import SkipIfS3
+from tests.common.skip import SkipIfABFS, SkipIfADLS, SkipIfS3
 
 LOG = logging.getLogger(__name__)
 
@@ -132,6 +132,8 @@ class TestLineage(CustomClusterTestSuite):
           assert lineage_json["queryText"] == query
           assert lineage_json["tableLocation"] is not None
 
+  @SkipIfABFS.hbase
+  @SkipIfADLS.hbase
   @SkipIfS3.hbase
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args("--lineage_event_log_dir={0}"
diff --git a/tests/custom_cluster/test_mt_dop.py b/tests/custom_cluster/test_mt_dop.py
index 0bf8a1b..d4f664e 100644
--- a/tests/custom_cluster/test_mt_dop.py
+++ b/tests/custom_cluster/test_mt_dop.py
@@ -20,7 +20,7 @@ from copy import deepcopy
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
 from tests.common.environ import build_flavor_timeout
-from tests.common.skip import SkipIfNotHdfsMinicluster
+from tests.common.skip import SkipIfABFS, SkipIfNotHdfsMinicluster
 
 WAIT_TIME_MS = build_flavor_timeout(60000, slow_build_timeout=100000)
 
@@ -36,6 +36,7 @@ class TestMtDopFlags(CustomClusterTestSuite):
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(impalad_args="--unlock_mt_dop=true")
+  @SkipIfABFS.file_or_folder_name_ends_with_period
   def test_mt_dop_all(self, vector, unique_database):
     """Test joins, inserts and runtime filters with mt_dop > 0"""
     vector = deepcopy(vector)
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 3a3c982..c51e0f4 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -40,7 +40,6 @@ from tests.common.impala_cluster import ImpalaCluster
 # Validates DDL statements (create, drop)
 class TestDdlStatements(TestDdlBase):
   @SkipIfLocal.hdfs_client
-  @SkipIfABFS.trash
   def test_drop_table_with_purge(self, unique_database):
     """This test checks if the table data is permanently deleted in
     DROP TABLE <tbl> PURGE queries"""
@@ -451,7 +450,6 @@ class TestDdlStatements(TestDdlBase):
         use_db=unique_database, multiple_impalad=self._use_multiple_impalad(vector))
 
   @SkipIfLocal.hdfs_client
-  @SkipIfABFS.trash
   def test_drop_partition_with_purge(self, vector, unique_database):
     """Verfies whether alter <tbl> drop partition purge actually skips trash"""
     self.client.execute(
diff --git a/tests/query_test/test_insert.py b/tests/query_test/test_insert.py
index 173a272..7807030 100644
--- a/tests/query_test/test_insert.py
+++ b/tests/query_test/test_insert.py
@@ -129,7 +129,7 @@ class TestInsertQueries(ImpalaTestSuite):
   # Erasure coding doesn't respect memory limit
   @SkipIfEC.fix_later
   # ABFS partition names cannot end in periods
-  @SkipIfABFS.jira(reason="HADOOP-15860")
+  @SkipIfABFS.file_or_folder_name_ends_with_period
   def test_insert(self, vector):
     if (vector.get_value('table_format').file_format == 'parquet'):
       vector.get_value('exec_option')['COMPRESSION_CODEC'] = \