You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/04/04 19:04:05 UTC

[1/5] impala git commit: IMPALA-6571: NullPointerException in SHOW CREATE TABLE for HBase tables

Repository: impala
Updated Branches:
  refs/heads/master cf76953cc -> 890bd4c72


IMPALA-6571: NullPointerException in SHOW CREATE TABLE for HBase tables

This patch fixes the NullPointerException in SHOW CREATE TABLE for HBase
tables.

Testing:
- Moved the content of back hbase-show-create-table.test to
  show-create-table.test
- Ran show-create-table end-to-end tests

Change-Id: Ibe018313168fac5dcbd80be9a8f28b71a2c0389b
Reviewed-on: http://gerrit.cloudera.org:8080/9884
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 8173e9ab4d8c7e1ad6875e7696ea347c6a604d9a
Parents: cf76953
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Mon Apr 2 09:19:36 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Apr 4 00:12:30 2018 +0000

----------------------------------------------------------------------
 .../org/apache/impala/analysis/ToSqlUtils.java  | 11 +++++----
 .../QueryTest/hbase-show-create-table.test      | 24 --------------------
 .../queries/QueryTest/show-create-table.test    | 24 ++++++++++++++++++++
 tests/metadata/test_show_create_table.py        |  3 ++-
 4 files changed, 33 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/8173e9ab/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
index 1940cde..9a164f0 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ToSqlUtils.java
@@ -45,6 +45,7 @@ import org.apache.impala.catalog.Function;
 import org.apache.impala.catalog.HBaseTable;
 import org.apache.impala.catalog.HdfsCompression;
 import org.apache.impala.catalog.HdfsFileFormat;
+import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.KuduColumn;
 import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.RowFormat;
@@ -231,10 +232,8 @@ public class ToSqlUtils {
       }
     }
     RowFormat rowFormat = RowFormat.fromStorageDescriptor(msTable.getSd());
-    HdfsFileFormat format = HdfsFileFormat.fromHdfsInputFormatClass(
-        msTable.getSd().getInputFormat());
-    HdfsCompression compression = HdfsCompression.fromHdfsInputFormatClass(
-        msTable.getSd().getInputFormat());
+    HdfsFileFormat format = null;
+    HdfsCompression compression = null;
     String location = isHbaseTable ? null : msTable.getSd().getLocation();
     Map<String, String> serdeParameters = msTable.getSd().getSerdeInfo().getParameters();
 
@@ -270,6 +269,10 @@ public class ToSqlUtils {
         // We shouldn't output the columns for external tables
         colsSql = null;
       }
+    } else if (table instanceof HdfsTable) {
+      String inputFormat = msTable.getSd().getInputFormat();
+      format = HdfsFileFormat.fromHdfsInputFormatClass(inputFormat);
+      compression = HdfsCompression.fromHdfsInputFormatClass(inputFormat);
     }
     HdfsUri tableLocation = location == null ? null : new HdfsUri(location);
     return getCreateTableSql(table.getDb().getName(), table.getName(), comment, colsSql,

http://git-wip-us.apache.org/repos/asf/impala/blob/8173e9ab/testdata/workloads/functional-query/queries/QueryTest/hbase-show-create-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/hbase-show-create-table.test b/testdata/workloads/functional-query/queries/QueryTest/hbase-show-create-table.test
deleted file mode 100644
index 8062216..0000000
--- a/testdata/workloads/functional-query/queries/QueryTest/hbase-show-create-table.test
+++ /dev/null
@@ -1,24 +0,0 @@
-====
----- QUERY
-SHOW CREATE TABLE alltypes
----- RESULTS
-CREATE EXTERNAL TABLE alltypes (
-  id INT COMMENT 'Add a comment',
-  bigint_col BIGINT,
-  bool_col BOOLEAN,
-  date_string_col STRING,
-  double_col DOUBLE,
-  float_col FLOAT,
-  int_col INT,
-  month INT,
-  smallint_col SMALLINT,
-  string_col STRING,
-  timestamp_col TIMESTAMP,
-  tinyint_col TINYINT,
-  year INT
-)
-STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
-WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,d:bool_col,d:tinyint_col,d:smallint_col,d:int_col,d:bigint_col,d:float_col,d:double_col,d:date_string_col,d:string_col,d:timestamp_col,d:year,d:month',
-                      'serialization.format'='1')
-TBLPROPERTIES ('hbase.table.name'='functional_hbase.alltypes',
-               'storage_handler'='org.apache.hadoop.hive.hbase.HBaseStorageHandler')

http://git-wip-us.apache.org/repos/asf/impala/blob/8173e9ab/testdata/workloads/functional-query/queries/QueryTest/show-create-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/show-create-table.test b/testdata/workloads/functional-query/queries/QueryTest/show-create-table.test
index 81b619b..db38984 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/show-create-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/show-create-table.test
@@ -372,3 +372,27 @@ SORT BY (id)
 STORED AS TEXTFILE
 LOCATION '$$location_uri$$'
 ====
+---- QUERY
+SHOW CREATE TABLE functional_hbase.alltypes
+---- RESULTS
+CREATE EXTERNAL TABLE functional_hbase.alltypes (
+  id INT COMMENT 'Add a comment',
+  bigint_col BIGINT,
+  bool_col BOOLEAN,
+  date_string_col STRING,
+  double_col DOUBLE,
+  float_col FLOAT,
+  int_col INT,
+  month INT,
+  smallint_col SMALLINT,
+  string_col STRING,
+  timestamp_col TIMESTAMP,
+  tinyint_col TINYINT,
+  year INT
+)
+STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler'
+WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,d:bool_col,d:tinyint_col,d:smallint_col,d:int_col,d:bigint_col,d:float_col,d:double_col,d:date_string_col,d:string_col,d:timestamp_col,d:year,d:month',
+                      'serialization.format'='1')
+TBLPROPERTIES ('hbase.table.name'='functional_hbase.alltypes',
+               'storage_handler'='org.apache.hadoop.hive.hbase.HBaseStorageHandler')
+====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/8173e9ab/tests/metadata/test_show_create_table.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_show_create_table.py b/tests/metadata/test_show_create_table.py
index d0ce87f..00b63ef 100644
--- a/tests/metadata/test_show_create_table.py
+++ b/tests/metadata/test_show_create_table.py
@@ -34,7 +34,8 @@ class TestShowCreateTable(ImpalaTestSuite):
   # Properties to filter before comparing results
   FILTER_TBL_PROPERTIES = ["transient_lastDdlTime", "numFiles", "numPartitions",
                            "numRows", "rawDataSize", "totalSize", "COLUMN_STATS_ACCURATE",
-                           "STATS_GENERATED_VIA_STATS_TASK"]
+                           "STATS_GENERATED_VIA_STATS_TASK", "last_modified_by",
+                           "last_modified_time"]
 
   @classmethod
   def get_workload(self):


[5/5] impala git commit: IMPALA-6785: reset failed heartbeat count when re-registering

Posted by ta...@apache.org.
IMPALA-6785: reset failed heartbeat count when re-registering

When a subscriber re-registers with the same subscriber ID, we need
to reset failure detection info so that we don't erroneously think
that the subscriber has failed.

The bug is a mix-up between whether the subscriber or registration ID is
the key for the failure detected. The symptom was that, if the first
topic update won the race with the first heartbeat, no subsequent topic
updates were delivered. It was introduced when IMPALA-3613 updated some,
but not all, places to use the subscriber ID as the key. It wasn't
detected because we had no tests directly testing this code path, and
it appears that the race almost never happens on smaller clusters.

This patch fixes the problem in two places. Fixing either place is
actually sufficient to fix the bug.

Testing:
Add a regression that reliably reproduces the issue by delaying the
heartbeat and checking that topic updates continue to be sent. The
test reliably fails before the fix and passes after.

Change-Id: I2ad409e2a8e22d081fce97b085b9469ab046bf07
Reviewed-on: http://gerrit.cloudera.org:8080/9913
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-by: Dan Hecht <dh...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 890bd4c72babef838bf00b71d62ac9605719e897
Parents: 4c1f0ac
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Tue Apr 3 14:13:28 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Apr 4 18:58:42 2018 +0000

----------------------------------------------------------------------
 be/src/statestore/statestore.cc     |  8 +++++---
 be/src/statestore/statestore.h      |  6 +++++-
 tests/statestore/test_statestore.py | 28 +++++++++++++++++++++++++++-
 3 files changed, 37 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/890bd4c7/be/src/statestore/statestore.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index 1072dee..02363fe 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -548,8 +548,7 @@ Status Statestore::RegisterSubscriber(const SubscriberId& subscriber_id,
     shared_ptr<Subscriber> current_registration(
         new Subscriber(subscriber_id, *registration_id, location, topic_registrations));
     subscribers_.emplace(subscriber_id, current_registration);
-    failure_detector_->UpdateHeartbeat(
-        PrintId(current_registration->registration_id()), true);
+    failure_detector_->UpdateHeartbeat(subscriber_id, true);
     num_subscribers_metric_->SetValue(subscribers_.size());
     subscriber_set_metric_->Add(subscriber_id);
 
@@ -878,6 +877,9 @@ void Statestore::DoSubscriberUpdate(UpdateKind update_kind, int thread_id,
                   << "or re-registered (last known registration ID: "
                   << update.registration_id << ")";
         UnregisterSubscriber(subscriber.get());
+      } else {
+        LOG(INFO) << "Failure was already detected for subscriber '" << subscriber->id()
+                  << "'. Won't send another " << update_kind_str;
       }
     } else {
       // Schedule the next message.
@@ -907,7 +909,7 @@ void Statestore::UnregisterSubscriber(Subscriber* subscriber) {
   heartbeat_client_cache_->CloseConnections(subscriber->network_address());
 
   // Prevent the failure detector from growing without bound
-  failure_detector_->EvictPeer(PrintId(subscriber->registration_id()));
+  failure_detector_->EvictPeer(subscriber->id());
 
   // Delete all transient entries
   {

http://git-wip-us.apache.org/repos/asf/impala/blob/890bd4c7/be/src/statestore/statestore.h
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.h b/be/src/statestore/statestore.h
index 3058e94..482e48b 100644
--- a/be/src/statestore/statestore.h
+++ b/be/src/statestore/statestore.h
@@ -517,7 +517,11 @@ class Statestore : public CacheLineAligned {
 
   /// Failure detector for subscribers. If a subscriber misses a configurable number of
   /// consecutive heartbeat messages, it is considered failed and a) its transient topic
-  /// entries are removed and b) its entry in the subscriber map is erased.
+  /// entries are removed and b) its entry in the subscriber map is erased. The
+  /// subscriber ID is used to identify peers for failure detection purposes. Subscriber
+  /// state is evicted from the failure detector when the subscriber is unregistered,
+  /// so old subscribers do not occupy memory and the failure detection state does not
+  /// carry over to any new registrations of the previous subscriber.
   boost::scoped_ptr<MissedHeartbeatFailureDetector> failure_detector_;
 
   /// Metric that track the registered, non-failed subscribers.

http://git-wip-us.apache.org/repos/asf/impala/blob/890bd4c7/tests/statestore/test_statestore.py
----------------------------------------------------------------------
diff --git a/tests/statestore/test_statestore.py b/tests/statestore/test_statestore.py
index ba51c8d..737ff9c 100644
--- a/tests/statestore/test_statestore.py
+++ b/tests/statestore/test_statestore.py
@@ -17,6 +17,7 @@
 
 from collections import defaultdict
 import json
+import logging
 import socket
 import threading
 import traceback
@@ -39,6 +40,8 @@ from Status.ttypes import TStatus
 
 from tests.common.environ import specific_build_type_timeout
 
+LOG = logging.getLogger('test_statestore')
+
 # Tests for the statestore. The StatestoreSubscriber class is a skeleton implementation of
 # a Python-based statestore subscriber with additional hooks to allow testing. Each
 # StatestoreSubscriber runs its own server so that the statestore may contact it.
@@ -174,6 +177,7 @@ class StatestoreSubscriber(object):
     # Variables to notify for updates on each topic.
     self.update_event = threading.Condition()
     self.heartbeat_cb, self.update_cb = heartbeat_cb, update_cb
+    self.subscriber_id = "python-test-client-%s" % uuid.uuid1()
     self.exception = None
 
   def Heartbeat(self, args):
@@ -250,7 +254,6 @@ class StatestoreSubscriber(object):
 
   def register(self, topics=None):
     """Call the Register() RPC"""
-    self.subscriber_id = "python-test-client-%s" % uuid.uuid1()
     if topics is None: topics = []
     request = Subscriber.TRegisterSubscriberRequest(
       topic_registrations=topics,
@@ -515,3 +518,26 @@ class TestStatestore():
           .wait_for_update(persistent_topic_name, 1)
           .wait_for_update(transient_topic_name, 1)
     )
+
+  def test_heartbeat_failure_reset(self):
+    """Regression test for IMPALA-6785: the heartbeat failure count for the subscriber ID
+    should be reset when it resubscribes, not after the first successful heartbeat. Delay
+    the heartbeat to force the topic update to finish first."""
+
+    sub = StatestoreSubscriber(heartbeat_cb=lambda sub, args: time.sleep(0.5))
+    topic_name = "test_heartbeat_failure_reset"
+    reg = TTopicRegistration(topic_name=topic_name, is_transient=True)
+    sub.start()
+    sub.register(topics=[reg])
+    LOG.info("Registered with id {0}".format(sub.subscriber_id))
+    sub.wait_for_heartbeat(1)
+    sub.kill()
+    LOG.info("Killed, waiting for statestore to detect failure via heartbeats")
+    sub.wait_for_failure()
+    # IMPALA-6785 caused only one topic update to be send. Wait for multiple updates to
+    # be received to confirm that the subsequent updates are being scheduled repeatedly.
+    target_updates = sub.update_counts[topic_name] + 5
+    sub.start()
+    sub.register(topics=[reg])
+    LOG.info("Re-registered with id {0}, waiting for update".format(sub.subscriber_id))
+    sub.wait_for_update(topic_name, target_updates)


[3/5] impala git commit: IMPALA-4168: [DOCS] Adds Oracle-style hint placement for INSERT/UPSERT

Posted by ta...@apache.org.
IMPALA-4168: [DOCS] Adds Oracle-style hint placement for INSERT/UPSERT

Change-Id: I43e0a782087c2e67f2e012424fb9261be445efc9
Reviewed-on: http://gerrit.cloudera.org:8080/9030
Reviewed-by: Alex Rodoni <ar...@cloudera.com>
Reviewed-by: Kim Jin Chul <ji...@gmail.com>
Reviewed-by: John Russell <jr...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: e279f0f250cb2616e8774662e6a08150851c7f67
Parents: f33431d
Author: Jinchul <ji...@gmail.com>
Authored: Tue Jan 16 21:13:43 2018 +0900
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Apr 4 00:55:57 2018 +0000

----------------------------------------------------------------------
 docs/topics/impala_hints.xml  | 42 ++++++++++++++++++++++++++++++++++++++
 docs/topics/impala_insert.xml |  9 ++++----
 docs/topics/impala_upsert.xml |  2 +-
 3 files changed, 48 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/e279f0f2/docs/topics/impala_hints.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_hints.xml b/docs/topics/impala_hints.xml
index d489048..6cafcfb 100644
--- a/docs/topics/impala_hints.xml
+++ b/docs/topics/impala_hints.xml
@@ -77,6 +77,12 @@ INSERT <varname>insert_clauses</varname>
   [{ /* +SHUFFLE */ | /* +NOSHUFFLE */ }]
   [<ph rev="IMPALA-2522 2.8.0">/* +CLUSTERED */</ph>]
   SELECT <varname>remainder_of_query</varname>;
+
+<ph rev="2.12.0 IMPALA-4168">
+UPSERT [{ /* +SHUFFLE */ | /* +NOSHUFFLE */ }]
+  [<ph rev="IMPALA-2522 2.8.0">/* +CLUSTERED */</ph>]
+  <varname>upsert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
 </codeblock>
 
     <p rev="2.0.0">
@@ -112,6 +118,26 @@ INSERT <varname>insert_clauses</varname>
   -- +SHUFFLE|NOSHUFFLE
   SELECT <varname>remainder_of_query</varname>;
 
+<ph rev="2.12.0 IMPALA-4168">
+INSERT /* +SHUFFLE|NOSHUFFLE */
+  <varname>insert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
+
+<ph rev="2.12.0 IMPALA-4168">
+INSERT -- +SHUFFLE|NOSHUFFLE
+  <varname>insert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
+
+<ph rev="2.12.0 IMPALA-4168">
+UPSERT /* +SHUFFLE|NOSHUFFLE */
+  <varname>upsert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
+
+<ph rev="2.12.0 IMPALA-4168">
+UPSERT -- +SHUFFLE|NOSHUFFLE
+  <varname>upsert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
+
 <ph rev="IMPALA-2924">SELECT <varname>select_list</varname> FROM
 <varname>table_ref</varname>
   /* +{SCHEDULE_CACHE_LOCAL | SCHEDULE_DISK_LOCAL | SCHEDULE_REMOTE}
@@ -125,6 +151,22 @@ INSERT <varname>insert_clauses</varname>
 INSERT <varname>insert_clauses</varname>
   /* +CLUSTERED */
   SELECT <varname>remainder_of_query</varname>;</ph>
+
+<ph rev="2.12.0 IMPALA-4168">INSERT -- +CLUSTERED
+  <varname>insert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;
+
+INSERT /* +CLUSTERED */
+  <varname>insert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;
+
+UPSERT -- +CLUSTERED
+  <varname>upsert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;
+
+UPSERT /* +CLUSTERED */
+  <varname>upsert_clauses</varname>
+  SELECT <varname>remainder_of_query</varname>;</ph>
 </codeblock>
 
     <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>

http://git-wip-us.apache.org/repos/asf/impala/blob/e279f0f2/docs/topics/impala_insert.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_insert.xml b/docs/topics/impala_insert.xml
index a83692d..3880a70 100644
--- a/docs/topics/impala_insert.xml
+++ b/docs/topics/impala_insert.xml
@@ -49,7 +49,7 @@ under the License.
     <p conref="../shared/impala_common.xml#common/syntax_blurb"/>
 
 <codeblock>[<varname>with_clause</varname>]
-INSERT { INTO | OVERWRITE } [TABLE] <varname>table_name</varname>
+  INSERT <ph rev="2.12.0 IMPALA-4168">[<varname>hint_clause</varname>]</ph> { INTO | OVERWRITE } [TABLE] <varname>table_name</varname>
   [(<varname>column_list</varname>)]
   [ PARTITION (<varname>partition_clause</varname>)]
 {
@@ -204,9 +204,10 @@ hint_with_brackets ::= [SHUFFLE] | [NOSHUFFLE]
       </li>
 
       <li rev="1.2.2">
-        An optional hint clause immediately before the <codeph>SELECT</codeph> keyword, to fine-tune the behavior
-        when doing an <codeph>INSERT ... SELECT</codeph> operation into partitioned Parquet tables. The hint
-        keywords are <codeph>[SHUFFLE]</codeph> and <codeph>[NOSHUFFLE]</codeph>, including the square brackets.
+        An optional hint clause immediately either before the <codeph>SELECT</codeph> keyword or after the
+        <codeph>INSERT</codeph> keyword, to fine-tune the behavior when doing an <codeph>INSERT ... SELECT</codeph>
+        operation into partitioned Parquet tables. The hint clause cannot be specified in multiple places.
+        The hint keywords are <codeph>[SHUFFLE]</codeph> and <codeph>[NOSHUFFLE]</codeph>, including the square brackets.
         Inserting into partitioned Parquet tables can be a resource-intensive operation because it potentially
         involves many files being written to HDFS simultaneously, and separate
         <ph rev="parquet_block_size">large</ph> memory buffers being allocated to buffer the data for each

http://git-wip-us.apache.org/repos/asf/impala/blob/e279f0f2/docs/topics/impala_upsert.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_upsert.xml b/docs/topics/impala_upsert.xml
index 5830675..a95b0f8 100644
--- a/docs/topics/impala_upsert.xml
+++ b/docs/topics/impala_upsert.xml
@@ -68,7 +68,7 @@ under the License.
     <p conref="../shared/impala_common.xml#common/syntax_blurb"/>
 
 <codeblock>
-UPSERT INTO [TABLE] [<varname>db_name</varname>.]<varname>table_name</varname>
+UPSERT <ph rev="2.12.0 IMPALA-4168">[<varname>hint_clause</varname>]</ph> INTO [TABLE] [<varname>db_name</varname>.]<varname>table_name</varname>
   [(<varname>column_list</varname>)]
 {
     [<varname>hint_clause</varname>] <varname>select_statement</varname>


[2/5] impala git commit: IMPALA-6688: [DOCS] compact_catalog_topic setting is enabled by default

Posted by ta...@apache.org.
IMPALA-6688: [DOCS] compact_catalog_topic setting is enabled by default

- Marked IMPALA-5500 as fixed in Impala 2.10 in the impala_known_issues.
xml
- In the same section, removed the mention of default setting being false.
- Added a section in impala_upgrading.xml for default setting Changes.
- Added new keydefs for 2.12 and 3.0 version strings.

Change-Id: Id6e9eef7e20fcc520865f3d6d065ed508b2917bb
Reviewed-on: http://gerrit.cloudera.org:8080/9922
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: f33431dc4c5f53fb46a6392d61c858f7818b068c
Parents: 8173e9a
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Tue Apr 3 15:39:34 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Apr 4 00:55:23 2018 +0000

----------------------------------------------------------------------
 docs/impala_keydefs.ditamap         |  6 ++++++
 docs/topics/impala_known_issues.xml | 22 +++++++++++-----------
 docs/topics/impala_upgrading.xml    | 25 +++++++++++++++++++++++++
 3 files changed, 42 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/f33431dc/docs/impala_keydefs.ditamap
----------------------------------------------------------------------
diff --git a/docs/impala_keydefs.ditamap b/docs/impala_keydefs.ditamap
index 35dd0b8..be64501 100644
--- a/docs/impala_keydefs.ditamap
+++ b/docs/impala_keydefs.ditamap
@@ -10521,6 +10521,8 @@ under the License.
   <keydef href="https://issues.apache.org/jira/browse/IMPALA-9999" scope="external" format="html" keys="IMPALA-9999"/>
 
 <!-- Short form of mapping from Impala release to vendor-specific releases, for use in headings. -->
+  <keydef keys="impala30"><topicmeta><keywords><keyword>Impala 3.0</keyword></keywords></topicmeta></keydef>
+  <keydef keys="impala212"><topicmeta><keywords><keyword>Impala 2.12</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala211"><topicmeta><keywords><keyword>Impala 2.11</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala210"><topicmeta><keywords><keyword>Impala 2.10</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala29"><topicmeta><keywords><keyword>Impala 2.9</keyword></keywords></topicmeta></keydef>
@@ -10538,6 +10540,8 @@ under the License.
 
 <!-- 3-part forms of version numbers, for use in release notes. -->
 <!-- Using spaced-out form to avoid conflict with variable for 2.1.10 -->
+  <keydef keys="impala3_00_0"><topicmeta><keywords><keyword>Impala 3.0.0</keyword></keywords></topicmeta></keydef>
+  <keydef keys="impala2_12_0"><topicmeta><keywords><keyword>Impala 2.12.0</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala2_11_0"><topicmeta><keywords><keyword>Impala 2.11.0</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala2100"><topicmeta><keywords><keyword>Impala 2.10.0</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala290"><topicmeta><keywords><keyword>Impala 2.9.0</keyword></keywords></topicmeta></keydef>
@@ -10578,6 +10582,8 @@ under the License.
   <keydef keys="impala130"><topicmeta><keywords><keyword>Impala 1.3.0</keyword></keywords></topicmeta></keydef>
 
 <!-- Long form of mapping from Impala release to vendor-specific releases, for use in running text. -->
+  <keydef keys="impala30_full"><topicmeta><keywords><keyword>Impala 3.0</keyword></keywords></topicmeta></keydef>
+  <keydef keys="impala212_full"><topicmeta><keywords><keyword>Impala 2.12</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala211_full"><topicmeta><keywords><keyword>Impala 2.11</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala210_full"><topicmeta><keywords><keyword>Impala 2.10</keyword></keywords></topicmeta></keydef>
   <keydef keys="impala29_full"><topicmeta><keywords><keyword>Impala 2.9</keyword></keywords></topicmeta></keydef>

http://git-wip-us.apache.org/repos/asf/impala/blob/f33431dc/docs/topics/impala_known_issues.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_known_issues.xml b/docs/topics/impala_known_issues.xml
index 39fed3b..a8a8451 100644
--- a/docs/topics/impala_known_issues.xml
+++ b/docs/topics/impala_known_issues.xml
@@ -736,12 +736,10 @@ impala hard nproc 262144
         <p><b>Bug:</b> <xref keyref="IMPALA-5500">IMPALA-5500</xref></p>
         <p><b>Severity:</b> Medium</p>
         <p>
-          <b>Workaround:</b> Consider leaving the <codeph>compact_catalog_topic</codeph>
-          configuration setting at its default value of <codeph>false</codeph> until
-          this issue is resolved.
-        </p>
-        <p><b>Resolution:</b> A fix is in the pipeline. Check the status of
-        <xref keyref="IMPALA-5500">IMPALA-5500</xref> for the release where the fix is available.</p>
+          <b>Workaround:</b> Consider setting the
+            <codeph>compact_catalog_topic</codeph> configuration setting to
+            <codeph>false</codeph> until this issue is resolved. </p>
+        <p><b>Resolution:</b> Fixed in <keyword keyref="impala210"/>.</p>
       </conbody>
     </concept>
 
@@ -752,7 +750,7 @@ impala hard nproc 262144
         <p><b>Bug:</b> <xref keyref="IMPALA-2294">IMPALA-2294</xref></p>
         <p><b>Severity:</b> High</p>
         <p><b>Workaround:</b></p>
-        <p conref="../shared/impala_common.xml#common/vm_overcommit_memory_start" conrefend="vm_overcommit_memory_end"/>
+        <p conref="../shared/impala_common.xml#common/vm_overcommit_memory_start" conrefend="../shared/impala_common.xml#common/vm_overcommit_memory_end"/>
       </conbody>
     </concept>
 
@@ -1672,13 +1670,15 @@ select
         </p>
 
         <note type="warning">
-          Only use this for external tables, or Impala will remove the data files. In case of an internal table, set it to external first:
+          <p>Only use this for external tables, or Impala will remove the data
+            files. In case of an internal table, set it to external first:
 <codeblock>
 ALTER TABLE table_name SET TBLPROPERTIES('EXTERNAL'='TRUE');
 </codeblock>
-          (The part in parentheses is case sensitive.) Make sure to pick the right choice between internal and external when recreating the
-          table. See <xref href="impala_tables.xml#tables"/> for the differences between internal and external tables.
-        </note>
+          (The part in parentheses is case sensitive.) Make sure to pick the
+          right choice between internal and external when recreating the table.
+          See <xref href="impala_tables.xml#tables"/> for the differences
+          between internal and external tables. </p></note>
 
         <p>
           <b>Severity:</b> High

http://git-wip-us.apache.org/repos/asf/impala/blob/f33431dc/docs/topics/impala_upgrading.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_upgrading.xml b/docs/topics/impala_upgrading.xml
index 37ef2b2..75ca7dd 100644
--- a/docs/topics/impala_upgrading.xml
+++ b/docs/topics/impala_upgrading.xml
@@ -136,4 +136,29 @@ $ ps ax | grep [i]mpalad
       </note>
     </conbody>
   </concept>
+  <concept id="concept_a2p_szq_jdb">
+    <title>Impala Upgrade Considerations</title>
+    <concept id="concept_mkn_ygr_jdb">
+      <title>Default Setting Changes</title>
+      <conbody>
+        <simpletable frame="all" id="simpletable_x55_ghr_jdb">
+          <sthead>
+            <stentry>Release Changed</stentry>
+            <stentry>Setting</stentry>
+            <stentry>Default Value</stentry>
+          </sthead>
+          <strow>
+            <stentry><keyword keyref="impala212_full"/></stentry>
+            <stentry><codeph>compact_catalog_topic</codeph></stentry>
+            <stentry><codeph>true</codeph></stentry>
+          </strow>
+          <strow>
+            <stentry><keyword keyref="impala212_full"/></stentry>
+            <stentry><codeph>max_cached_file_handle</codeph></stentry>
+            <stentry><codeph>20000</codeph></stentry>
+          </strow>
+        </simpletable>
+      </conbody>
+    </concept>
+  </concept>
 </concept>


[4/5] impala git commit: IMPALA-4430: Update build scripts to die hard when IMPALA_HOME has spaces

Posted by ta...@apache.org.
IMPALA-4430: Update build scripts to die hard when IMPALA_HOME has spaces

Change-Id: I08b3d2b3f3e14c568d1672ee86ff2c52e8017b81
Reviewed-on: http://gerrit.cloudera.org:8080/9385
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 4c1f0ac6c8938bfcfafb3f1e70df06c960a5db56
Parents: e279f0f
Author: njanarthanan <nj...@cloudera.com>
Authored: Wed Feb 21 13:39:30 2018 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Apr 4 01:48:24 2018 +0000

----------------------------------------------------------------------
 bin/impala-config.sh | 17 ++++++++++++-----
 buildall.sh          | 10 ++++++++--
 2 files changed, 20 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4c1f0ac6/bin/impala-config.sh
----------------------------------------------------------------------
diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index e527e02..02ec950 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -45,14 +45,21 @@ if [[ ! -e "$JAVA" ]]; then
   return 1
 fi
 
-if [ -z "$IMPALA_HOME" ]; then
-  if [[ ! -z "$ZSH_NAME" ]]; then
-    export IMPALA_HOME=$(dirname "$(cd $(dirname ${(%):-%x}) >/dev/null && pwd)")
-  else
-    export IMPALA_HOME=$(dirname "$(cd $(dirname "${BASH_SOURCE[0]}") >/dev/null && pwd)")
+if ! [[ "'$IMPALA_HOME'" =~ [[:blank:]] ]]; then
+  if [ -z "$IMPALA_HOME" ]; then
+    if [[ ! -z "$ZSH_NAME" ]]; then
+      export IMPALA_HOME=$(dirname "$(cd $(dirname ${(%):-%x}) >/dev/null && pwd)")
+    else
+      export IMPALA_HOME=$(dirname "$(cd $(dirname "${BASH_SOURCE[0]}") >/dev/null && pwd)")
+    fi
   fi
 fi
 
+if [[ "'$IMPALA_HOME'" =~ [[:blank:]] ]]; then
+  echo "IMPALA_HOME cannot have spaces in the path"
+  exit 1
+fi
+
 export IMPALA_TOOLCHAIN=${IMPALA_TOOLCHAIN-"$IMPALA_HOME/toolchain"}
 if [ -z "$IMPALA_TOOLCHAIN" ]; then
   echo "IMPALA_TOOLCHAIN must be specified. Please set it to a valid directory or"\

http://git-wip-us.apache.org/repos/asf/impala/blob/4c1f0ac6/buildall.sh
----------------------------------------------------------------------
diff --git a/buildall.sh b/buildall.sh
index d14c2ea..9d8b15e 100755
--- a/buildall.sh
+++ b/buildall.sh
@@ -18,13 +18,19 @@
 # under the License.
 
 set -euo pipefail
-trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
 
 # run buildall.sh -help to see options
-
 ROOT=`dirname "$0"`
 ROOT=`cd "$ROOT" >/dev/null; pwd`
 
+if [[ "'$ROOT'" =~ [[:blank:]] ]]
+then
+   echo "IMPALA_HOME cannot have spaces in the path"
+   exit 1
+fi
+
+trap 'echo Error in $0 at line $LINENO: $(cd "'$PWD'" && awk "NR == $LINENO" $0)' ERR
+
 # Grab this *before* we source impala-config.sh to see if the caller has
 # kerberized environment variables already or not.
 NEEDS_RE_SOURCE_NOTE=1