You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jr...@apache.org on 2017/01/18 19:46:21 UTC

[1/6] incubator-impala git commit: IMPALA-4765: Avoid using several loading threads on one table.

Repository: incubator-impala
Updated Branches:
  refs/heads/master 2665866c1 -> 8377b9949


IMPALA-4765: Avoid using several loading threads on one table.

When there are multiple concurrent requests to the catalogd to
prioritize loading the same table, then several catalog loading
threads may end up waiting for that single table to be loaded,
effectively reducing the number of catalog loading threads. In
extreme examples, this might degrade to serial loading of tables.

This patch augments the existing data structures and code to
prevent using several loading threads for the same table.
Some of the existing data structures and code could be
consolidated/simplified but this patch does not try to address
that issue to minimize the risk of this change.

Testing: I could easily reproduce the bug locally with the steps
described in the JIRA. After this patch, I could not observe threads
being wasted anymore.

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


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

Branch: refs/heads/master
Commit: fa4a054cde012fc0cfc74b79cbdb7008491226bb
Parents: 2665866
Author: Alex Behm <al...@cloudera.com>
Authored: Thu Jan 12 17:51:51 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Sun Jan 15 08:38:13 2017 +0000

----------------------------------------------------------------------
 .../apache/impala/catalog/TableLoadingMgr.java  | 55 ++++++++++++--------
 1 file changed, 34 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/fa4a054c/fe/src/main/java/org/apache/impala/catalog/TableLoadingMgr.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/TableLoadingMgr.java b/fe/src/main/java/org/apache/impala/catalog/TableLoadingMgr.java
index 17f962d..35cb902 100644
--- a/fe/src/main/java/org/apache/impala/catalog/TableLoadingMgr.java
+++ b/fe/src/main/java/org/apache/impala/catalog/TableLoadingMgr.java
@@ -17,11 +17,8 @@
 
 package org.apache.impala.catalog;
 
-import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
@@ -30,11 +27,12 @@ import java.util.concurrent.Future;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.log4j.Logger;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.impala.thrift.TTableName;
 import org.apache.impala.util.HdfsCachingUtil;
+import org.apache.log4j.Logger;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 
@@ -102,14 +100,18 @@ public class TableLoadingMgr {
   private final LinkedBlockingDeque<TTableName> tableLoadingDeque_ =
       new LinkedBlockingDeque<TTableName>();
 
-  // A thread safe HashSet of table names that are in the tableLoadingDeque_. Used to
-  // efficiently check for existence of items in the deque.
-  // Updates may lead/lag updates to the tableLoadingDeque_ - they are added to this set
-  // immediately before being added to the deque and removed immediately after removing
-  // from the deque. The fact the updates are not synchronized shouldn't impact
-  // functionality since this set is only used for efficient lookups.
-  private final Set<TTableName> tableLoadingSet_ =
-      Collections.synchronizedSet(new HashSet<TTableName>());
+  // Maps from table name to a boolean indicating whether that table is currently
+  // being loaded by a table loading thread. Used to prevent adding superfluous
+  // entries to the deque, and to ensure that only a single table loading thread
+  // is consumed per table.
+  // Entries are added to this map immediately before being added to the deque and
+  // removed after a load has completed.
+  // Once the load of a table begins, its associated boolean is set to true, and
+  // attempts to load the same table by a different thread become no-ops.
+  // This map is different from loadingTables_ because the latter tracks all in-flight
+  // loads - even those being processed by threads other than table loading threads.
+  private final ConcurrentHashMap<TTableName, AtomicBoolean> tableLoadingBarrier_ =
+      new ConcurrentHashMap<TTableName, AtomicBoolean>();
 
   // Map of table name to a FutureTask associated with the table load. Used to
   // prevent duplicate loads of the same table.
@@ -175,7 +177,10 @@ public class TableLoadingMgr {
    * Prioritizes the loading of the given table.
    */
   public void prioritizeLoad(TTableName tblName) {
-    tableLoadingSet_.add(tblName);
+    AtomicBoolean isLoading =
+        tableLoadingBarrier_.putIfAbsent(tblName, new AtomicBoolean(false));
+    // Only queue the table if a load is not already in progress.
+    if (isLoading != null && isLoading.get()) return;
     tableLoadingDeque_.offerFirst(tblName);
   }
 
@@ -183,9 +188,9 @@ public class TableLoadingMgr {
    * Submits a single table for background (low priority) loading.
    */
   public void backgroundLoad(TTableName tblName) {
-    // Only queue for background loading if the table doesn't already exist
-    // in the table loading set.
-    if (tableLoadingSet_.add(tblName)) {
+    // Only queue for background loading if the table isn't already queued or
+    // currently being loaded.
+    if (tableLoadingBarrier_.putIfAbsent(tblName, new AtomicBoolean(false)) == null) {
       tableLoadingDeque_.offerLast(tblName);
     }
   }
@@ -277,10 +282,16 @@ public class TableLoadingMgr {
   private void loadNextTable() throws InterruptedException {
     // Always get the next table from the head of the deque.
     final TTableName tblName = tableLoadingDeque_.takeFirst();
-    tableLoadingSet_.remove(tblName);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Loading next table. Remaining items in queue: "
-          + tableLoadingDeque_.size());
+    LOG.info("Loading next table from queue: " +
+        tblName.db_name + "." + tblName.table_name);
+    LOG.info(String.format("Remaining items in queue: %s. Loads in progress: %s",
+        tableLoadingDeque_.size(), loadingTables_.size()));
+
+    AtomicBoolean isLoading = tableLoadingBarrier_.get(tblName);
+    if (isLoading == null || !isLoading.compareAndSet(false, true)) {
+      // Another thread has already completed the load or the load is still in progress.
+      // Return so this thread can work on another table in the queue.
+      return;
     }
     try {
       // TODO: Instead of calling "getOrLoad" here we could call "loadAsync". We would
@@ -288,6 +299,8 @@ public class TableLoadingMgr {
       catalog_.getOrLoadTable(tblName.getDb_name(), tblName.getTable_name());
     } catch (CatalogException e) {
       // Ignore.
+    } finally {
+      tableLoadingBarrier_.remove(tblName);
     }
   }
 


[6/6] incubator-impala git commit: Global search/replace: audience="Cloudera" -> audience="hidden".

Posted by jr...@apache.org.
Global search/replace: audience="Cloudera" -> audience="hidden".

For this change to land in master, the audience="hidden" code review
needs to be completed first. Otherwise, the doc build would still work
but the audience="hidden" content would be visible rather than hidden as
desired.

Some work happening in parallel might introduce additional instances of
audience="Cloudera". I suggest addressing those in a followup CR so this
global change can land quickly.

Since the changes apply across so many different files, but are so
narrow in scope, I suggest that the way to validate (check that no
extraneous changes were introduced accidentally) is to diff just the
changed lines:

git diff -U0 HEAD^ HEAD

In patch set 2, I updated other topics marked audience="Cloudera"
by CRs that were pushed in the meantime.

Change-Id: Ic93d89da77e1f51bbf548a522d98d0c4e2fb31c8
Reviewed-on: http://gerrit.cloudera.org:8080/5613
Reviewed-by: John Russell <jr...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/master
Commit: 8377b9949cef975881e7f64eadb9400fcbf157b3
Parents: 3b7a179
Author: John Russell <jr...@cloudera.com>
Authored: Fri Dec 9 13:01:16 2016 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Jan 18 19:31:57 2017 +0000

----------------------------------------------------------------------
 docs/impala.ditamap                             |  20 +--
 docs/impala_keydefs.ditamap                     |   2 +-
 docs/topics/impala_abort_on_error.xml           |   2 +-
 docs/topics/impala_admission.xml                |  26 ++--
 docs/topics/impala_aggregate_functions.xml      |   2 +-
 docs/topics/impala_alter_function.xml           |   2 +-
 docs/topics/impala_alter_table.xml              |   2 +-
 docs/topics/impala_alter_view.xml               |   2 +-
 docs/topics/impala_analytic_functions.xml       |   6 +-
 docs/topics/impala_appx_count_distinct.xml      |   2 +-
 docs/topics/impala_appx_median.xml              |   2 +-
 docs/topics/impala_authorization.xml            |   6 +-
 docs/topics/impala_avg.xml                      |   2 +-
 docs/topics/impala_avro.xml                     |   4 +-
 docs/topics/impala_batch_size.xml               |   2 +-
 docs/topics/impala_bit_functions.xml            |  22 ++--
 docs/topics/impala_breakpad.xml                 |   2 +-
 docs/topics/impala_char.xml                     |   2 +-
 docs/topics/impala_cluster_sizing.xml           |   2 +-
 docs/topics/impala_cm_installation.xml          |   2 +-
 docs/topics/impala_comments.xml                 |   2 +-
 docs/topics/impala_complex_types.xml            |  14 +--
 docs/topics/impala_compression_codec.xml        |   2 +-
 docs/topics/impala_compute_stats.xml            |   2 +-
 docs/topics/impala_concepts.xml                 |  18 +--
 docs/topics/impala_conditional_functions.xml    |  34 ++---
 docs/topics/impala_config_options.xml           |  36 +++---
 docs/topics/impala_config_performance.xml       |   6 +-
 docs/topics/impala_conversion_functions.xml     |  52 ++++----
 docs/topics/impala_count.xml                    |   2 +-
 docs/topics/impala_create_data_source.xml       |   4 +-
 docs/topics/impala_create_database.xml          |   2 +-
 docs/topics/impala_create_function.xml          |   4 +-
 docs/topics/impala_create_role.xml              |   2 +-
 docs/topics/impala_create_table.xml             |   2 +-
 docs/topics/impala_create_view.xml              |   2 +-
 docs/topics/impala_datatypes.xml                |   2 +-
 docs/topics/impala_date.xml                     |   4 +-
 docs/topics/impala_datetime_functions.xml       | 102 +++++++--------
 docs/topics/impala_debug_action.xml             |   2 +-
 docs/topics/impala_decimal.xml                  |   2 +-
 docs/topics/impala_delete.xml                   |   2 +-
 docs/topics/impala_describe.xml                 |   2 +-
 docs/topics/impala_development.xml              |   6 +-
 docs/topics/impala_disable_cached_reads.xml     |   2 +-
 docs/topics/impala_disable_codegen.xml          |   2 +-
 docs/topics/impala_disable_outermost_topn.xml   |   2 +-
 .../impala_disable_row_runtime_filtering.xml    |   2 +-
 ...impala_disable_streaming_preaggregations.xml |   2 +-
 docs/topics/impala_disable_unsafe_spills.xml    |   2 +-
 docs/topics/impala_distinct.xml                 |   2 +-
 docs/topics/impala_drop_data_source.xml         |   4 +-
 docs/topics/impala_drop_database.xml            |   2 +-
 docs/topics/impala_drop_function.xml            |   2 +-
 docs/topics/impala_drop_role.xml                |   2 +-
 docs/topics/impala_drop_stats.xml               |   2 +-
 docs/topics/impala_drop_table.xml               |   2 +-
 docs/topics/impala_drop_view.xml                |   2 +-
 .../impala_exec_single_node_rows_threshold.xml  |   2 +-
 docs/topics/impala_explain.xml                  |   2 +-
 docs/topics/impala_explain_level.xml            |   2 +-
 docs/topics/impala_faq.xml                      |   8 +-
 docs/topics/impala_file_formats.xml             |   4 +-
 docs/topics/impala_fixed_issues.xml             |  38 +++---
 docs/topics/impala_grant.xml                    |   2 +-
 docs/topics/impala_group_concat.xml             |   2 +-
 docs/topics/impala_hbase.xml                    |  10 +-
 docs/topics/impala_hbase_cache_blocks.xml       |   2 +-
 docs/topics/impala_hbase_caching.xml            |   2 +-
 docs/topics/impala_hints.xml                    |   2 +-
 docs/topics/impala_impala_shell.xml             |   2 +-
 docs/topics/impala_incompatible_changes.xml     |   6 +-
 docs/topics/impala_insert.xml                   |   2 +-
 docs/topics/impala_install.xml                  |  18 +--
 docs/topics/impala_intro.xml                    |   2 +-
 docs/topics/impala_invalidate_metadata.xml      |   2 +-
 docs/topics/impala_isilon.xml                   |   6 +-
 docs/topics/impala_jdbc.xml                     |   2 +-
 docs/topics/impala_joins.xml                    |   2 +-
 docs/topics/impala_known_issues.xml             |  10 +-
 docs/topics/impala_kudu.xml                     |   2 +-
 docs/topics/impala_langref_unsupported.xml      |   4 +-
 docs/topics/impala_ldap.xml                     |   4 +-
 docs/topics/impala_lineage.xml                  |   4 +-
 docs/topics/impala_literals.xml                 |   8 +-
 docs/topics/impala_live_progress.xml            |   2 +-
 docs/topics/impala_live_summary.xml             |   4 +-
 docs/topics/impala_load_data.xml                |   2 +-
 docs/topics/impala_logging.xml                  |   8 +-
 docs/topics/impala_math_functions.xml           | 126 +++++++++----------
 docs/topics/impala_max.xml                      |   2 +-
 docs/topics/impala_max_block_mgr_memory.xml     |   2 +-
 docs/topics/impala_max_errors.xml               |   2 +-
 docs/topics/impala_max_num_runtime_filters.xml  |   2 +-
 docs/topics/impala_max_scan_range_length.xml    |   2 +-
 docs/topics/impala_mem_limit.xml                |   2 +-
 docs/topics/impala_min.xml                      |   2 +-
 docs/topics/impala_misc_functions.xml           |  16 +--
 docs/topics/impala_ndv.xml                      |   2 +-
 docs/topics/impala_new_features.xml             |  30 ++---
 docs/topics/impala_num_nodes.xml                |   2 +-
 docs/topics/impala_num_scanner_threads.xml      |   2 +-
 docs/topics/impala_odbc.xml                     |   2 +-
 docs/topics/impala_operators.xml                |  44 +++----
 .../impala_optimize_partition_key_scans.xml     |   2 +-
 docs/topics/impala_parquet.xml                  |   6 +-
 .../impala_parquet_annotate_strings_utf8.xml    |   2 +-
 .../topics/impala_parquet_compression_codec.xml |   2 +-
 ...mpala_parquet_fallback_schema_resolution.xml |   2 +-
 docs/topics/impala_parquet_file_size.xml        |   2 +-
 docs/topics/impala_partitioning.xml             |   6 +-
 docs/topics/impala_perf_hdfs_caching.xml        |   4 +-
 docs/topics/impala_perf_stats.xml               |   4 +-
 docs/topics/impala_performance.xml              |  24 ++--
 docs/topics/impala_planning.xml                 |   2 +-
 docs/topics/impala_porting.xml                  |   2 +-
 docs/topics/impala_ports.xml                    |   2 +-
 docs/topics/impala_prefetch_mode.xml            |   2 +-
 docs/topics/impala_prereqs.xml                  |  56 ++++-----
 docs/topics/impala_processes.xml                |   8 +-
 docs/topics/impala_proxy.xml                    |  16 +--
 docs/topics/impala_query_timeout_s.xml          |   2 +-
 docs/topics/impala_rcfile.xml                   |   6 +-
 docs/topics/impala_refresh.xml                  |   2 +-
 docs/topics/impala_replica_preference.xml       |   2 +-
 docs/topics/impala_request_pool.xml             |   2 +-
 .../impala_reservation_request_timeout.xml      |   2 +-
 docs/topics/impala_reserved_words.xml           |   2 +-
 docs/topics/impala_resource_management.xml      |   8 +-
 docs/topics/impala_revoke.xml                   |   2 +-
 docs/topics/impala_rm_initial_mem.xml           |   2 +-
 .../topics/impala_runtime_bloom_filter_size.xml |   2 +-
 docs/topics/impala_runtime_filter_max_size.xml  |   2 +-
 docs/topics/impala_runtime_filter_min_size.xml  |   2 +-
 docs/topics/impala_runtime_filter_mode.xml      |   2 +-
 .../impala_runtime_filter_wait_time_ms.xml      |   2 +-
 docs/topics/impala_runtime_filtering.xml        |   2 +-
 docs/topics/impala_s3.xml                       |  12 +-
 docs/topics/impala_s3_skip_insert_staging.xml   |   2 +-
 docs/topics/impala_scalability.xml              |  10 +-
 .../impala_scan_node_codegen_threshold.xml      |   2 +-
 docs/topics/impala_schedule_random_replica.xml  |   2 +-
 docs/topics/impala_schema_objects.xml           |   2 +-
 docs/topics/impala_select.xml                   |   4 +-
 docs/topics/impala_seq_compression_mode.xml     |   2 +-
 docs/topics/impala_seqfile.xml                  |   6 +-
 docs/topics/impala_set.xml                      |   2 +-
 docs/topics/impala_shell_commands.xml           |   2 +-
 docs/topics/impala_show.xml                     |   6 +-
 docs/topics/impala_ssl.xml                      |   4 +-
 docs/topics/impala_stddev.xml                   |   6 +-
 docs/topics/impala_string_functions.xml         |  66 +++++-----
 docs/topics/impala_subqueries.xml               |   2 +-
 docs/topics/impala_sum.xml                      |   2 +-
 docs/topics/impala_support_start_over.xml       |   2 +-
 docs/topics/impala_sync_ddl.xml                 |   2 +-
 docs/topics/impala_tables.xml                   |   4 +-
 docs/topics/impala_trouble_bad_results.xml      |   6 +-
 docs/topics/impala_trouble_memory.xml           |   4 +-
 docs/topics/impala_trouble_query_fail.xml       |   2 +-
 docs/topics/impala_trouble_sql.xml              |   4 +-
 docs/topics/impala_trouble_startup.xml          |   2 +-
 docs/topics/impala_troubleshooting.xml          |   8 +-
 docs/topics/impala_truncate_table.xml           |   2 +-
 docs/topics/impala_tutorial.xml                 |  12 +-
 docs/topics/impala_txtfile.xml                  |  20 +--
 docs/topics/impala_udf.xml                      |   4 +-
 docs/topics/impala_update.xml                   |   2 +-
 docs/topics/impala_upgrading.xml                |   6 +-
 docs/topics/impala_upsert.xml                   |   2 +-
 docs/topics/impala_use.xml                      |   2 +-
 docs/topics/impala_v_cpu_cores.xml              |   2 +-
 docs/topics/impala_varchar.xml                  |   2 +-
 docs/topics/impala_variance.xml                 |  10 +-
 docs/topics/impala_webui.xml                    |  16 +--
 docs/topics/impala_window_functions.xml         |   2 +-
 176 files changed, 642 insertions(+), 642 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/impala.ditamap
----------------------------------------------------------------------
diff --git a/docs/impala.ditamap b/docs/impala.ditamap
index 6ffeb84..46b8c7f 100644
--- a/docs/impala.ditamap
+++ b/docs/impala.ditamap
@@ -51,7 +51,7 @@ under the License.
     <topicref href="topics/impala_schema_design.xml"/>
   </topicref>
   <topicref audience="standalone" href="topics/impala_install.xml#install">
-    <topicref href="topics/impala_cm_installation.xml#cm_installation" audience="Cloudera"/>
+    <topicref href="topics/impala_cm_installation.xml#cm_installation" audience="hidden"/>
     <topicref href="topics/impala_noncm_installation.xml#noncm_installation"/>
 <!-- Removed when Hue docs were removed from main library. Might be worth resurrecting someday as general Impala+Hue topic. -->
 <!--		<topicref href="impala_hue_xi20559.xml"/> -->
@@ -101,7 +101,7 @@ under the License.
       <topicref href="topics/impala_boolean.xml"/>
       <topicref href="topics/impala_char.xml"/>
       <topicref href="topics/impala_decimal.xml"/>
-      <topicref audience="Cloudera" href="topics/impala_date.xml"/>
+      <topicref audience="hidden" href="topics/impala_date.xml"/>
       <topicref href="topics/impala_double.xml"/>
       <topicref href="topics/impala_float.xml"/>
       <topicref href="topics/impala_int.xml"/>
@@ -120,7 +120,7 @@ under the License.
     <topicref href="topics/impala_schema_objects.xml">
       <topicref href="topics/impala_aliases.xml"/>
       <topicref href="topics/impala_databases.xml"/>
-      <topicref audience="Cloudera" href="topics/impala_data_sources.xml"/>
+      <topicref audience="hidden" href="topics/impala_data_sources.xml"/>
       <topicref href="topics/impala_functions_overview.xml"/>
       <topicref href="topics/impala_identifiers.xml"/>
       <topicref href="topics/impala_tables.xml"/>
@@ -132,7 +132,7 @@ under the License.
       <topicref href="topics/impala_alter_table.xml"/>
       <topicref href="topics/impala_alter_view.xml"/>
       <topicref href="topics/impala_compute_stats.xml"/>
-      <topicref audience="Cloudera" href="topics/impala_create_data_source.xml"/>
+      <topicref audience="hidden" href="topics/impala_create_data_source.xml"/>
       <topicref href="topics/impala_create_database.xml"/>
       <topicref href="topics/impala_create_function.xml"/>
       <topicref href="topics/impala_create_role.xml"/>
@@ -140,7 +140,7 @@ under the License.
       <topicref href="topics/impala_create_view.xml"/>
       <topicref href="topics/impala_delete.xml"/>
       <topicref href="topics/impala_describe.xml"/>
-      <topicref audience="Cloudera" href="topics/impala_drop_data_source.xml"/>
+      <topicref audience="hidden" href="topics/impala_drop_data_source.xml"/>
       <topicref href="topics/impala_drop_database.xml"/>
       <topicref href="topics/impala_drop_function.xml"/>
       <topicref href="topics/impala_drop_role.xml"/>
@@ -177,9 +177,9 @@ under the License.
           <topicref href="topics/impala_compression_codec.xml"/>
           <topicref href="topics/impala_debug_action.xml"/>
           <topicref href="topics/impala_default_order_by_limit.xml"/>
-          <topicref audience="Cloudera" href="topics/impala_disable_cached_reads.xml"/>
+          <topicref audience="hidden" href="topics/impala_disable_cached_reads.xml"/>
           <topicref href="topics/impala_disable_codegen.xml"/>
-          <topicref audience="Cloudera" href="topics/impala_disable_outermost_topn.xml"/>
+          <topicref audience="hidden" href="topics/impala_disable_outermost_topn.xml"/>
           <topicref rev="2.5.0" href="topics/impala_disable_row_runtime_filtering.xml"/>
           <topicref rev="2.5.0" href="topics/impala_disable_streaming_preaggregations.xml"/>
           <topicref href="topics/impala_disable_unsafe_spills.xml"/>
@@ -206,7 +206,7 @@ under the License.
           <topicref href="topics/impala_request_pool.xml"/>
           <topicref rev="2.5.0" href="topics/impala_schedule_random_replica.xml"/>
           <topicref rev="2.7.0" href="topics/impala_replica_preference.xml"/>
-          <topicref audience="Cloudera" href="topics/impala_rm_initial_mem.xml"/>
+          <topicref audience="hidden" href="topics/impala_rm_initial_mem.xml"/>
           <topicref href="topics/impala_reservation_request_timeout.xml"/>
           <topicref rev="2.5.0" href="topics/impala_runtime_bloom_filter_size.xml"/>
           <topicref rev="2.6.0" href="topics/impala_runtime_filter_max_size.xml"/>
@@ -272,7 +272,7 @@ under the License.
     <topicref href="topics/impala_perf_testing.xml"/>
     <topicref href="topics/impala_explain_plan.xml"/>
     <topicref href="topics/impala_perf_skew.xml"/>
-    <topicref audience="Cloudera" href="topics/impala_perf_ddl.xml"/>
+    <topicref audience="hidden" href="topics/impala_perf_ddl.xml"/>
   </topicref>
   <topicref href="topics/impala_scalability.xml"/>
   <topicref href="topics/impala_partitioning.xml"/>
@@ -305,7 +305,7 @@ under the License.
   </topicref>
   <!-- Release coordinator: don't remove this pre-release tag, it will
        probably remain in pre-release state for some time. -->
-  <topicref audience="Cloudera" href="topics/impala_errata.xml"/>
+  <topicref audience="hidden" href="topics/impala_errata.xml"/>
 
 <!-- Substitution variables and link destinations abstracted
      into a .h-like side file. -->

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/impala_keydefs.ditamap
----------------------------------------------------------------------
diff --git a/docs/impala_keydefs.ditamap b/docs/impala_keydefs.ditamap
index 2332f86..952430a 100644
--- a/docs/impala_keydefs.ditamap
+++ b/docs/impala_keydefs.ditamap
@@ -667,7 +667,7 @@ https://issues.cloudera.org/secure/IssueNavigator.jspa?reset=true&amp;jqlQuery=p
   <keydef href="topics/impala_cluster_sizing.xml" keys="cluster_sizing"/>
   <keydef href="topics/impala_schema_design.xml" keys="schema_design"/>
   <keydef href="topics/impala_install.xml" keys="install"/>
-  <keydef audience="Cloudera" href="topics/impala_cm_installation.xml" keys="cm_installation"/>
+  <keydef audience="hidden" href="topics/impala_cm_installation.xml" keys="cm_installation"/>
   <keydef href="topics/impala_noncm_installation.xml" keys="noncm_installation"/>
   <keydef href="topics/impala_config.xml" keys="config"/>
   <keydef href="topics/impala_config_performance.xml" keys="config_performance"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_abort_on_error.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_abort_on_error.xml b/docs/topics/impala_abort_on_error.xml
index 81eac8d..81b0583 100644
--- a/docs/topics/impala_abort_on_error.xml
+++ b/docs/topics/impala_abort_on_error.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">ABORT_ON_ERROR query option</indexterm>
+      <indexterm audience="hidden">ABORT_ON_ERROR query option</indexterm>
       When this option is enabled, Impala cancels a query immediately when any of the nodes encounters an error,
       rather than continuing and possibly returning incomplete results. This option is disabled by default, to help
       gather maximum diagnostic information when an error occurs, for example, whether the same problem occurred on

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_admission.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_admission.xml b/docs/topics/impala_admission.xml
index 0c55e69..ea47d89 100644
--- a/docs/topics/impala_admission.xml
+++ b/docs/topics/impala_admission.xml
@@ -429,7 +429,7 @@ under the License.
               <codeph>queue_wait_timeout_ms</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--queue_wait_timeout_ms</indexterm>
+              <indexterm audience="hidden">--queue_wait_timeout_ms</indexterm>
               <b>Purpose:</b> Maximum amount of time (in milliseconds) that a
               request waits to be admitted before timing out.
               <p>
@@ -445,7 +445,7 @@ under the License.
               <codeph>default_pool_max_requests</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--default_pool_max_requests</indexterm>
+              <indexterm audience="hidden">--default_pool_max_requests</indexterm>
               <b>Purpose:</b> Maximum number of concurrent outstanding requests
               allowed to run before incoming requests are queued. Because this
               limit applies cluster-wide, but each Impala node makes independent
@@ -468,7 +468,7 @@ under the License.
               <codeph>default_pool_max_queued</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--default_pool_max_queued</indexterm>
+              <indexterm audience="hidden">--default_pool_max_queued</indexterm>
               <b>Purpose:</b> Maximum number of requests allowed to be queued
               before rejecting requests. Because this limit applies
               cluster-wide, but each Impala node makes independent decisions to
@@ -492,7 +492,7 @@ under the License.
               <codeph>default_pool_mem_limit</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--default_pool_mem_limit</indexterm>
+              <indexterm audience="hidden">--default_pool_mem_limit</indexterm>
               <b>Purpose:</b> Maximum amount of memory (across the entire
               cluster) that all outstanding requests in this pool can use before
               new requests to this pool are queued. Specified in bytes,
@@ -523,7 +523,7 @@ under the License.
               <codeph>disable_admission_control</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--disable_admission_control</indexterm>
+              <indexterm audience="hidden">--disable_admission_control</indexterm>
               <b>Purpose:</b> Turns off the admission control feature entirely,
               regardless of other configuration option settings.
               <p>
@@ -539,7 +539,7 @@ under the License.
               <codeph>disable_pool_max_requests</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--disable_pool_max_requests</indexterm>
+              <indexterm audience="hidden">--disable_pool_max_requests</indexterm>
               <b>Purpose:</b> Disables all per-pool limits on the maximum number
               of running requests. <p>
                 <b>Type:</b> Boolean </p>
@@ -554,7 +554,7 @@ under the License.
               <codeph>disable_pool_mem_limits</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--disable_pool_mem_limits</indexterm>
+              <indexterm audience="hidden">--disable_pool_mem_limits</indexterm>
               <b>Purpose:</b> Disables all per-pool mem limits. <p>
                 <b>Type:</b> Boolean </p>
               <p>
@@ -568,7 +568,7 @@ under the License.
               <codeph>fair_scheduler_allocation_path</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--fair_scheduler_allocation_path</indexterm>
+              <indexterm audience="hidden">--fair_scheduler_allocation_path</indexterm>
               <b>Purpose:</b> Path to the fair scheduler allocation file
                 (<codeph>fair-scheduler.xml</codeph>). <p
                 conref="../shared/impala_common.xml#common/type_string" />
@@ -589,7 +589,7 @@ under the License.
               <codeph>llama_site_path</codeph>
             </dt>
             <dd>
-              <indexterm audience="Cloudera">--llama_site_path</indexterm>
+              <indexterm audience="hidden">--llama_site_path</indexterm>
               <b>Purpose:</b> Path to the configuration file used by admission control
                 (<codeph>llama-site.xml</codeph>). If set,
                 <codeph>fair_scheduler_allocation_path</codeph> must also be set.
@@ -606,7 +606,7 @@ under the License.
       </section>
     </conbody>
 
-    <concept id="admission_config_cm" audience="Cloudera">
+    <concept id="admission_config_cm" audience="hidden">
 
 <!-- TK: Maybe all this stuff overlaps with admin_impala_admission_control and can be delegated there. -->
 
@@ -625,7 +625,7 @@ under the License.
           <xref href="http://www.cloudera.com/documentation/enterprise/latest/topics/cm_mc_managing_resources.html" scope="external" format="html">Managing Resources with Cloudera Manager</xref>.
         </p>
 
-        <p audience="Cloudera"><!-- Hiding link because that subtopic is now hidden. -->
+        <p audience="hidden"><!-- Hiding link because that subtopic is now hidden. -->
           See <xref href="#admission_examples"/> for a sample setup for admission control under
           Cloudera Manager.
         </p>
@@ -694,7 +694,7 @@ impala.admission-control.pool-queue-timeout-ms.<varname>queue_name</varname></ph
           <keyword keyref="impala25_full"/> and higher.
         </p>
 
-        <p audience="Cloudera"><!-- Hiding link because that subtopic is now hidden. -->
+        <p audience="hidden"><!-- Hiding link because that subtopic is now hidden. -->
           See <xref href="#admission_examples/section_etq_qgb_rq"/> for sample configuration files
           for admission control using multiple resource pools, without Cloudera Manager.
         </p>
@@ -708,7 +708,7 @@ impala.admission-control.pool-queue-timeout-ms.<varname>queue_name</varname></ph
 
       <conbody>
 
-        <section id="section_fqn_qgb_rq" audience="Cloudera">
+        <section id="section_fqn_qgb_rq" audience="hidden">
 
           <title>Example Admission Control Configurations Using Cloudera Manager</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_aggregate_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_aggregate_functions.xml b/docs/topics/impala_aggregate_functions.xml
index a0299c1..1e01ad7 100644
--- a/docs/topics/impala_aggregate_functions.xml
+++ b/docs/topics/impala_aggregate_functions.xml
@@ -43,7 +43,7 @@ under the License.
     <p conref="../shared/impala_common.xml#common/aggr3"/>
 
     <p>
-      <indexterm audience="Cloudera">aggregate functions</indexterm>
+      <indexterm audience="hidden">aggregate functions</indexterm>
     </p>
 
     <p outputclass="toc"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_alter_function.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_alter_function.xml b/docs/topics/impala_alter_function.xml
index d9d1b10..243222f 100644
--- a/docs/topics/impala_alter_function.xml
+++ b/docs/topics/impala_alter_function.xml
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
-<concept audience="Cloudera" rev="1.x" id="alter_function">
+<concept audience="hidden" rev="1.x" id="alter_function">
 
   <title>ALTER FUNCTION Statement</title>
   <titlealts audience="PDF"><navtitle>ALTER FUNCTION</navtitle></titlealts>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_alter_table.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_alter_table.xml b/docs/topics/impala_alter_table.xml
index d763891..5065319 100644
--- a/docs/topics/impala_alter_table.xml
+++ b/docs/topics/impala_alter_table.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">ALTER TABLE statement</indexterm>
+      <indexterm audience="hidden">ALTER TABLE statement</indexterm>
       The <codeph>ALTER TABLE</codeph> statement changes the structure or properties of an existing Impala table.
     </p>
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_alter_view.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_alter_view.xml b/docs/topics/impala_alter_view.xml
index f48a308..e0d21d9 100644
--- a/docs/topics/impala_alter_view.xml
+++ b/docs/topics/impala_alter_view.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">ALTER VIEW statement</indexterm>
+      <indexterm audience="hidden">ALTER VIEW statement</indexterm>
       Changes the characteristics of a view. The syntax has two forms:
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_analytic_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_analytic_functions.xml b/docs/topics/impala_analytic_functions.xml
index 6b94418..bdaacca 100644
--- a/docs/topics/impala_analytic_functions.xml
+++ b/docs/topics/impala_analytic_functions.xml
@@ -44,9 +44,9 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">analytic functions</indexterm>
+      <indexterm audience="hidden">analytic functions</indexterm>
 
-      <indexterm audience="Cloudera">window functions</indexterm>
+      <indexterm audience="hidden">window functions</indexterm>
       Analytic functions (also known as window functions) are a special category of built-in functions. Like
       aggregate functions, they examine the contents of multiple input rows to compute each output value. However,
       rather than being limited to one result value per <codeph>GROUP BY</codeph> group, they operate on
@@ -1210,7 +1210,7 @@ from stock_ticker
 
   </concept>
 
-  <concept audience="Cloudera" rev="2.x.x" id="nth_value">
+  <concept audience="hidden" rev="2.x.x" id="nth_value">
 
     <title>NTH_VALUE Function</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_appx_count_distinct.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_appx_count_distinct.xml b/docs/topics/impala_appx_count_distinct.xml
index be03d96..8655968 100644
--- a/docs/topics/impala_appx_count_distinct.xml
+++ b/docs/topics/impala_appx_count_distinct.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">APPX_COUNT_DISTINCT query option</indexterm>
+      <indexterm audience="hidden">APPX_COUNT_DISTINCT query option</indexterm>
       Allows multiple <codeph>COUNT(DISTINCT)</codeph> operations within a single query, by internally rewriting
       each <codeph>COUNT(DISTINCT)</codeph> to use the <codeph>NDV()</codeph> function. The resulting count is
       approximate rather than precise.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_appx_median.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_appx_median.xml b/docs/topics/impala_appx_median.xml
index bb06e72..94a31ad 100644
--- a/docs/topics/impala_appx_median.xml
+++ b/docs/topics/impala_appx_median.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">appx_median() function</indexterm>
+      <indexterm audience="hidden">appx_median() function</indexterm>
       An aggregate function that returns a value that is approximately the median (midpoint) of values in the set
       of input values.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_authorization.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_authorization.xml b/docs/topics/impala_authorization.xml
index c0449b9..39f0e81 100644
--- a/docs/topics/impala_authorization.xml
+++ b/docs/topics/impala_authorization.xml
@@ -222,7 +222,7 @@ under the License.
       </p>
 
       <ul>
-        <li audience="Cloudera">
+        <li audience="hidden">
           <p>
             In an environment managed by Cloudera Manager, the server name is specified through
             <menucascade><uicontrol>Impala (Service-Wide)</uicontrol><uicontrol>Category</uicontrol><uicontrol>Advanced</uicontrol><uicontrol>Sentry Service</uicontrol></menucascade> and
@@ -698,7 +698,7 @@ ERROR: AuthorizationException: User 'cloudera' does not have privileges to acces
 
         </example>
 
-        <example audience="Cloudera" id="sec_ex_views" rev="2.3.0 collevelauth">
+        <example audience="hidden" id="sec_ex_views" rev="2.3.0 collevelauth">
 
           <title>Controlling Access at the Column Level through Views</title>
 
@@ -1586,7 +1586,7 @@ column-level security was by creating views that referenced particular sets of c
     </conbody>
   </concept>
 
-  <concept id="sentry_cm" audience="Cloudera">
+  <concept id="sentry_cm" audience="hidden">
 
     <title>Managing Sentry for Impala through Cloudera Manager</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_avg.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_avg.xml b/docs/topics/impala_avg.xml
index 2d402ed..c42dd30 100644
--- a/docs/topics/impala_avg.xml
+++ b/docs/topics/impala_avg.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">avg() function</indexterm>
+      <indexterm audience="hidden">avg() function</indexterm>
       An aggregate function that returns the average value from a set of numbers or <codeph>TIMESTAMP</codeph> values.
       Its single argument can be numeric column, or the numeric result of a function or expression applied to the
       column value. Rows with a <codeph>NULL</codeph> value for the specified column are ignored. If the table is empty,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_avro.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_avro.xml b/docs/topics/impala_avro.xml
index d1c225b..852cf8c 100644
--- a/docs/topics/impala_avro.xml
+++ b/docs/topics/impala_avro.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="1.4.0">
-      <indexterm audience="Cloudera">Avro support in Impala</indexterm>
+      <indexterm audience="hidden">Avro support in Impala</indexterm>
       Impala supports using tables whose data files use the Avro file format. Impala can query Avro
       tables, and in Impala 1.4.0 and higher can create them, but currently cannot insert data into them. For
       insert operations, use Hive, then switch back to Impala to run queries.
@@ -382,7 +382,7 @@ for this information.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">compression</indexterm>
+        <indexterm audience="hidden">compression</indexterm>
         To enable compression for Avro tables, specify settings in the Hive shell to enable compression and to
         specify a codec, then issue a <codeph>CREATE TABLE</codeph> statement as in the preceding examples. Impala
         supports the <codeph>snappy</codeph> and <codeph>deflate</codeph> codecs for Avro tables.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_batch_size.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_batch_size.xml b/docs/topics/impala_batch_size.xml
index 5a2d073..d9f36ec 100644
--- a/docs/topics/impala_batch_size.xml
+++ b/docs/topics/impala_batch_size.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">BATCH_SIZE query option</indexterm>
+      <indexterm audience="hidden">BATCH_SIZE query option</indexterm>
       Number of rows evaluated at a time by SQL operators. Unspecified or a size of 0 uses a predefined default
       size. Using a large number improves responsiveness, especially for scan operations, at the cost of a higher memory footprint.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_bit_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_bit_functions.xml b/docs/topics/impala_bit_functions.xml
index 75666b2..24f8478 100644
--- a/docs/topics/impala_bit_functions.xml
+++ b/docs/topics/impala_bit_functions.xml
@@ -95,7 +95,7 @@ shiftright
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">bitand() function</indexterm>
+          <indexterm audience="hidden">bitand() function</indexterm>
           <b>Purpose:</b> Returns an integer value representing the bits that are set to 1 in both of the arguments.
           If the arguments are of different sizes, the smaller is promoted to the type of the larger.
           <p>
@@ -164,7 +164,7 @@ select bitand(-1,15); /* 11111111 &amp; 00001111 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">bitnot() function</indexterm>
+          <indexterm audience="hidden">bitnot() function</indexterm>
           <b>Purpose:</b> Inverts all the bits of the input argument.
           <p>
             <b>Usage notes:</b> The <codeph>bitnot()</codeph> function is equivalent to the <codeph>~</codeph> unary operator.
@@ -221,7 +221,7 @@ select bitnot(-128); /* 10000000 -> 01111111 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">bitor() function</indexterm>
+          <indexterm audience="hidden">bitor() function</indexterm>
           <b>Purpose:</b> Returns an integer value representing the bits that are set to 1 in either of the arguments.
           If the arguments are of different sizes, the smaller is promoted to the type of the larger.
           <p>
@@ -265,7 +265,7 @@ select bitor(0,7); /* 00000000 | 00000111 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">bitxor() function</indexterm>
+          <indexterm audience="hidden">bitxor() function</indexterm>
           <b>Purpose:</b> Returns an integer value representing the bits that are set to 1 in one but not both of the arguments.
           If the arguments are of different sizes, the smaller is promoted to the type of the larger.
           <p>
@@ -319,7 +319,7 @@ select bitxor(3,7); /* 00000011 ^ 00000111 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">countset() function</indexterm>
+          <indexterm audience="hidden">countset() function</indexterm>
           <b>Purpose:</b> By default, returns the number of 1 bits in the specified integer value.
           If the optional second argument is set to zero, it returns the number of 0 bits instead.
           <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>
@@ -387,7 +387,7 @@ select countset(7,0); /* 00000111 = 5 0 bits; third argument can only be 0 or 1
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">getbit() function</indexterm>
+          <indexterm audience="hidden">getbit() function</indexterm>
           <b>Purpose:</b> Returns a 0 or 1 representing the bit at a
           specified position. The positions are numbered right to left, starting at zero.
           The position argument cannot be negative.
@@ -461,7 +461,7 @@ select getbit(cast(-1 as int),25); /* 11111111111111111111111111111111 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">rotateleft() function</indexterm>
+          <indexterm audience="hidden">rotateleft() function</indexterm>
           <b>Purpose:</b> Rotates an integer value left by a specified number of bits.
           As the most significant bit is taken out of the original value,
           if it is a 1 bit, it is <q>rotated</q> back to the least significant bit.
@@ -525,7 +525,7 @@ select rotateleft(-127,3); /* 10000001 -> 00001100 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">rotateright() function</indexterm>
+          <indexterm audience="hidden">rotateright() function</indexterm>
           <b>Purpose:</b> Rotates an integer value right by a specified number of bits.
           As the least significant bit is taken out of the original value,
           if it is a 1 bit, it is <q>rotated</q> back to the most significant bit.
@@ -588,7 +588,7 @@ select rotateright(-127,3); /* 10000001 -> 00110000 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">setbit() function</indexterm>
+          <indexterm audience="hidden">setbit() function</indexterm>
           <b>Purpose:</b> By default, changes a bit at a specified position to a 1, if it is not already.
           If the optional third argument is set to zero, the specified bit is set to 0 instead.
           <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>
@@ -672,7 +672,7 @@ select setbit(7,2,0); /* 00000111 -> 00000011; third argument of 0 clears instea
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">shiftleft() function</indexterm>
+          <indexterm audience="hidden">shiftleft() function</indexterm>
           <b>Purpose:</b> Shifts an integer value left by a specified number of bits.
           As the most significant bit is taken out of the original value,
           it is discarded and the least significant bit becomes 0.
@@ -748,7 +748,7 @@ select shiftleft(-1,4); /* 11111111 -> 11110000 */
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">shiftright() function</indexterm>
+          <indexterm audience="hidden">shiftright() function</indexterm>
           <b>Purpose:</b> Shifts an integer value right by a specified number of bits.
           As the least significant bit is taken out of the original value,
           it is discarded and the most significant bit becomes 0.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_breakpad.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_breakpad.xml b/docs/topics/impala_breakpad.xml
index b437da3..e5e8cc6 100644
--- a/docs/topics/impala_breakpad.xml
+++ b/docs/topics/impala_breakpad.xml
@@ -85,7 +85,7 @@ under the License.
         <!-- Location stated in IMPALA-3581; overridden by different location from IMPALA-2686?
              <filepath><varname>log_directory</varname>/minidumps/<varname>daemon_name</varname></filepath> -->
          <ul>
-          <li audience="Cloudera">
+          <li audience="hidden">
             <p>
               Clusters managed by Cloudera Manager: <filepath>/var/log/impala-minidumps/<varname>daemon_name</varname></filepath>
             </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_char.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_char.xml b/docs/topics/impala_char.xml
index 448bd01..ca6f314 100644
--- a/docs/topics/impala_char.xml
+++ b/docs/topics/impala_char.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">CHAR data type</indexterm>
+      <indexterm audience="hidden">CHAR data type</indexterm>
       A fixed-length character type, padded with trailing spaces if necessary to achieve the specified length. If
       values are longer than the specified length, Impala truncates any trailing characters.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_cluster_sizing.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_cluster_sizing.xml b/docs/topics/impala_cluster_sizing.xml
index e68bbdf..9708644 100644
--- a/docs/topics/impala_cluster_sizing.xml
+++ b/docs/topics/impala_cluster_sizing.xml
@@ -44,7 +44,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">cluster sizing</indexterm>
+      <indexterm audience="hidden">cluster sizing</indexterm>
       This document provides a very rough guideline to estimate the size of a cluster needed for a specific
       customer application. You can use this information when planning how much and what type of hardware to
       acquire for a new cluster, or when adding Impala workloads to an existing cluster.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_cm_installation.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_cm_installation.xml b/docs/topics/impala_cm_installation.xml
index a7aea7b..f1a3b04 100644
--- a/docs/topics/impala_cm_installation.xml
+++ b/docs/topics/impala_cm_installation.xml
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
-<concept id="cm_installation" audience="Cloudera">
+<concept id="cm_installation" audience="hidden">
 
   <title>Installing Impala with Cloudera Manager</title>
   <prolog>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_comments.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_comments.xml b/docs/topics/impala_comments.xml
index 82f6017..aeaa928 100644
--- a/docs/topics/impala_comments.xml
+++ b/docs/topics/impala_comments.xml
@@ -33,7 +33,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">comments (SQL)</indexterm>
+      <indexterm audience="hidden">comments (SQL)</indexterm>
       Impala supports the familiar styles of SQL comments:
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_complex_types.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_complex_types.xml b/docs/topics/impala_complex_types.xml
index 11566bd..bb53c62 100644
--- a/docs/topics/impala_complex_types.xml
+++ b/docs/topics/impala_complex_types.xml
@@ -35,9 +35,9 @@ under the License.
   <conbody>
 
     <p rev="2.3.0">
-      <indexterm audience="Cloudera">complex types</indexterm>
+      <indexterm audience="hidden">complex types</indexterm>
 
-      <indexterm audience="Cloudera">nested types</indexterm>
+      <indexterm audience="hidden">nested types</indexterm>
       <term>Complex types</term> (also referred to as <term>nested types</term>) let you represent multiple data values within a single
       row/column position. They differ from the familiar column types such as <codeph>BIGINT</codeph> and <codeph>STRING</codeph>, known as
       <term>scalar types</term> or <term>primitive types</term>, which represent a single data value within a given row/column position.
@@ -2236,7 +2236,7 @@ SELECT count(*) FROM supplier;
 
       </conbody>
 
-      <concept audience="Cloudera" id="complex_python">
+      <concept audience="hidden" id="complex_python">
 
 <!-- Hiding this subtopic for the moment because there isn't enough material related to Ibis yet, which would be useful for background or to construct examples. -->
 
@@ -2282,7 +2282,7 @@ SELECT count(*) FROM supplier;
 
     </concept>
 
-    <concept id="complex_types_ex_aggregation" audience="Cloudera">
+    <concept id="complex_types_ex_aggregation" audience="hidden">
 
       <title>Aggregating the Elements in an ARRAY</title>
 
@@ -2297,7 +2297,7 @@ SELECT count(*) FROM supplier;
 
     </concept>
 
-    <concept id="complex_types_ex_map_keys" audience="Cloudera">
+    <concept id="complex_types_ex_map_keys" audience="hidden">
 
       <title>Finding the Distinct Keys in a MAP</title>
 
@@ -2312,7 +2312,7 @@ SELECT count(*) FROM supplier;
 
     </concept>
 
-    <concept id="complex_types_ex_map_struct" audience="Cloudera">
+    <concept id="complex_types_ex_map_struct" audience="hidden">
 
       <title>Using a STRUCT as the Value Part of a MAP</title>
 
@@ -2487,7 +2487,7 @@ SELECT t1.continent, t1.country.name, t2.item
 
     </concept>
 
-    <concept audience="Cloudera" id="complex_types_no_joins">
+    <concept audience="hidden" id="complex_types_no_joins">
 
       <title>Using Complex Types without Join Queries</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_compression_codec.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_compression_codec.xml b/docs/topics/impala_compression_codec.xml
index f83cdb4..07caad3 100644
--- a/docs/topics/impala_compression_codec.xml
+++ b/docs/topics/impala_compression_codec.xml
@@ -43,7 +43,7 @@ under the License.
 <!-- Could turn into a conref. -->
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">COMPRESSION_CODEC query option</indexterm>
+      <indexterm audience="hidden">COMPRESSION_CODEC query option</indexterm>
       When Impala writes Parquet data files using the <codeph>INSERT</codeph> statement, the underlying compression
       is controlled by the <codeph>COMPRESSION_CODEC</codeph> query option.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_compute_stats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_compute_stats.xml b/docs/topics/impala_compute_stats.xml
index 31d9249..8142da4 100644
--- a/docs/topics/impala_compute_stats.xml
+++ b/docs/topics/impala_compute_stats.xml
@@ -39,7 +39,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">COMPUTE STATS statement</indexterm>
+      <indexterm audience="hidden">COMPUTE STATS statement</indexterm>
       Gathers information about volume and distribution of data in a table and all associated columns and
       partitions. The information is stored in the metastore database, and used by Impala to help optimize queries.
       For example, if Impala can determine that a table is large or small, or has many or few distinct values it

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_concepts.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_concepts.xml b/docs/topics/impala_concepts.xml
index 942cbd0..b8b32a0 100644
--- a/docs/topics/impala_concepts.xml
+++ b/docs/topics/impala_concepts.xml
@@ -48,14 +48,14 @@ under the License.
 
 <!-- These other topics are waiting to be filled in. Could become subtopics or top-level topics depending on the depth of coverage in each case. -->
 
-  <concept id="intro_data_lifecycle" audience="Cloudera">
+  <concept id="intro_data_lifecycle" audience="hidden">
 
     <title>Overview of the Data Lifecycle for Impala</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_etl" audience="Cloudera">
+  <concept id="intro_etl" audience="hidden">
 
     <title>Overview of the Extract, Transform, Load (ETL) Process for Impala</title>
   <prolog>
@@ -69,28 +69,28 @@ under the License.
     <conbody/>
   </concept>
 
-  <concept id="intro_hadoop_data" audience="Cloudera">
+  <concept id="intro_hadoop_data" audience="hidden">
 
     <title>How Impala Works with Hadoop Data Files</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_web_ui" audience="Cloudera">
+  <concept id="intro_web_ui" audience="hidden">
 
     <title>Overview of the Impala Web Interface</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_bi" audience="Cloudera">
+  <concept id="intro_bi" audience="hidden">
 
     <title>Using Impala with Business Intelligence Tools</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_ha" audience="Cloudera">
+  <concept id="intro_ha" audience="hidden">
 
     <title>Overview of Impala Availability and Fault Tolerance</title>
 
@@ -102,7 +102,7 @@ under the License.
      to make it visible.
 -->
 
-  <concept id="intro_llvm" audience="Cloudera">
+  <concept id="intro_llvm" audience="hidden">
 
     <title>Overview of Impala Runtime Code Generation</title>
 
@@ -187,7 +187,7 @@ under the License.
 
 <!-- Same as the previous section: adapted from CIDR paper, ready to externalize after deciding where to go. -->
 
-  <concept audience="Cloudera" id="intro_io">
+  <concept audience="hidden" id="intro_io">
 
     <title>Overview of Impala I/O</title>
 
@@ -215,7 +215,7 @@ under the License.
 
 <!-- Although good idea to get some answers from Henry first. -->
 
-  <concept audience="Cloudera" id="intro_state_distribution">
+  <concept audience="hidden" id="intro_state_distribution">
 
     <title>State distribution</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_conditional_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_conditional_functions.xml b/docs/topics/impala_conditional_functions.xml
index 1056db1..9006a54 100644
--- a/docs/topics/impala_conditional_functions.xml
+++ b/docs/topics/impala_conditional_functions.xml
@@ -46,7 +46,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">CASE expression</indexterm>
+          <indexterm audience="hidden">CASE expression</indexterm>
           <b>Purpose:</b> Compares an expression to one or more possible values, and returns a corresponding result
           when a match is found.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
@@ -83,7 +83,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">CASE expression</indexterm>
+          <indexterm audience="hidden">CASE expression</indexterm>
           <b>Purpose:</b> Tests whether any of a sequence of expressions is true, and returns a corresponding
           result for the first true expression.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
@@ -146,7 +146,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">coalesce() function</indexterm>
+          <indexterm audience="hidden">coalesce() function</indexterm>
           <b>Purpose:</b> Returns the first specified argument that is not <codeph>NULL</codeph>, or
           <codeph>NULL</codeph> if all arguments are <codeph>NULL</codeph>.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
@@ -162,7 +162,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">decode() function</indexterm>
+          <indexterm audience="hidden">decode() function</indexterm>
           <b>Purpose:</b> Compares an expression to one or more possible values, and returns a corresponding result
           when a match is found.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
@@ -201,7 +201,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">if() function</indexterm>
+          <indexterm audience="hidden">if() function</indexterm>
           <b>Purpose:</b> Tests an expression and returns a corresponding result depending on whether the result is
           true, false, or <codeph>NULL</codeph>.
           <p>
@@ -218,7 +218,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">isnull() function</indexterm>
+          <indexterm audience="hidden">isnull() function</indexterm>
           <b>Purpose:</b> Alias for the <codeph>isnull()</codeph> function, with the same behavior. To simplify
           porting SQL with vendor extensions to Impala.
           <p conref="../shared/impala_common.xml#common/added_in_130"/>
@@ -233,7 +233,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">isfalse() function</indexterm>
+          <indexterm audience="hidden">isfalse() function</indexterm>
           <b>Purpose:</b> Tests if a Boolean expression is <codeph>false</codeph> or not.
           Returns <codeph>true</codeph> if so.
           If the argument is <codeph>NULL</codeph>, returns <codeph>false</codeph>.
@@ -251,7 +251,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">isnotfalse() function</indexterm>
+          <indexterm audience="hidden">isnotfalse() function</indexterm>
           <b>Purpose:</b> Tests if a Boolean expression is not <codeph>false</codeph> (that is, either <codeph>true</codeph> or <codeph>NULL</codeph>).
           Returns <codeph>true</codeph> if so.
           If the argument is <codeph>NULL</codeph>, returns <codeph>true</codeph>.
@@ -270,7 +270,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">isnottrue() function</indexterm>
+          <indexterm audience="hidden">isnottrue() function</indexterm>
           <b>Purpose:</b> Tests if a Boolean expression is not <codeph>true</codeph> (that is, either <codeph>false</codeph> or <codeph>NULL</codeph>).
           Returns <codeph>true</codeph> if so.
           If the argument is <codeph>NULL</codeph>, returns <codeph>true</codeph>.
@@ -288,7 +288,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">isnull() function</indexterm>
+          <indexterm audience="hidden">isnull() function</indexterm>
           <b>Purpose:</b> Tests if an expression is <codeph>NULL</codeph>, and returns the expression result value
           if not. If the first argument is <codeph>NULL</codeph>, returns the second argument.
           <p>
@@ -310,7 +310,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">istrue() function</indexterm>
+          <indexterm audience="hidden">istrue() function</indexterm>
           <b>Purpose:</b> Tests if a Boolean expression is <codeph>true</codeph> or not.
           Returns <codeph>true</codeph> if so.
           If the argument is <codeph>NULL</codeph>, returns <codeph>false</codeph>.
@@ -329,7 +329,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">function</indexterm>
+          <indexterm audience="hidden">function</indexterm>
           <b>Purpose:</b> Tests if an expression (of any type) is <codeph>NULL</codeph> or not.
           Returns <codeph>false</codeph> if so.
           The converse of <codeph>nullvalue()</codeph>.
@@ -347,7 +347,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">nullif() function</indexterm>
+          <indexterm audience="hidden">nullif() function</indexterm>
           <b>Purpose:</b> Returns <codeph>NULL</codeph> if the two specified arguments are equal. If the specified
           arguments are not equal, returns the value of <varname>expr1</varname>. The data types of the expressions
           must be compatible, according to the conversion rules from <xref href="impala_datatypes.xml#datatypes"/>.
@@ -384,7 +384,7 @@ END</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">nullifzero() function</indexterm>
+          <indexterm audience="hidden">nullifzero() function</indexterm>
           <b>Purpose:</b> Returns <codeph>NULL</codeph> if the numeric expression evaluates to 0, otherwise returns
           the result of the expression.
           <p>
@@ -405,7 +405,7 @@ END</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">function</indexterm>
+          <indexterm audience="hidden">function</indexterm>
           <b>Purpose:</b> Tests if an expression (of any type) is <codeph>NULL</codeph> or not.
           Returns <codeph>true</codeph> if so.
           The converse of <codeph>nonnullvalue()</codeph>.
@@ -423,7 +423,7 @@ END</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">nvl() function</indexterm>
+          <indexterm audience="hidden">nvl() function</indexterm>
           <b>Purpose:</b> Alias for the <codeph>isnull()</codeph> function. Tests if an expression is
           <codeph>NULL</codeph>, and returns the expression result value if not. If the first argument is
           <codeph>NULL</codeph>, returns the second argument. Equivalent to the <codeph>nvl()</codeph> function
@@ -443,7 +443,7 @@ END</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">zeroifnull() function</indexterm>
+          <indexterm audience="hidden">zeroifnull() function</indexterm>
           <b>Purpose:</b> Returns 0 if the numeric expression evaluates to <codeph>NULL</codeph>, otherwise returns
           the result of the expression.
           <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_config_options.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_config_options.xml b/docs/topics/impala_config_options.xml
index bc91f9d..0f966bc 100644
--- a/docs/topics/impala_config_options.xml
+++ b/docs/topics/impala_config_options.xml
@@ -34,29 +34,29 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">defaults file</indexterm>
+      <indexterm audience="hidden">defaults file</indexterm>
 
-      <indexterm audience="Cloudera">configuration file</indexterm>
+      <indexterm audience="hidden">configuration file</indexterm>
 
-      <indexterm audience="Cloudera">options</indexterm>
+      <indexterm audience="hidden">options</indexterm>
 
-      <indexterm audience="Cloudera">IMPALA_STATE_STORE_PORT</indexterm>
+      <indexterm audience="hidden">IMPALA_STATE_STORE_PORT</indexterm>
 
-      <indexterm audience="Cloudera">IMPALA_BACKEND_PORT</indexterm>
+      <indexterm audience="hidden">IMPALA_BACKEND_PORT</indexterm>
 
-      <indexterm audience="Cloudera">IMPALA_LOG_DIR</indexterm>
+      <indexterm audience="hidden">IMPALA_LOG_DIR</indexterm>
 
-      <indexterm audience="Cloudera">IMPALA_STATE_STORE_ARGS</indexterm>
+      <indexterm audience="hidden">IMPALA_STATE_STORE_ARGS</indexterm>
 
-      <indexterm audience="Cloudera">IMPALA_SERVER_ARGS</indexterm>
+      <indexterm audience="hidden">IMPALA_SERVER_ARGS</indexterm>
 
-      <indexterm audience="Cloudera">ENABLE_CORE_DUMPS</indexterm>
+      <indexterm audience="hidden">ENABLE_CORE_DUMPS</indexterm>
 
-      <indexterm audience="Cloudera">core dumps</indexterm>
+      <indexterm audience="hidden">core dumps</indexterm>
 
-      <indexterm audience="Cloudera">restarting services</indexterm>
+      <indexterm audience="hidden">restarting services</indexterm>
 
-      <indexterm audience="Cloudera">services</indexterm>
+      <indexterm audience="hidden">services</indexterm>
       The configuration options for the Impala-related daemons let you choose which hosts and
       ports to use for the services that run on a single host, specify directories for logging,
       control resource usage and security, and specify other aspects of the Impala software.
@@ -66,7 +66,7 @@ under the License.
 
   </conbody>
 
-  <concept id="config_options_cm" audience="Cloudera">
+  <concept id="config_options_cm" audience="hidden">
 
     <title>Configuring Impala Startup Options through Cloudera Manager</title>
 
@@ -243,7 +243,7 @@ Starting Impala Catalog Server:                            [  OK  ]</codeblock>
             to:
           </p>
 <codeblock>export ENABLE_CORE_DUMPS=${ENABLE_COREDUMPS:-true}</codeblock>
-          <p audience="Cloudera">
+          <p audience="hidden">
             On systems managed by Cloudera Manager, enable the <uicontrol>Enable Core
             Dump</uicontrol> setting for the Impala service.
           </p>
@@ -339,11 +339,11 @@ Starting Impala Catalog Server:                            [  OK  ]</codeblock>
         </p>
       </note>
 
-      <p audience="Cloudera" outputclass="toc inpage"/>
+      <p audience="hidden" outputclass="toc inpage"/>
 
     </conbody>
 
-    <concept audience="Cloudera" id="config_options_impalad_details">
+    <concept audience="hidden" id="config_options_impalad_details">
 
       <title>Configuration Options for impalad Daemon</title>
 
@@ -510,7 +510,7 @@ Starting Impala Catalog Server:                            [  OK  ]</codeblock>
 
     </concept>
 
-    <concept audience="Cloudera" id="config_options_statestored_details">
+    <concept audience="hidden" id="config_options_statestored_details">
 
       <title>Configuration Options for statestored Daemon</title>
 
@@ -522,7 +522,7 @@ Starting Impala Catalog Server:                            [  OK  ]</codeblock>
 
     </concept>
 
-    <concept audience="Cloudera" id="config_options_catalogd_details">
+    <concept audience="hidden" id="config_options_catalogd_details">
 
       <title>Configuration Options for catalogd Daemon</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_config_performance.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_config_performance.xml b/docs/topics/impala_config_performance.xml
index 7c2e270..d14407e 100644
--- a/docs/topics/impala_config_performance.xml
+++ b/docs/topics/impala_config_performance.xml
@@ -90,10 +90,10 @@ under the License.
           Impala configuration directory. The default Impala configuration
           location is <codeph>/etc/impala/conf</codeph>. </li>
         <li>
-          <indexterm audience="Cloudera"
+          <indexterm audience="hidden"
             >dfs.client.read.shortcircuit</indexterm>
-          <indexterm audience="Cloudera">dfs.domain.socket.path</indexterm>
-          <indexterm audience="Cloudera"
+          <indexterm audience="hidden">dfs.domain.socket.path</indexterm>
+          <indexterm audience="hidden"
             >dfs.client.file-block-storage-locations.timeout.millis</indexterm>
           On all Impala nodes, configure the following properties in <!-- Exact timing is unclear, since we say farther down to copy /etc/hadoop/conf/hdfs-site.xml to /etc/impala/conf.
      Which wouldn't work if we already modified the Impala version of the file here. Not to mention that this

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_conversion_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_conversion_functions.xml b/docs/topics/impala_conversion_functions.xml
index d69b9ed..4d38790 100644
--- a/docs/topics/impala_conversion_functions.xml
+++ b/docs/topics/impala_conversion_functions.xml
@@ -67,7 +67,7 @@ under the License.
 </dt>
 
 <dd>
-<indexterm audience="Cloudera">cast() function</indexterm>
+<indexterm audience="hidden">cast() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to any other type.
 If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Usage notes:</b>
@@ -136,12 +136,12 @@ describe t2;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttobigint" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttobigint" audience="hidden">
 <dt>
 <codeph>casttobigint(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttobigint() function</indexterm>
+<indexterm audience="hidden">casttobigint() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>BIGINT</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>bigint</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -166,12 +166,12 @@ describe big_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttoboolean" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttoboolean" audience="hidden">
 <dt>
 <codeph>casttoboolean(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttoboolean() function</indexterm>
+<indexterm audience="hidden">casttoboolean() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>BOOLEAN</codeph>.
 Numeric values of 0 evaluate to <codeph>false</codeph>, and non-zero values evaluate to <codeph>true</codeph>.
 If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
@@ -255,12 +255,12 @@ select casttoboolean('false');
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttochar" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttochar" audience="hidden">
 <dt>
 <codeph>casttochar(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttochar() function</indexterm>
+<indexterm audience="hidden">casttochar() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>CHAR</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>char</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -286,12 +286,12 @@ describe char_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttodecimal" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttodecimal" audience="hidden">
 <dt>
 <codeph>casttodecimal(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttodecimal() function</indexterm>
+<indexterm audience="hidden">casttodecimal() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>DECIMAL</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>decimal</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -308,12 +308,12 @@ describe char_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttodouble" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttodouble" audience="hidden">
 <dt>
 <codeph>casttodouble(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttodouble() function</indexterm>
+<indexterm audience="hidden">casttodouble() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>DOUBLE</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>double</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -358,12 +358,12 @@ select casttodouble(now());
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttofloat" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttofloat" audience="hidden">
 <dt>
 <codeph>casttofloat(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttofloat() function</indexterm>
+<indexterm audience="hidden">casttofloat() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>FLOAT</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>float</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -408,12 +408,12 @@ select casttofloat(now());
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttoint" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttoint" audience="hidden">
 <dt>
 <codeph>casttoint(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttoint() function</indexterm>
+<indexterm audience="hidden">casttoint() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>INT</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>int</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -458,12 +458,12 @@ select casttoint('3');
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttosmallint" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttosmallint" audience="hidden">
 <dt>
 <codeph>casttosmallint(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttosmallint() function</indexterm>
+<indexterm audience="hidden">casttosmallint() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>SMALLINT</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>smallint</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -488,12 +488,12 @@ describe small_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttostring" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttostring" audience="hidden">
 <dt>
 <codeph>casttostring(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttostring() function</indexterm>
+<indexterm audience="hidden">casttostring() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>STRING</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>string</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -518,12 +518,12 @@ describe string_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttotimestamp" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttotimestamp" audience="hidden">
 <dt>
 <codeph>casttotimestamp(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttotimestamp() function</indexterm>
+<indexterm audience="hidden">casttotimestamp() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>TIMESTAMP</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>timestamp</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -554,12 +554,12 @@ select casttotimestamp('1000');
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttotinyint" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttotinyint" audience="hidden">
 <dt>
 <codeph>casttotinyint(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttotinyint() function</indexterm>
+<indexterm audience="hidden">casttotinyint() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>TINYINT</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>tinyint</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -584,12 +584,12 @@ describe tiny_types;
 </dd>
 </dlentry>
 
-<dlentry rev="2.3.0" id="casttovarchar" audience="Cloudera">
+<dlentry rev="2.3.0" id="casttovarchar" audience="hidden">
 <dt>
 <codeph>casttovarchar(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">casttovarchar() function</indexterm>
+<indexterm audience="hidden">casttovarchar() function</indexterm>
 <b>Purpose:</b> Converts the value of an expression to <codeph>VARCHAR</codeph>. If the expression value is of a type that cannot be converted to the target type, the result is <codeph>NULL</codeph>.
 <p><b>Return type:</b> <codeph>varchar</codeph></p>
 <p conref="../shared/impala_common.xml#common/cast_convenience_fn_usage"/>
@@ -639,7 +639,7 @@ select casttovarchar(true);
 <codeph>typeof(type value)</codeph>
 </dt>
 <dd>
-<indexterm audience="Cloudera">typeof() function</indexterm>
+<indexterm audience="hidden">typeof() function</indexterm>
 <b>Purpose:</b> Returns the name of the data type corresponding to an expression. For types with
 extra attributes, such as length for <codeph>CHAR</codeph> and <codeph>VARCHAR</codeph>,
 or precision and scale for <codeph>DECIMAL</codeph>, includes the full specification of the type.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_count.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_count.xml b/docs/topics/impala_count.xml
index d492a8f..59180c7 100644
--- a/docs/topics/impala_count.xml
+++ b/docs/topics/impala_count.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">count() function</indexterm>
+      <indexterm audience="hidden">count() function</indexterm>
       An aggregate function that returns the number of rows, or the number of non-<codeph>NULL</codeph> rows.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_data_source.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_data_source.xml b/docs/topics/impala_create_data_source.xml
index 9372147..cc59d0e 100644
--- a/docs/topics/impala_create_data_source.xml
+++ b/docs/topics/impala_create_data_source.xml
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
-<concept audience="Cloudera" rev="1.4.0" id="create_data_source">
+<concept audience="hidden" rev="1.4.0" id="create_data_source">
 
   <title>CREATE DATA SOURCE Statement</title>
   <titlealts audience="PDF"><navtitle>CREATE DATA SOURCE</navtitle></titlealts>
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE DATA SOURCE statement</indexterm>
+      <indexterm audience="hidden">CREATE DATA SOURCE statement</indexterm>
     </p>
 
     <p conref="../shared/impala_common.xml#common/ddl_blurb"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_database.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_database.xml b/docs/topics/impala_create_database.xml
index 1fc9d45..871bbc6 100644
--- a/docs/topics/impala_create_database.xml
+++ b/docs/topics/impala_create_database.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE DATABASE statement</indexterm>
+      <indexterm audience="hidden">CREATE DATABASE statement</indexterm>
       Creates a new database.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_function.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_function.xml b/docs/topics/impala_create_function.xml
index a02038e..15f8eb2 100644
--- a/docs/topics/impala_create_function.xml
+++ b/docs/topics/impala_create_function.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE FUNCTION statement</indexterm>
+      <indexterm audience="hidden">CREATE FUNCTION statement</indexterm>
       Creates a user-defined function (UDF), which you can use to implement custom logic during
       <codeph>SELECT</codeph> or <codeph>INSERT</codeph> operations.
     </p>
@@ -215,7 +215,7 @@ DROP FUNCTION IF EXISTS bar;
       determine the names based on the first such clause, so the others are optional.
     </p>
 
-    <p audience="Cloudera">
+    <p audience="hidden">
       The <codeph>INTERMEDIATE</codeph> clause specifies the data type of intermediate values passed from the
       <q>update</q> phase to the <q>merge</q> phase, and from the <q>merge</q> phase to the <q>finalize</q> phase.
       You can use any of the existing Impala data types, or the special notation

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_role.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_role.xml b/docs/topics/impala_create_role.xml
index 3f186b1..7c48ae9 100644
--- a/docs/topics/impala_create_role.xml
+++ b/docs/topics/impala_create_role.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE ROLE statement</indexterm>
+      <indexterm audience="hidden">CREATE ROLE statement</indexterm>
 <!-- Copied from Sentry docs. Turn into conref. -->
       The <codeph>CREATE ROLE</codeph> statement creates a role to which privileges can be granted. Privileges can
       be granted to roles, which can then be assigned to users. A user that has been assigned a role will only be

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_table.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_table.xml b/docs/topics/impala_create_table.xml
index bd2e12c..818553d 100644
--- a/docs/topics/impala_create_table.xml
+++ b/docs/topics/impala_create_table.xml
@@ -41,7 +41,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE TABLE statement</indexterm>
+      <indexterm audience="hidden">CREATE TABLE statement</indexterm>
       Creates a new table and specifies its characteristics. While creating a table, you optionally specify aspects
       such as:
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_create_view.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_create_view.xml b/docs/topics/impala_create_view.xml
index de0dbbf..c638b69 100644
--- a/docs/topics/impala_create_view.xml
+++ b/docs/topics/impala_create_view.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">CREATE VIEW statement</indexterm>
+      <indexterm audience="hidden">CREATE VIEW statement</indexterm>
       The <codeph>CREATE VIEW</codeph> statement lets you create a shorthand abbreviation for a more complicated
       query. The base query can involve joins, expressions, reordered columns, column aliases, and other SQL
       features that can make a query hard to understand or maintain.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_datatypes.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_datatypes.xml b/docs/topics/impala_datatypes.xml
index 7c6547c..8076df6 100644
--- a/docs/topics/impala_datatypes.xml
+++ b/docs/topics/impala_datatypes.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">data types</indexterm>
+      <indexterm audience="hidden">data types</indexterm>
       Impala supports a set of data types that you can use for table columns, expression values, and function
       arguments and return values.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_date.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_date.xml b/docs/topics/impala_date.xml
index 0576152..fcd5a25 100644
--- a/docs/topics/impala_date.xml
+++ b/docs/topics/impala_date.xml
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
-<concept audience="Cloudera" id="date" rev="2.0.0">
+<concept audience="hidden" id="date" rev="2.0.0">
 
   <title>DATE Data Type (<keyword keyref="impala21"/> or higher only)</title>
   <prolog>
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DATE data type</indexterm>
+      <indexterm audience="hidden">DATE data type</indexterm>
       A type representing the date (year, month, and day) as a single numeric value. Used to represent a broader
       date range than possible with the <codeph>TIMESTAMP</codeph> type, with fewer distinct values than
       <codeph>TIMESTAMP</codeph>, and in a more compact and efficient form than using a <codeph>STRING</codeph>


[2/6] incubator-impala git commit: IMPALA-4768: Improve logging of table loading.

Posted by jr...@apache.org.
IMPALA-4768: Improve logging of table loading.

- Improves the logging for several important events,
  in particular, during table loading.
- Uses LOG.info() for such messages to clarify their
  intent.

The goal is to improve supportability without having
to turn on trace debugging which can generate a
significant log volume.

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


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

Branch: refs/heads/master
Commit: 3b7a179197bccdebf9c8492477f0cf88c6df0e18
Parents: fa4a054
Author: Alex Behm <al...@cloudera.com>
Authored: Thu Jan 12 16:54:31 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Jan 17 02:25:11 2017 +0000

----------------------------------------------------------------------
 .../impala/catalog/CatalogServiceCatalog.java   | 37 +++++++++-----------
 .../org/apache/impala/catalog/HdfsTable.java    | 27 +++++++-------
 .../org/apache/impala/catalog/TableLoader.java  |  5 +--
 .../impala/service/CatalogOpExecutor.java       |  2 +-
 .../org/apache/impala/service/Frontend.java     |  7 ++--
 5 files changed, 39 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3b7a1791/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
index b51c9aa..31001a2 100644
--- a/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
+++ b/fe/src/main/java/org/apache/impala/catalog/CatalogServiceCatalog.java
@@ -194,9 +194,8 @@ public class CatalogServiceCatalog extends Catalog {
     }
 
     public void run() {
-      if (LOG.isTraceEnabled()) {
-        LOG.trace("Reloading cache pool names from HDFS");
-      }
+      if (LOG.isTraceEnabled()) LOG.trace("Reloading cache pool names from HDFS");
+
       // Map of cache pool name to CachePoolInfo. Stored in a map to allow Set operations
       // to be performed on the keys.
       Map<String, CachePoolInfo> currentCachePools = Maps.newHashMap();
@@ -523,9 +522,7 @@ public class CatalogServiceCatalog extends Catalog {
   private void loadFunctionsFromDbParams(Db db,
       org.apache.hadoop.hive.metastore.api.Database msDb) {
     if (msDb == null || msDb.getParameters() == null) return;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Loading native functions for database: " + db.getName());
-    }
+    LOG.info("Loading native functions for database: " + db.getName());
     TCompactProtocol.Factory protocolFactory = new TCompactProtocol.Factory();
     for (String key: msDb.getParameters().keySet()) {
       if (!key.startsWith(Db.FUNCTION_INDEX_PREFIX)) continue;
@@ -541,6 +538,7 @@ public class CatalogServiceCatalog extends Catalog {
             + ",continuing", e);
       }
     }
+    LOG.info("Loaded native functions for database: " + db.getName());
   }
 
   /**
@@ -551,9 +549,7 @@ public class CatalogServiceCatalog extends Catalog {
   private void loadJavaFunctions(Db db,
       List<org.apache.hadoop.hive.metastore.api.Function> functions) {
     Preconditions.checkNotNull(functions);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Loading Java functions for database: " + db.getName());
-    }
+    LOG.info("Loading Java functions for database: " + db.getName());
     for (org.apache.hadoop.hive.metastore.api.Function function: functions) {
       try {
         for (Function fn: extractFunctions(db.getName(), function)) {
@@ -564,6 +560,7 @@ public class CatalogServiceCatalog extends Catalog {
         LOG.error("Skipping function load: " + function.getFunctionName(), e);
       }
     }
+    LOG.info("Loaded Java functions for database: " + db.getName());
   }
 
   /**
@@ -622,6 +619,8 @@ public class CatalogServiceCatalog extends Catalog {
    * Resets this catalog instance by clearing all cached table and database metadata.
    */
   public void reset() throws CatalogException {
+    LOG.info("Invalidating all metadata.");
+
     // First update the policy metadata.
     if (sentryProxy_ != null) {
       // Sentry Service is enabled.
@@ -665,6 +664,7 @@ public class CatalogServiceCatalog extends Catalog {
     } finally {
       catalogLock_.writeLock().unlock();
     }
+    LOG.info("Invalidated all metadata.");
   }
 
   /**
@@ -893,9 +893,7 @@ public class CatalogServiceCatalog extends Catalog {
    * Throws a CatalogException if there is an error loading table metadata.
    */
   public Table reloadTable(Table tbl) throws CatalogException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Refreshing table metadata: %s", tbl.getFullName()));
-    }
+    LOG.info(String.format("Refreshing table metadata: %s", tbl.getFullName()));
     TTableName tblName = new TTableName(tbl.getDb().getName().toLowerCase(),
         tbl.getName().toLowerCase());
     Db db = tbl.getDb();
@@ -917,6 +915,7 @@ public class CatalogServiceCatalog extends Catalog {
         return replaceTableIfUnchanged(loadReq.get(), previousCatalogVersion);
       } finally {
         loadReq.close();
+        LOG.info(String.format("Refreshed table metadata: %s", tbl.getFullName()));
       }
     }
 
@@ -936,6 +935,7 @@ public class CatalogServiceCatalog extends Catalog {
         tbl.load(false, msClient.getHiveClient(), msTbl);
       }
       tbl.setCatalogVersion(newCatalogVersion);
+      LOG.info(String.format("Refreshed table metadata: %s", tbl.getFullName()));
       return tbl;
     }
   }
@@ -1024,12 +1024,9 @@ public class CatalogServiceCatalog extends Catalog {
     Preconditions.checkNotNull(updatedObjects);
     updatedObjects.first = null;
     updatedObjects.second = null;
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("Invalidating table metadata: %s.%s",
-          tableName.getDb_name(), tableName.getTable_name()));
-    }
     String dbName = tableName.getDb_name();
     String tblName = tableName.getTable_name();
+    LOG.info(String.format("Invalidating table metadata: %s.%s", dbName, tblName));
 
     // Stores whether the table exists in the metastore. Can have three states:
     // 1) true - Table exists in metastore.
@@ -1266,10 +1263,8 @@ public class CatalogServiceCatalog extends Catalog {
       String partitionName = hdfsPartition == null
           ? HdfsTable.constructPartitionName(partitionSpec)
           : hdfsPartition.getPartitionName();
-      if (LOG.isTraceEnabled()) {
-        LOG.trace(String.format("Refreshing Partition metadata: %s %s",
-            hdfsTable.getFullName(), partitionName));
-      }
+      LOG.info(String.format("Refreshing partition metadata: %s %s",
+          hdfsTable.getFullName(), partitionName));
       try (MetaStoreClient msClient = getMetaStoreClient()) {
         org.apache.hadoop.hive.metastore.api.Partition hmsPartition = null;
         try {
@@ -1290,6 +1285,8 @@ public class CatalogServiceCatalog extends Catalog {
         hdfsTable.reloadPartition(hdfsPartition, hmsPartition);
       }
       hdfsTable.setCatalogVersion(newCatalogVersion);
+      LOG.info(String.format("Refreshed partition metadata: %s %s",
+          hdfsTable.getFullName(), partitionName));
       return hdfsTable;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3b7a1791/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 8fa4f21..857216f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -272,6 +272,7 @@ public class HdfsTable extends Table {
         synthesizeBlockMetadata(fs, dirPath, partsByPath);
         return;
       }
+
       int unknownDiskIdCount = 0;
       RemoteIterator<LocatedFileStatus> fileStatusIter = fs.listFiles(dirPath, true);
       while (fileStatusIter.hasNext()) {
@@ -728,7 +729,7 @@ public class HdfsTable extends Table {
         }
       }
     }
-    if (LOG.isTraceEnabled()) LOG.trace("partsByPath size: " + partsByPath.size());
+
     loadMetadataAndDiskIds(dirsToLoad, partsByPath);
   }
 
@@ -746,8 +747,13 @@ public class HdfsTable extends Table {
    */
   private void loadMetadataAndDiskIds(List<Path> locations,
       HashMap<Path, List<HdfsPartition>> partsByPath) {
+    LOG.info(String.format("Loading file and block metadata for %s partitions: %s",
+        partsByPath.size(), getFullName()));
     for (Path location: locations) { loadBlockMetadata(location, partsByPath); }
+    LOG.info(String.format("Loaded file and block metadata for %s partitions: %s",
+        partsByPath.size(), getFullName()));
   }
+
   /**
    * Gets the AccessLevel that is available for Impala for this table based on the
    * permissions Impala has on the given path. If the path does not exist, recurses up
@@ -1029,9 +1035,7 @@ public class HdfsTable extends Table {
       // Load partition and file metadata
       if (reuseMetadata) {
         // Incrementally update this table's partitions and file metadata
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("incremental update for table: " + db_.getName() + "." + name_);
-        }
+        LOG.info("Incrementally loading table metadata for: " + getFullName());
         Preconditions.checkState(partitionsToUpdate == null || loadFileMetadata);
         updateMdFromHmsTable(msTbl);
         if (msTbl.getPartitionKeysSize() == 0) {
@@ -1039,15 +1043,14 @@ public class HdfsTable extends Table {
         } else {
           updatePartitionsFromHms(client, partitionsToUpdate, loadFileMetadata);
         }
+        LOG.info("Incrementally loaded table metadata for: " + getFullName());
       } else {
         // Load all partitions from Hive Metastore, including file metadata.
-        if (LOG.isTraceEnabled()) {
-          LOG.trace("load table from Hive Metastore: " + db_.getName() + "." + name_);
-        }
+        LOG.info("Fetching partition metadata from the Metastore: " + getFullName());
         List<org.apache.hadoop.hive.metastore.api.Partition> msPartitions =
-            Lists.newArrayList();
-        msPartitions.addAll(MetaStoreUtil.fetchAllPartitions(
-            client, db_.getName(), name_, NUM_PARTITION_FETCH_RETRIES));
+            MetaStoreUtil.fetchAllPartitions(
+                client, db_.getName(), name_, NUM_PARTITION_FETCH_RETRIES);
+        LOG.info("Fetched partition metadata from the Metastore: " + getFullName());
         loadAllPartitions(msPartitions, msTbl);
       }
       if (loadTableSchema) setAvroSchema(client, msTbl);
@@ -1107,9 +1110,7 @@ public class HdfsTable extends Table {
    */
   private void updatePartitionsFromHms(IMetaStoreClient client,
       Set<String> partitionsToUpdate, boolean loadFileMetadata) throws Exception {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("sync table partitions: " + name_);
-    }
+    if (LOG.isTraceEnabled()) LOG.trace("Sync table partitions: " + name_);
     org.apache.hadoop.hive.metastore.api.Table msTbl = getMetaStoreTable();
     Preconditions.checkNotNull(msTbl);
     Preconditions.checkState(msTbl.getPartitionKeysSize() != 0);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3b7a1791/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
index 7f17a5c..2a1ee15 100644
--- a/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
+++ b/fe/src/main/java/org/apache/impala/catalog/TableLoader.java
@@ -55,7 +55,7 @@ public class TableLoader {
    */
   public Table load(Db db, String tblName) {
     String fullTblName = db.getName() + "." + tblName;
-    if (LOG.isTraceEnabled()) LOG.trace("Loading metadata for: " + fullTblName);
+    LOG.info("Loading metadata for: " + fullTblName);
     Table table;
     // turn all exceptions into TableLoadingException
     try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
@@ -88,12 +88,13 @@ public class TableLoader {
           "catalog.");
       table = IncompleteTable.createFailedMetadataLoadTable(
           db, tblName, tableDoesNotExist);
-    } catch (Exception e) {
+    } catch (Throwable e) {
       table = IncompleteTable.createFailedMetadataLoadTable(
           db, tblName, new TableLoadingException(
           "Failed to load metadata for table: " + fullTblName + ". Running " +
           "'invalidate metadata " + fullTblName + "' may resolve this problem.", e));
     }
+    LOG.info("Loaded metadata for: " + fullTblName);
     return table;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3b7a1791/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index d9448b3..208ff2b 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -559,7 +559,7 @@ public class CatalogOpExecutor {
 
   /**
    * Loads the metadata of a table 'tbl' and assigns a new catalog version.
-   * reloadFileMetadata', 'reloadTableSchema', and 'partitionsToUpdate'
+   * 'reloadFileMetadata', 'reloadTableSchema', and 'partitionsToUpdate'
    * are used only for HdfsTables and control which metadata to reload.
    * Throws a CatalogException if there is an error loading table metadata.
    */

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/3b7a1791/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index eca8d65..1030111 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -887,7 +887,7 @@ public class Frontend {
 
     AnalysisContext analysisCtx = new AnalysisContext(impaladCatalog_, queryCtx,
         authzConfig_);
-    if (LOG.isTraceEnabled()) LOG.trace("analyze query " + queryCtx.client_request.stmt);
+    LOG.info("Compiling query: " + queryCtx.client_request.stmt);
 
     // Run analysis in a loop until it any of the following events occur:
     // 1) Analysis completes successfully.
@@ -909,8 +909,8 @@ public class Frontend {
 
           // Some tables/views were missing, request and wait for them to load.
           if (!requestTblLoadAndWait(missingTbls, MISSING_TBL_LOAD_WAIT_TIMEOUT_MS)) {
-            if (LOG.isTraceEnabled()) {
-              LOG.trace(String.format("Missing tables were not received in %dms. Load " +
+            if (LOG.isWarnEnabled()) {
+              LOG.warn(String.format("Missing tables were not received in %dms. Load " +
                   "request will be retried.", MISSING_TBL_LOAD_WAIT_TIMEOUT_MS));
             }
             analysisCtx.getTimeline().markEvent("Metadata load timeout");
@@ -924,6 +924,7 @@ public class Frontend {
       // AuthorizationExceptions must take precedence over any AnalysisException
       // that has been thrown, so perform the authorization first.
       analysisCtx.authorize(getAuthzChecker());
+      LOG.info("Compiled query.");
     }
   }
 


[4/6] incubator-impala git commit: Global search/replace: audience="Cloudera" -> audience="hidden".

Posted by jr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_math_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_math_functions.xml b/docs/topics/impala_math_functions.xml
index bfdb91c..d63b0e7 100644
--- a/docs/topics/impala_math_functions.xml
+++ b/docs/topics/impala_math_functions.xml
@@ -77,7 +77,7 @@ under the License.
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">abs() function</indexterm>
+          <indexterm audience="hidden">abs() function</indexterm>
           <b>Purpose:</b> Returns the absolute value of the argument.
           <p rev="2.0.1" conref="../shared/impala_common.xml#common/return_type_same"/>
           <p>
@@ -96,7 +96,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">acos() function</indexterm>
+          <indexterm audience="hidden">acos() function</indexterm>
           <b>Purpose:</b> Returns the arccosine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -112,7 +112,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">asin() function</indexterm>
+          <indexterm audience="hidden">asin() function</indexterm>
           <b>Purpose:</b> Returns the arcsine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -128,7 +128,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">atan() function</indexterm>
+          <indexterm audience="hidden">atan() function</indexterm>
           <b>Purpose:</b> Returns the arctangent of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -144,7 +144,7 @@ under the License.
         </dt>
 
         <dd rev="2.3.0 IMPALA-1771">
-          <indexterm audience="Cloudera">atan2() function</indexterm>
+          <indexterm audience="hidden">atan2() function</indexterm>
           <b>Purpose:</b> Returns the arctangent of the two arguments, with the signs of the arguments used to determine the
           quadrant of the result.
           <p>
@@ -161,7 +161,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">bin() function</indexterm>
+          <indexterm audience="hidden">bin() function</indexterm>
           <b>Purpose:</b> Returns the binary representation of an integer value, that is, a string of 0 and 1
           digits.
           <p>
@@ -183,7 +183,7 @@ under the License.
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">ceil() function</indexterm>
+          <indexterm audience="hidden">ceil() function</indexterm>
           <b>Purpose:</b> Returns the smallest integer that is greater than or equal to the argument.
           <p>
             <b>Return type:</b> <codeph>int</codeph> or <codeph>decimal(p,s)</codeph> depending on the type of the
@@ -201,7 +201,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">conv() function</indexterm>
+          <indexterm audience="hidden">conv() function</indexterm>
           <b>Purpose:</b> Returns a string representation of an integer value in a particular base. The input value
           can be a string, for example to convert a hexadecimal number such as <codeph>fce2</codeph> to decimal. To
           use the return value as a number (for example, when converting to base 10), use <codeph>CAST()</codeph>
@@ -220,7 +220,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">cos() function</indexterm>
+          <indexterm audience="hidden">cos() function</indexterm>
           <b>Purpose:</b> Returns the cosine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -236,7 +236,7 @@ under the License.
         </dt>
 
         <dd rev="2.3.0 IMPALA-1771">
-          <indexterm audience="Cloudera">cosh() function</indexterm>
+          <indexterm audience="hidden">cosh() function</indexterm>
           <b>Purpose:</b> Returns the hyperbolic cosine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -252,7 +252,7 @@ under the License.
         </dt>
 
         <dd rev="2.3.0 IMPALA-1771">
-          <indexterm audience="Cloudera">cot() function</indexterm>
+          <indexterm audience="hidden">cot() function</indexterm>
           <b>Purpose:</b> Returns the cotangent of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -269,7 +269,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">degrees() function</indexterm>
+          <indexterm audience="hidden">degrees() function</indexterm>
           <b>Purpose:</b> Converts argument value from radians to degrees.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -285,7 +285,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">e() function</indexterm>
+          <indexterm audience="hidden">e() function</indexterm>
           <b>Purpose:</b> Returns the
           <xref href="https://en.wikipedia.org/wiki/E_(mathematical_constant" scope="external" format="html">mathematical
           constant e</xref>.
@@ -304,7 +304,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">exp() function</indexterm>
+          <indexterm audience="hidden">exp() function</indexterm>
           <b>Purpose:</b> Returns the
           <xref href="https://en.wikipedia.org/wiki/E_(mathematical_constant" scope="external" format="html">mathematical
           constant e</xref> raised to the power of the argument.
@@ -321,7 +321,7 @@ under the License.
           <codeph>factorial(integer_type a)</codeph>
         </dt>
         <dd rev="2.3.0">
-          <indexterm audience="Cloudera">factorial() function</indexterm>
+          <indexterm audience="hidden">factorial() function</indexterm>
           <b>Purpose:</b> Computes the <xref href="https://en.wikipedia.org/wiki/Factorial" scope="external" format="html">factorial</xref> of an integer value.
           It works with any integer type.
           <p conref="../shared/impala_common.xml#common/added_in_230"/>
@@ -377,7 +377,7 @@ select factorial(-100);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">floor() function</indexterm>
+          <indexterm audience="hidden">floor() function</indexterm>
           <b>Purpose:</b> Returns the largest integer that is less than or equal to the argument.
           <p>
             <b>Return type:</b> <codeph>bigint</codeph> or <codeph>decimal(p,s)</codeph> depending on the type of
@@ -394,7 +394,7 @@ select factorial(-100);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">fmod() function</indexterm>
+          <indexterm audience="hidden">fmod() function</indexterm>
           <b>Purpose:</b> Returns the modulus of a floating-point number. Equivalent to the <codeph>%</codeph> arithmetic operator.
           <p>
             <b>Return type:</b> <codeph>float</codeph> or <codeph>double</codeph>, depending on type of arguments
@@ -477,7 +477,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.2.2">
-          <indexterm audience="Cloudera">fnv_hash() function</indexterm>
+          <indexterm audience="hidden">fnv_hash() function</indexterm>
           <b>Purpose:</b> Returns a consistent 64-bit value derived from the input argument, for convenience of
           implementing hashing logic in an application.
           <p>
@@ -567,7 +567,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">greatest() function</indexterm>
+          <indexterm audience="hidden">greatest() function</indexterm>
           <b>Purpose:</b> Returns the largest value from a list of expressions.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
         </dd>
@@ -581,7 +581,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">hex() function</indexterm>
+          <indexterm audience="hidden">hex() function</indexterm>
           <b>Purpose:</b> Returns the hexadecimal representation of an integer value, or of the characters in a
           string.
           <p>
@@ -598,7 +598,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">is_inf() function</indexterm>
+          <indexterm audience="hidden">is_inf() function</indexterm>
           <b>Purpose:</b> Tests whether a value is equal to the special value <q>inf</q>, signifying infinity.
           <p>
             <b>Return type:</b> <codeph>boolean</codeph>
@@ -616,7 +616,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">is_nan() function</indexterm>
+          <indexterm audience="hidden">is_nan() function</indexterm>
           <b>Purpose:</b> Tests whether a value is equal to the special value <q>NaN</q>, signifying <q>not a
           number</q>.
           <p>
@@ -637,7 +637,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">least() function</indexterm>
+          <indexterm audience="hidden">least() function</indexterm>
           <b>Purpose:</b> Returns the smallest value from a list of expressions.
           <p conref="../shared/impala_common.xml#common/return_same_type"/>
         </dd>
@@ -652,8 +652,8 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">ln() function</indexterm>
-          <indexterm audience="Cloudera">dlog1() function</indexterm>
+          <indexterm audience="hidden">ln() function</indexterm>
+          <indexterm audience="hidden">dlog1() function</indexterm>
           <b>Purpose:</b> Returns the
           <xref href="https://en.wikipedia.org/wiki/Natural_logarithm" scope="external" format="html">natural
           logarithm</xref> of the argument.
@@ -671,7 +671,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">log() function</indexterm>
+          <indexterm audience="hidden">log() function</indexterm>
           <b>Purpose:</b> Returns the logarithm of the second argument to the specified base.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -688,8 +688,8 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">log10() function</indexterm>
-          <indexterm audience="Cloudera">dlog10() function</indexterm>
+          <indexterm audience="hidden">log10() function</indexterm>
+          <indexterm audience="hidden">dlog10() function</indexterm>
           <b>Purpose:</b> Returns the logarithm of the argument to the base 10.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -705,7 +705,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">log2() function</indexterm>
+          <indexterm audience="hidden">log2() function</indexterm>
           <b>Purpose:</b> Returns the logarithm of the argument to the base 2.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -722,10 +722,10 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">max_int() function</indexterm>
-          <indexterm audience="Cloudera">max_tinyint() function</indexterm>
-          <indexterm audience="Cloudera">max_smallint() function</indexterm>
-          <indexterm audience="Cloudera">max_bigint() function</indexterm>
+          <indexterm audience="hidden">max_int() function</indexterm>
+          <indexterm audience="hidden">max_tinyint() function</indexterm>
+          <indexterm audience="hidden">max_smallint() function</indexterm>
+          <indexterm audience="hidden">max_bigint() function</indexterm>
           <b>Purpose:</b> Returns the largest value of the associated integral type.
           <p>
             <b>Return type:</b> The same as the integral type being checked.
@@ -749,10 +749,10 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">min_int() function</indexterm>
-          <indexterm audience="Cloudera">min_tinyint() function</indexterm>
-          <indexterm audience="Cloudera">min_smallint() function</indexterm>
-          <indexterm audience="Cloudera">min_bigint() function</indexterm>
+          <indexterm audience="hidden">min_int() function</indexterm>
+          <indexterm audience="hidden">min_tinyint() function</indexterm>
+          <indexterm audience="hidden">min_smallint() function</indexterm>
+          <indexterm audience="hidden">min_bigint() function</indexterm>
           <b>Purpose:</b> Returns the smallest value of the associated integral type (a negative number).
           <p>
             <b>Return type:</b> The same as the integral type being checked.
@@ -774,7 +774,7 @@ select fmod(9.9,3.3);
         </dt>
 
         <dd rev="2.2.0">
-          <indexterm audience="Cloudera">mod() function</indexterm>
+          <indexterm audience="hidden">mod() function</indexterm>
           <b>Purpose:</b> Returns the modulus of a number. Equivalent to the <codeph>%</codeph> arithmetic operator.
           Works with any size integer type, any size floating-point type, and <codeph>DECIMAL</codeph>
           with any precision and scale.
@@ -867,7 +867,7 @@ select mod(9.9,3.0);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">negative() function</indexterm>
+          <indexterm audience="hidden">negative() function</indexterm>
           <b>Purpose:</b> Returns the argument with the sign reversed; returns a positive value if the argument was
           already negative.
           <p rev="2.0.1" conref="../shared/impala_common.xml#common/return_type_same"/>
@@ -892,7 +892,7 @@ select mod(9.9,3.0);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">pi() function</indexterm>
+          <indexterm audience="hidden">pi() function</indexterm>
           <b>Purpose:</b> Returns the constant pi.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -908,7 +908,7 @@ select mod(9.9,3.0);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">pmod() function</indexterm>
+          <indexterm audience="hidden">pmod() function</indexterm>
           <b>Purpose:</b> Returns the positive modulus of a number.
           Primarily for <xref href="https://issues.apache.org/jira/browse/HIVE-656" scope="external" format="html">HiveQL compatibility</xref>.
           <p>
@@ -974,7 +974,7 @@ select pmod(5,-2);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">positive() function</indexterm>
+          <indexterm audience="hidden">positive() function</indexterm>
           <b>Purpose:</b> Returns the original argument unchanged (even if the argument is negative).
           <p rev="2.0.1" conref="../shared/impala_common.xml#common/return_type_same"/>
 <!--
@@ -1001,10 +1001,10 @@ select pmod(5,-2);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">pow() function</indexterm>
-          <indexterm audience="Cloudera">power() function</indexterm>
-          <indexterm audience="Cloudera">dpow() function</indexterm>
-          <indexterm audience="Cloudera">fpow() function</indexterm>
+          <indexterm audience="hidden">pow() function</indexterm>
+          <indexterm audience="hidden">power() function</indexterm>
+          <indexterm audience="hidden">dpow() function</indexterm>
+          <indexterm audience="hidden">fpow() function</indexterm>
           <b>Purpose:</b> Returns the first argument raised to the power of the second argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1020,7 +1020,7 @@ select pmod(5,-2);
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">precision() function</indexterm>
+          <indexterm audience="hidden">precision() function</indexterm>
           <b>Purpose:</b> Computes the precision (number of decimal digits) needed to represent the type of the
           argument expression as a <codeph>DECIMAL</codeph> value.
           <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>
@@ -1045,7 +1045,7 @@ select pmod(5,-2);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">quotient() function</indexterm>
+          <indexterm audience="hidden">quotient() function</indexterm>
           <b>Purpose:</b> Returns the first argument divided by the second argument, discarding any fractional
           part. Avoids promoting arguments to <codeph>DOUBLE</codeph> as happens with the <codeph>/</codeph> SQL
           operator.
@@ -1063,7 +1063,7 @@ select pmod(5,-2);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">radians() function</indexterm>
+          <indexterm audience="hidden">radians() function</indexterm>
           <b>Purpose:</b> Converts argument value from degrees to radians.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1081,7 +1081,7 @@ select pmod(5,-2);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">rand() function</indexterm>
+          <indexterm audience="hidden">rand() function</indexterm>
           <b>Purpose:</b> Returns a random value between 0 and 1. After <codeph>rand()</codeph> is called with a
           seed argument, it produces a consistent random sequence based on the seed value.
           <p>
@@ -1179,8 +1179,8 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">round() function</indexterm>
-          <indexterm audience="Cloudera">dround() function</indexterm>
+          <indexterm audience="hidden">round() function</indexterm>
+          <indexterm audience="hidden">dround() function</indexterm>
           <b>Purpose:</b> Rounds a floating-point value. By default (with a single argument), rounds to the nearest
           integer. Values ending in .5 are rounded up for positive numbers, down for negative numbers (that is,
           away from zero). The optional second argument specifies how many digits to leave after the decimal point;
@@ -1204,7 +1204,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd rev="1.4.0">
-          <indexterm audience="Cloudera">scale() function</indexterm>
+          <indexterm audience="hidden">scale() function</indexterm>
           <b>Purpose:</b> Computes the scale (number of decimal digits to the right of the decimal point) needed to
           represent the type of the argument expression as a <codeph>DECIMAL</codeph> value.
           <p conref="../shared/impala_common.xml#common/usage_notes_blurb"/>
@@ -1229,7 +1229,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">sign() function</indexterm>
+          <indexterm audience="hidden">sign() function</indexterm>
           <b>Purpose:</b> Returns -1, 0, or 1 to indicate the signedness of the argument value.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -1245,7 +1245,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">sin() function</indexterm>
+          <indexterm audience="hidden">sin() function</indexterm>
           <b>Purpose:</b> Returns the sine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1261,7 +1261,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd rev="2.3.0 IMPALA-1771">
-          <indexterm audience="Cloudera">sinh() function</indexterm>
+          <indexterm audience="hidden">sinh() function</indexterm>
           <b>Purpose:</b> Returns the hyperbolic sine of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1278,8 +1278,8 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">sqrt() function</indexterm>
-          <indexterm audience="Cloudera">dsqrt() function</indexterm>
+          <indexterm audience="hidden">sqrt() function</indexterm>
+          <indexterm audience="hidden">dsqrt() function</indexterm>
           <b>Purpose:</b> Returns the square root of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1295,7 +1295,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">tan() function</indexterm>
+          <indexterm audience="hidden">tan() function</indexterm>
           <b>Purpose:</b> Returns the tangent of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1311,7 +1311,7 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd rev="2.3.0 IMPALA-1771">
-          <indexterm audience="Cloudera">tanh() function</indexterm>
+          <indexterm audience="hidden">tanh() function</indexterm>
           <b>Purpose:</b> Returns the hyperbolic tangent of the argument.
           <p>
             <b>Return type:</b> <codeph>double</codeph>
@@ -1328,8 +1328,8 @@ select x, unix_timestamp(now()), rand(unix_timestamp(now()))
         </dt>
 
         <dd rev="2.3.0">
-          <indexterm audience="Cloudera">truncate() function</indexterm>
-          <indexterm audience="Cloudera">dtrunc() function</indexterm>
+          <indexterm audience="hidden">truncate() function</indexterm>
+          <indexterm audience="hidden">dtrunc() function</indexterm>
           <b>Purpose:</b> Removes some or all fractional digits from a numeric value.
           With no argument, removes all fractional digits, leaving an integer value.
           The optional argument specifies the number of fractional digits to include
@@ -1393,7 +1393,7 @@ select truncate(3.456,7)
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">unhex() function</indexterm>
+          <indexterm audience="hidden">unhex() function</indexterm>
           <b>Purpose:</b> Returns a string of characters with ASCII values corresponding to pairs of hexadecimal
           digits in the argument.
           <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_max.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_max.xml b/docs/topics/impala_max.xml
index 1e291f7..5cebf0d 100644
--- a/docs/topics/impala_max.xml
+++ b/docs/topics/impala_max.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">max() function</indexterm>
+      <indexterm audience="hidden">max() function</indexterm>
       An aggregate function that returns the maximum value from a set of numbers. Opposite of the
       <codeph>MIN</codeph> function. Its single argument can be numeric column, or the numeric result of a function
       or expression applied to the column value. Rows with a <codeph>NULL</codeph> value for the specified column

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_max_block_mgr_memory.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_max_block_mgr_memory.xml b/docs/topics/impala_max_block_mgr_memory.xml
index 2bf4061..c66b89c 100644
--- a/docs/topics/impala_max_block_mgr_memory.xml
+++ b/docs/topics/impala_max_block_mgr_memory.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p rev="2.1.0">
-      <indexterm audience="Cloudera">MAX_BLOCK_MGR_MEMORY query option</indexterm>
+      <indexterm audience="hidden">MAX_BLOCK_MGR_MEMORY query option</indexterm>
     </p>
 
     <p></p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_max_errors.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_max_errors.xml b/docs/topics/impala_max_errors.xml
index cb70bc9..63bae5b 100644
--- a/docs/topics/impala_max_errors.xml
+++ b/docs/topics/impala_max_errors.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">MAX_ERRORS query option</indexterm>
+      <indexterm audience="hidden">MAX_ERRORS query option</indexterm>
       Maximum number of non-fatal errors for any particular query that are recorded in the Impala log file. For
       example, if a billion-row table had a non-fatal data error in every row, you could diagnose the problem
       without all billion errors being logged. Unspecified or 0 indicates the built-in default value of 1000.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_max_num_runtime_filters.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_max_num_runtime_filters.xml b/docs/topics/impala_max_num_runtime_filters.xml
index c50fadf..7ac06ee 100644
--- a/docs/topics/impala_max_num_runtime_filters.xml
+++ b/docs/topics/impala_max_num_runtime_filters.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">MAX_NUM_RUNTIME_FILTERS query option</indexterm>
+      <indexterm audience="hidden">MAX_NUM_RUNTIME_FILTERS query option</indexterm>
       The <codeph>MAX_NUM_RUNTIME_FILTERS</codeph> query option
       sets an upper limit on the number of runtime filters that can be produced for each query.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_max_scan_range_length.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_max_scan_range_length.xml b/docs/topics/impala_max_scan_range_length.xml
index a828e45..3fbd4ca 100644
--- a/docs/topics/impala_max_scan_range_length.xml
+++ b/docs/topics/impala_max_scan_range_length.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">MAX_SCAN_RANGE_LENGTH query option</indexterm>
+      <indexterm audience="hidden">MAX_SCAN_RANGE_LENGTH query option</indexterm>
       Maximum length of the scan range. Interacts with the number of HDFS blocks in the table to determine how many
       CPU cores across the cluster are involved with the processing for a query. (Each core processes one scan
       range.)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_mem_limit.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_mem_limit.xml b/docs/topics/impala_mem_limit.xml
index ca0f2aa..98bc470 100644
--- a/docs/topics/impala_mem_limit.xml
+++ b/docs/topics/impala_mem_limit.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">MEM_LIMIT query option</indexterm>
+      <indexterm audience="hidden">MEM_LIMIT query option</indexterm>
       When resource management is not enabled, defines the maximum amount of memory a query can allocate on each node.
       Therefore, the total memory that can be used by a query is the <codeph>MEM_LIMIT</codeph> times the number of nodes.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_min.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_min.xml b/docs/topics/impala_min.xml
index 8c242e3..f85ac62 100644
--- a/docs/topics/impala_min.xml
+++ b/docs/topics/impala_min.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">min() function</indexterm>
+      <indexterm audience="hidden">min() function</indexterm>
       An aggregate function that returns the minimum value from a set of numbers. Opposite of the
       <codeph>MAX</codeph> function. Its single argument can be numeric column, or the numeric result of a function
       or expression applied to the column value. Rows with a <codeph>NULL</codeph> value for the specified column

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_misc_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_misc_functions.xml b/docs/topics/impala_misc_functions.xml
index 15ecaa8..c5ca9eb 100644
--- a/docs/topics/impala_misc_functions.xml
+++ b/docs/topics/impala_misc_functions.xml
@@ -47,7 +47,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">current_database() function</indexterm>
+          <indexterm audience="hidden">current_database() function</indexterm>
           <b>Purpose:</b> Returns the database that the session is currently using, either <codeph>default</codeph>
           if no database has been selected, or whatever database the session switched to through a
           <codeph>USE</codeph> statement or the <cmdname>impalad</cmdname><codeph>-d</codeph> option.
@@ -65,7 +65,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">effective_user() function</indexterm>
+          <indexterm audience="hidden">effective_user() function</indexterm>
           <b>Purpose:</b> Typically returns the same value as <codeph>user()</codeph>,
           except if delegation is enabled, in which case it returns the ID of the delegated user.
           <p>
@@ -85,7 +85,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">pid() function</indexterm>
+          <indexterm audience="hidden">pid() function</indexterm>
           <b>Purpose:</b> Returns the process ID of the <cmdname>impalad</cmdname> daemon that the session is
           connected to. You can use it during low-level debugging, to issue Linux commands that trace, show the
           arguments, and so on the <cmdname>impalad</cmdname> process.
@@ -96,14 +96,14 @@ under the License.
 
       </dlentry>
 
-      <dlentry audience="Cloudera" id="sleep">
+      <dlentry audience="hidden" id="sleep">
 
         <dt>
           <codeph>sleep(int ms)</codeph>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">sleep() function</indexterm>
+          <indexterm audience="hidden">sleep() function</indexterm>
           <b>Purpose:</b> Pauses the query for a specified number of milliseconds. For slowing down queries with
           small result sets enough to monitor runtime execution, memory usage, or other factors that otherwise
           would be difficult to capture during the brief interval of query execution. When used in the
@@ -126,7 +126,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">user() function</indexterm>
+          <indexterm audience="hidden">user() function</indexterm>
           <b>Purpose:</b> Returns the username of the Linux user who is connected to the <cmdname>impalad</cmdname>
           daemon. Typically called a single time, in a query without any <codeph>FROM</codeph> clause, to
           understand how authorization settings apply in a security context; once you know the logged-in username,
@@ -150,7 +150,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">uuid() function</indexterm>
+          <indexterm audience="hidden">uuid() function</indexterm>
           <b>Purpose:</b> Returns a <xref href="https://en.wikipedia.org/wiki/Universally_unique_identifier" scope="external" format="html">universal unique identifier</xref>, a 128-bit value encoded as a string with groups of hexadecimal digits separated by dashes.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -198,7 +198,7 @@ select uuid() from four_row_table;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">version() function</indexterm>
+          <indexterm audience="hidden">version() function</indexterm>
           <b>Purpose:</b> Returns information such as the precise version number and build date for the
           <codeph>impalad</codeph> daemon that you are currently connected to. Typically used to confirm that you
           are connected to the expected level of Impala to use a particular feature, or to connect to several nodes

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_ndv.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_ndv.xml b/docs/topics/impala_ndv.xml
index f1bff69..0cc5443 100644
--- a/docs/topics/impala_ndv.xml
+++ b/docs/topics/impala_ndv.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">ndv() function</indexterm>
+      <indexterm audience="hidden">ndv() function</indexterm>
       An aggregate function that returns an approximate value similar to the result of <codeph>COUNT(DISTINCT
       <varname>col</varname>)</codeph>, the <q>number of distinct values</q>. It is much faster than the
       combination of <codeph>COUNT</codeph> and <codeph>DISTINCT</codeph>, and uses a constant amount of memory and

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_new_features.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_new_features.xml b/docs/topics/impala_new_features.xml
index 8c532d0..5c8405d 100644
--- a/docs/topics/impala_new_features.xml
+++ b/docs/topics/impala_new_features.xml
@@ -95,7 +95,7 @@ under the License.
             </li>
           </ul>
         </li>
-        <li audience="Cloudera">
+        <li audience="hidden">
           <p rev="IMPALA-3210 CDH-43736"><!-- Patch didn't make it into in <keyword keyref="impala27_full"/> -->
             [<xref href="https://issues.cloudera.org/browse/IMPALA-3210" scope="external" format="html">IMPALA-3210</xref>]
             The analytic functions <codeph>FIRST_VALUE()</codeph> and <codeph>LAST_VALUE()</codeph>
@@ -171,7 +171,7 @@ under the License.
             column definitions in Avro tables.
           </p>
         </li>
-        <li audience="Cloudera"><!-- Patch didn't make it into in <keyword keyref="impala27_full"/> -->
+        <li audience="hidden"><!-- Patch didn't make it into in <keyword keyref="impala27_full"/> -->
           <p rev="IMPALA-1654 CDH-43747">
             [<xref href="https://issues.cloudera.org/browse/IMPALA-1654" scope="external" format="html">IMPALA-1654</xref>]
             Several kinds of DDL operations
@@ -934,21 +934,21 @@ under the License.
             <ph audience="PDF">See <xref href="impala_optimize_partition_key_scans.xml"/> for details.</ph>
           </p>
         </li>
-        <li audience="Cloudera"><!-- All the other undocumented query options are not really new features for this release, so hiding this whole bullet. -->
+        <li audience="hidden"><!-- All the other undocumented query options are not really new features for this release, so hiding this whole bullet. -->
           <p>
             Other new query options:
           </p>
           <ul>
-            <li audience="Cloudera"><!-- Actually from a long way back, just never documented. Not sure if appropriate to keep internal-only or expose. -->
+            <li audience="hidden"><!-- Actually from a long way back, just never documented. Not sure if appropriate to keep internal-only or expose. -->
               <codeph>DISABLE_OUTERMOST_TOPN</codeph>
             </li>
-            <li audience="Cloudera"><!-- Actually from a long way back, just never documented. Not sure if appropriate to keep internal-only or expose. -->
+            <li audience="hidden"><!-- Actually from a long way back, just never documented. Not sure if appropriate to keep internal-only or expose. -->
               <codeph>RM_INITIAL_MEM</codeph>
             </li>
-            <li audience="Cloudera"><!-- Seems to be related to writing sequence files, a capability not externalized at this time. -->
+            <li audience="hidden"><!-- Seems to be related to writing sequence files, a capability not externalized at this time. -->
               <codeph>SEQ_COMPRESSION_MODE</codeph>
             </li>
-            <li audience="Cloudera"><!-- Actually, was only used for working around one JIRA. Being deprecated now in Impala 2.3 via IMPALA-2963. -->
+            <li audience="hidden"><!-- Actually, was only used for working around one JIRA. Being deprecated now in Impala 2.3 via IMPALA-2963. -->
               <codeph>DISABLE_CACHED_READS</codeph>
             </li>
           </ul>
@@ -1444,7 +1444,7 @@ under the License.
 
   </concept>
 
-  <concept audience="Cloudera" rev="5.4.6" id="new_features_226">
+  <concept audience="hidden" rev="5.4.6" id="new_features_226">
 
     <title>New Features in Impala 2.2.6 / CDH 5.4.6</title>
 
@@ -1500,7 +1500,7 @@ under the License.
 
 <!-- I let the 5.4.3/5.4.3 subtopic above remain in existence, but now back to hiding specific 5.4.x subtopics
      after the .0 one that has the actual new features.
-  <concept audience="Cloudera" rev="5.4.2" id="new_features_222">
+  <concept audience="hidden" rev="5.4.2" id="new_features_222">
 
     <title>New Features in Impala 2.2.x for CDH 5.4.2</title>
 
@@ -2130,7 +2130,7 @@ under the License.
               values are padded with spaces on the right. See <xref href="impala_char.xml#char"/> for details.
             </li>
 
-            <li audience="Cloudera">
+            <li audience="hidden">
 <!-- This feature will be undocumented in Impala 2.0, probably ready for prime time in 2.1. -->
               <codeph>DATE</codeph>. See <xref href="impala_date.xml#date"/> for details.
             </li>
@@ -2311,7 +2311,7 @@ under the License.
           </p>
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
 <!-- This feature will be undocumented in Impala 2.0, probably ready for prime time in 2.1. -->
           <p>
             Improved file format support. Impala can now write to Avro, compressed text, SequenceFile, and RCFile
@@ -2338,7 +2338,7 @@ under the License.
           </p>
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
 <!-- This feature will be undocumented in Impala 2.0, probably ready for prime time in 2.1. -->
           <p>
             The <codeph>COMPUTE STATS</codeph> statement can now gather statistics for newly added partitions
@@ -2579,7 +2579,7 @@ under the License.
           </p>
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
 <!-- Not documenting for 1.4. Revisit in a future release. -->
           <p>
             Data sources. <ph audience="PDF">See <xref href="impala_data_sources.xml#data_sources"/> for
@@ -2630,7 +2630,7 @@ under the License.
           <ul>
 <!-- This particular change has been pushed out to a later release. -->
 
-            <li audience="Cloudera">
+            <li audience="hidden">
               Certain simple aggregation operations (with no <codeph>GROUP BY</codeph> step) are multi-threaded if
               spare cores are available.
             </li>
@@ -3266,7 +3266,7 @@ under the License.
           </p>
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
           <p>
             Integration with the YARN resource management framework. Only available in combination with CDH 5. This
             feature makes use of the underlying YARN service, plus an additional service (Llama) that coordinates

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_num_nodes.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_num_nodes.xml b/docs/topics/impala_num_nodes.xml
index 44c1eb8..f885124 100644
--- a/docs/topics/impala_num_nodes.xml
+++ b/docs/topics/impala_num_nodes.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">NUM_NODES query option</indexterm>
+      <indexterm audience="hidden">NUM_NODES query option</indexterm>
       Limit the number of nodes that process a query, typically during debugging.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_num_scanner_threads.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_num_scanner_threads.xml b/docs/topics/impala_num_scanner_threads.xml
index c12f672..b7a47ae 100644
--- a/docs/topics/impala_num_scanner_threads.xml
+++ b/docs/topics/impala_num_scanner_threads.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">NUM_SCANNER_THREADS query option</indexterm>
+      <indexterm audience="hidden">NUM_SCANNER_THREADS query option</indexterm>
       Maximum number of scanner threads (on each node) used for each query. By default, Impala uses as many cores
       as are available (one thread per core). You might lower this value if queries are using excessive resources
       on a busy cluster. Impala imposes a maximum value automatically, so a high value has no practical effect.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_odbc.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_odbc.xml b/docs/topics/impala_odbc.xml
index c819a6d..402bc11 100644
--- a/docs/topics/impala_odbc.xml
+++ b/docs/topics/impala_odbc.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">ODBC</indexterm>
+      <indexterm audience="hidden">ODBC</indexterm>
       Third-party products can be designed to integrate with Impala using ODBC. For the best experience, ensure any
       third-party product you intend to use is supported. Verifying support includes checking that the versions of
       Impala, ODBC, the operating system, and the third-party product have all been approved for use together.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_operators.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_operators.xml b/docs/topics/impala_operators.xml
index dd916ff..2a944ce 100644
--- a/docs/topics/impala_operators.xml
+++ b/docs/topics/impala_operators.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">operators</indexterm>
+      <indexterm audience="hidden">operators</indexterm>
       SQL operators are a class of comparison functions that are widely used within the <codeph>WHERE</codeph> clauses of
       <codeph>SELECT</codeph> statements.
     </p>
@@ -50,7 +50,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">arithmetic operators</indexterm>
+        <indexterm audience="hidden">arithmetic operators</indexterm>
         The arithmetic operators use expressions with a left-hand argument, the operator, and then (in most cases) a right-hand argument.
       </p>
 
@@ -195,7 +195,7 @@ where nation.item.n_nationkey &lt; 5;
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">BETWEEN operator</indexterm>
+        <indexterm audience="hidden">BETWEEN operator</indexterm>
         In a <codeph>WHERE</codeph> clause, compares an expression to both a lower and upper bound. The comparison is successful is the
         expression is greater than or equal to the lower bound, and less than or equal to the upper bound. If the bound values are switched,
         so the lower bound is greater than the upper bound, does not match any values.
@@ -287,7 +287,7 @@ where nation.item.n_nationkey between 3 and 5
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">comparison operators</indexterm>
+        <indexterm audience="hidden">comparison operators</indexterm>
         Impala supports the familiar comparison operators for checking equality and sort order for the column data types:
       </p>
 
@@ -368,14 +368,14 @@ where nation.item.n_nationkey &lt; 5
 
   </concept>
 
-  <concept audience="Cloudera" rev="2.1.0" id="except">
+  <concept audience="hidden" rev="2.1.0" id="except">
 
     <title>EXCEPT Operator</title>
 
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">EXCEPT operator</indexterm>
+        <indexterm audience="hidden">EXCEPT operator</indexterm>
       </p>
 
     </conbody>
@@ -389,9 +389,9 @@ where nation.item.n_nationkey &lt; 5
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">EXISTS operator</indexterm>
+        <indexterm audience="hidden">EXISTS operator</indexterm>
 
-        <indexterm audience="Cloudera">NOT EXISTS operator</indexterm>
+        <indexterm audience="hidden">NOT EXISTS operator</indexterm>
         The <codeph>EXISTS</codeph> operator tests whether a subquery returns any results. You typically use it to find values from one
         table that have corresponding values in another table.
       </p>
@@ -651,7 +651,7 @@ ERROR: AnalysisException: couldn't resolve column reference: 'x'
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">ILIKE operator</indexterm>
+        <indexterm audience="hidden">ILIKE operator</indexterm>
         A case-insensitive comparison operator for <codeph>STRING</codeph> data, with basic wildcard capability using <codeph>_</codeph> to match a single
         character and <codeph>%</codeph> to match multiple characters. The argument expression must match the entire string value.
         Typically, it is more efficient to put any <codeph>%</codeph> wildcard match at the end of the string.
@@ -743,9 +743,9 @@ select 'ABCXYZ' not like 'ab_xyz';
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">IN operator</indexterm>
+        <indexterm audience="hidden">IN operator</indexterm>
 
-        <indexterm audience="Cloudera">NOT IN operator</indexterm>
+        <indexterm audience="hidden">NOT IN operator</indexterm>
         The <codeph>IN</codeph> operator compares an argument value to a set of values, and returns <codeph>TRUE</codeph> if the argument
         matches any value in the set. The <codeph>NOT IN</codeph> operator reverses the comparison, and checks if the argument value is not
         part of a set of values.
@@ -940,14 +940,14 @@ SELECT COUNT(DISTINCT(visitor_id)) FROM web_traffic WHERE month IN ('January','J
 
   </concept>
 
-  <concept audience="Cloudera" rev="2.1.0" id="intersect">
+  <concept audience="hidden" rev="2.1.0" id="intersect">
 
     <title>INTERSECT Operator</title>
 
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">INTERSECT operator</indexterm>
+        <indexterm audience="hidden">INTERSECT operator</indexterm>
       </p>
 
     </conbody>
@@ -961,7 +961,7 @@ SELECT COUNT(DISTINCT(visitor_id)) FROM web_traffic WHERE month IN ('January','J
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">IREGEXP operator</indexterm>
+        <indexterm audience="hidden">IREGEXP operator</indexterm>
         Tests whether a value matches a regular expression, using case-insensitive string comparisons.
         Uses the POSIX regular expression syntax where <codeph>^</codeph> and
         <codeph>$</codeph> match the beginning and end of the string, <codeph>.</codeph> represents any single character, <codeph>*</codeph>
@@ -1042,9 +1042,9 @@ SELECT COUNT(DISTINCT(visitor_id)) FROM web_traffic WHERE month IN ('January','J
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">IS DISTINCT FROM operator</indexterm>
+        <indexterm audience="hidden">IS DISTINCT FROM operator</indexterm>
 
-        <indexterm audience="Cloudera">IS NOT DISTINCT FROM operator</indexterm>
+        <indexterm audience="hidden">IS NOT DISTINCT FROM operator</indexterm>
         The <codeph>IS DISTINCT FROM</codeph> operator, and its converse the <codeph>IS NOT DISTINCT FROM</codeph> operator, test whether or
         not values are identical. <codeph>IS NOT DISTINCT FROM</codeph> is similar to the <codeph>=</codeph> operator, and <codeph>IS
         DISTINCT FROM</codeph> is similar to the <codeph>!=</codeph> operator, except that <codeph>NULL</codeph> values are treated as
@@ -1185,9 +1185,9 @@ select
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">IS NULL operator</indexterm>
+        <indexterm audience="hidden">IS NULL operator</indexterm>
 
-        <indexterm audience="Cloudera">IS NOT NULL operator</indexterm>
+        <indexterm audience="hidden">IS NOT NULL operator</indexterm>
         The <codeph>IS NULL</codeph> operator, and its converse the <codeph>IS NOT NULL</codeph> operator, test whether a specified value is
         <codeph><xref href="impala_literals.xml#null">NULL</xref></codeph>. Because using <codeph>NULL</codeph> with any of the other
         comparison operators such as <codeph>=</codeph> or <codeph>!=</codeph> also returns <codeph>NULL</codeph> rather than
@@ -1252,7 +1252,7 @@ select count(*) from web_traffic where weird_http_code is not null;</codeblock>
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">LIKE operator</indexterm>
+        <indexterm audience="hidden">LIKE operator</indexterm>
         A comparison operator for <codeph>STRING</codeph> data, with basic wildcard capability using the underscore
         (<codeph>_</codeph>) to match a single character and the percent sign (<codeph>%</codeph>) to match multiple
         characters. The argument expression must match the entire string value.
@@ -1299,7 +1299,7 @@ select distinct c_last_name from customer where c_last_name like 'M___';</codebl
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">logical operators</indexterm>
+        <indexterm audience="hidden">logical operators</indexterm>
         Logical operators return a <codeph>BOOLEAN</codeph> value, based on a binary or unary logical operation between arguments that are
         also Booleans. Typically, the argument expressions use <xref href="impala_operators.xml#comparison_operators">comparison
         operators</xref>.
@@ -1521,7 +1521,7 @@ where
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">REGEXP operator</indexterm>
+        <indexterm audience="hidden">REGEXP operator</indexterm>
         Tests whether a value matches a regular expression. Uses the POSIX regular expression syntax where <codeph>^</codeph> and
         <codeph>$</codeph> match the beginning and end of the string, <codeph>.</codeph> represents any single character, <codeph>*</codeph>
         represents a sequence of zero or more items, <codeph>+</codeph> represents a sequence of one or more items, <codeph>?</codeph>
@@ -1589,7 +1589,7 @@ where
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">RLIKE operator</indexterm>
+        <indexterm audience="hidden">RLIKE operator</indexterm>
         Synonym for the <codeph>REGEXP</codeph> operator. See <xref href="impala_operators.xml#regexp"/> for details.
       </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_optimize_partition_key_scans.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_optimize_partition_key_scans.xml b/docs/topics/impala_optimize_partition_key_scans.xml
index afdd376..070f359 100644
--- a/docs/topics/impala_optimize_partition_key_scans.xml
+++ b/docs/topics/impala_optimize_partition_key_scans.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0 IMPALA-2499">
-      <indexterm audience="Cloudera">OPTIMIZE_PARTITION_KEY_SCANS query option</indexterm>
+      <indexterm audience="hidden">OPTIMIZE_PARTITION_KEY_SCANS query option</indexterm>
       Enables a fast code path for queries that apply simple aggregate functions to partition key
       columns: <codeph>MIN(<varname>key_column</varname>)</codeph>, <codeph>MAX(<varname>key_column</varname>)</codeph>,
       or <codeph>COUNT(DISTINCT <varname>key_column</varname>)</codeph>.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_parquet.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_parquet.xml b/docs/topics/impala_parquet.xml
index 0a34413..0a0f4f5 100644
--- a/docs/topics/impala_parquet.xml
+++ b/docs/topics/impala_parquet.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">Parquet support in Impala</indexterm>
+      <indexterm audience="hidden">Parquet support in Impala</indexterm>
       Impala helps you to create, manage, and query Parquet tables. Parquet is a column-oriented binary file format
       intended to be highly efficient for the types of large-scale queries that Impala is best at. Parquet is
       especially good for queries scanning particular columns within a table, for example to query <q>wide</q>
@@ -443,7 +443,7 @@ Add an example here.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">COMPRESSION_CODEC query option</indexterm>
+        <indexterm audience="hidden">COMPRESSION_CODEC query option</indexterm>
         When Impala writes Parquet data files using the <codeph>INSERT</codeph> statement, the underlying
         compression is controlled by the <codeph>COMPRESSION_CODEC</codeph> query option. (Prior to Impala 2.0, the
         query option name was <codeph>PARQUET_COMPRESSION_CODEC</codeph>.) The allowed values for this query option
@@ -461,7 +461,7 @@ Add an example here.
       <conbody>
 
         <p>
-          <indexterm audience="Cloudera">compression</indexterm>
+          <indexterm audience="hidden">compression</indexterm>
           By default, the underlying data files for a Parquet table are compressed with Snappy. The combination of
           fast compression and decompression makes it a good choice for many data sets. To ensure Snappy
           compression is used, for example after experimenting with other compression codecs, set the

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_parquet_annotate_strings_utf8.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_parquet_annotate_strings_utf8.xml b/docs/topics/impala_parquet_annotate_strings_utf8.xml
index a10f117..9437fed 100644
--- a/docs/topics/impala_parquet_annotate_strings_utf8.xml
+++ b/docs/topics/impala_parquet_annotate_strings_utf8.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-2069">
-      <indexterm audience="Cloudera">PARQUET_ANNOTATE_STRINGS_UTF8 query option</indexterm>
+      <indexterm audience="hidden">PARQUET_ANNOTATE_STRINGS_UTF8 query option</indexterm>
       Causes Impala <codeph>INSERT</codeph> and <codeph>CREATE TABLE AS SELECT</codeph> statements
       to write Parquet files that use the UTF-8 annotation for <codeph>STRING</codeph> columns.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_parquet_compression_codec.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_parquet_compression_codec.xml b/docs/topics/impala_parquet_compression_codec.xml
index 6f3c3cc..d1f146c 100644
--- a/docs/topics/impala_parquet_compression_codec.xml
+++ b/docs/topics/impala_parquet_compression_codec.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">PARQUET_COMPRESSION_CODEC query option</indexterm>
+      <indexterm audience="hidden">PARQUET_COMPRESSION_CODEC query option</indexterm>
       Deprecated. Use <codeph>COMPRESSION_CODEC</codeph> in Impala 2.0 and later. See
       <xref href="impala_compression_codec.xml#compression_codec"/> for details.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_parquet_fallback_schema_resolution.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_parquet_fallback_schema_resolution.xml b/docs/topics/impala_parquet_fallback_schema_resolution.xml
index 1603e61..826839d 100644
--- a/docs/topics/impala_parquet_fallback_schema_resolution.xml
+++ b/docs/topics/impala_parquet_fallback_schema_resolution.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-2835 CDH-33330">
-      <indexterm audience="Cloudera">PARQUET_FALLBACK_SCHEMA_RESOLUTION query option</indexterm>
+      <indexterm audience="hidden">PARQUET_FALLBACK_SCHEMA_RESOLUTION query option</indexterm>
       Allows Impala to look up columns within Parquet files by column name, rather than column order,
       when necessary.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_parquet_file_size.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_parquet_file_size.xml b/docs/topics/impala_parquet_file_size.xml
index 42d6645..2471feb 100644
--- a/docs/topics/impala_parquet_file_size.xml
+++ b/docs/topics/impala_parquet_file_size.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">PARQUET_FILE_SIZE query option</indexterm>
+      <indexterm audience="hidden">PARQUET_FILE_SIZE query option</indexterm>
       Specifies the maximum size of each Parquet data file produced by Impala <codeph>INSERT</codeph> statements.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_partitioning.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_partitioning.xml b/docs/topics/impala_partitioning.xml
index 31fdcfc..4d723ad 100644
--- a/docs/topics/impala_partitioning.xml
+++ b/docs/topics/impala_partitioning.xml
@@ -41,7 +41,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">partitioning</indexterm>
+      <indexterm audience="hidden">partitioning</indexterm>
       By default, all the data files for a table are located in a single directory. Partitioning is a technique for physically dividing the
       data during loading, based on values from one or more columns, to speed up queries that test those columns. For example, with a
       <codeph>school_records</codeph> table partitioned on a <codeph>year</codeph> column, there is a separate data directory for each
@@ -332,7 +332,7 @@ insert into weather <b>partition (year=2014, month=04, day)</b> select 'sunny',2
       <conbody>
 
         <p rev="1.2.2">
-          <indexterm audience="Cloudera">predicate propagation</indexterm>
+          <indexterm audience="hidden">predicate propagation</indexterm>
           Impala can even do partition pruning in cases where the partition key column is not directly compared to a constant, by applying
           the transitive property to other parts of the <codeph>WHERE</codeph> clause. This technique is known as predicate propagation, and
           is available in Impala 1.2.2 and later. In this example, the census table includes another column indicating when the data was
@@ -575,7 +575,7 @@ SELECT COUNT(*) FROM sales_table WHERE year IN (2005, 2010, 2015);
 
   </concept>
 
-  <concept rev="kudu" id="partition_kudu" audience="Cloudera">
+  <concept rev="kudu" id="partition_kudu" audience="hidden">
 
     <title>Using Partitioning with Kudu Tables</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_perf_hdfs_caching.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_perf_hdfs_caching.xml b/docs/topics/impala_perf_hdfs_caching.xml
index 017c436..79d700a 100644
--- a/docs/topics/impala_perf_hdfs_caching.xml
+++ b/docs/topics/impala_perf_hdfs_caching.xml
@@ -469,11 +469,11 @@ Found 122 entries
         </li>
       </ul>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         <b>Cloudera Manager:</b>
       </p>
 
-      <ul audience="Cloudera">
+      <ul audience="hidden">
         <li>
           You can enable or disable HDFS caching through Cloudera Manager, using the configuration setting
           <uicontrol>Maximum Memory Used for Caching</uicontrol> for the HDFS service. This control sets the HDFS

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_perf_stats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_perf_stats.xml b/docs/topics/impala_perf_stats.xml
index f0d1428..e08fd0b 100644
--- a/docs/topics/impala_perf_stats.xml
+++ b/docs/topics/impala_perf_stats.xml
@@ -744,7 +744,7 @@ EXPLAIN SELECT ... FROM t1 WHERE year BETWEEN 2006 AND 2009;
 
 <!-- Might deserve its own conceptual topic at some point. -->
 
-  <concept audience="Cloudera" rev="1.2.2" id="perf_stats_joins">
+  <concept audience="hidden" rev="1.2.2" id="perf_stats_joins">
 
     <title>How Statistics Are Used in Join Queries</title>
 
@@ -756,7 +756,7 @@ EXPLAIN SELECT ... FROM t1 WHERE year BETWEEN 2006 AND 2009;
 
 <!-- Might deserve its own conceptual topic at some point. -->
 
-  <concept audience="Cloudera" rev="1.2.2" id="perf_stats_inserts">
+  <concept audience="hidden" rev="1.2.2" id="perf_stats_inserts">
 
     <title>How Statistics Are Used in INSERT Operations</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_performance.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_performance.xml b/docs/topics/impala_performance.xml
index ac55bda..c0afc32 100644
--- a/docs/topics/impala_performance.xml
+++ b/docs/topics/impala_performance.xml
@@ -101,7 +101,7 @@ under the License.
         Impala can use.
       </li>
 
-      <li rev="1.2" audience="Cloudera">
+      <li rev="1.2" audience="hidden">
         <xref href="impala_perf_hdfs_caching.xml#hdfs_caching"/>. Impala can use the HDFS caching feature to pin
         frequently accessed data in memory, reducing disk I/O.
       </li>
@@ -120,28 +120,28 @@ under the License.
 
 <!-- Empty/hidden stub sections that might be worth expanding later. -->
 
-  <concept id="perf_network" audience="Cloudera">
+  <concept id="perf_network" audience="hidden">
 
     <title>Network Traffic</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_partition_schema" audience="Cloudera">
+  <concept id="perf_partition_schema" audience="hidden">
 
     <title>Designing Partitioned Tables</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_partition_query" audience="Cloudera">
+  <concept id="perf_partition_query" audience="hidden">
 
     <title>Queries on Partitioned Tables</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_monitoring" audience="Cloudera">
+  <concept id="perf_monitoring" audience="hidden">
 
     <title>Monitoring Performance through the Impala Web Interface</title>
   <prolog>
@@ -153,42 +153,42 @@ under the License.
     <conbody/>
   </concept>
 
-  <concept id="perf_query_coord" audience="Cloudera">
+  <concept id="perf_query_coord" audience="hidden">
 
     <title>Query Coordination</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_bottlenecks" audience="Cloudera">
+  <concept id="perf_bottlenecks" audience="hidden">
 
     <title>Performance Bottlenecks</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_long_queries" audience="Cloudera">
+  <concept id="perf_long_queries" audience="hidden">
 
     <title>Managing Long-Running Queries</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_load" audience="Cloudera">
+  <concept id="perf_load" audience="hidden">
 
     <title>Performance Considerations for Loading Data</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_file_formats" audience="Cloudera">
+  <concept id="perf_file_formats" audience="hidden">
 
     <title>Performance Considerations for File Formats</title>
 
     <conbody/>
   </concept>
 
-  <concept id="perf_compression" audience="Cloudera">
+  <concept id="perf_compression" audience="hidden">
 
     <title>Performance Considerations for Compression</title>
   <prolog>
@@ -200,7 +200,7 @@ under the License.
     <conbody/>
   </concept>
 
-  <concept id="perf_codegen" audience="Cloudera">
+  <concept id="perf_codegen" audience="hidden">
 
     <title>Native Code Generation</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_planning.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_planning.xml b/docs/topics/impala_planning.xml
index 306d141..2b8ea0e 100644
--- a/docs/topics/impala_planning.xml
+++ b/docs/topics/impala_planning.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">planning</indexterm>
+      <indexterm audience="hidden">planning</indexterm>
       Before you set up Impala in production, do some planning to make sure that your hardware setup has sufficient
       capacity, that your cluster topology is optimal for Impala queries, and that your schema design and ETL
       processes follow the best practices for Impala.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_porting.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_porting.xml b/docs/topics/impala_porting.xml
index 166691c..f9f2081 100644
--- a/docs/topics/impala_porting.xml
+++ b/docs/topics/impala_porting.xml
@@ -41,7 +41,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">porting</indexterm>
+      <indexterm audience="hidden">porting</indexterm>
       Although Impala uses standard SQL for queries, you might need to modify SQL source when bringing applications
       to Impala, due to variations in data types, built-in functions, vendor language extensions, and
       Hadoop-specific syntax. Even when SQL is working correctly, you might make further minor modifications for

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_ports.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_ports.xml b/docs/topics/impala_ports.xml
index 3c90a3f..11f8e1e 100644
--- a/docs/topics/impala_ports.xml
+++ b/docs/topics/impala_ports.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody id="conbody_ports">
 
     <p>
-      <indexterm audience="Cloudera">ports</indexterm>
+      <indexterm audience="hidden">ports</indexterm>
       Impala uses the TCP ports listed in the following table. Before deploying Impala, ensure these ports are open
       on each system.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_prefetch_mode.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_prefetch_mode.xml b/docs/topics/impala_prefetch_mode.xml
index 9c908fb..71521f8 100644
--- a/docs/topics/impala_prefetch_mode.xml
+++ b/docs/topics/impala_prefetch_mode.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-3286">
-      <indexterm audience="Cloudera">PREFETCH_MODE query option</indexterm>
+      <indexterm audience="hidden">PREFETCH_MODE query option</indexterm>
       Determines whether the prefetching optimization is applied during
       join query processing.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_prereqs.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_prereqs.xml b/docs/topics/impala_prereqs.xml
index f122ee7..47e54fa 100644
--- a/docs/topics/impala_prereqs.xml
+++ b/docs/topics/impala_prereqs.xml
@@ -43,8 +43,8 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">prerequisites</indexterm>
-      <indexterm audience="Cloudera">requirements</indexterm>
+      <indexterm audience="hidden">prerequisites</indexterm>
+      <indexterm audience="hidden">requirements</indexterm>
       To perform as expected, Impala depends on the availability of the software, hardware, and configurations
       described in the following sections.
     </p>
@@ -83,14 +83,14 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">software requirements</indexterm>
-        <indexterm audience="Cloudera">Red Hat Enterprise Linux</indexterm>
-        <indexterm audience="Cloudera">RHEL</indexterm>
-        <indexterm audience="Cloudera">CentOS</indexterm>
-        <indexterm audience="Cloudera">SLES</indexterm>
-        <indexterm audience="Cloudera">Ubuntu</indexterm>
-        <indexterm audience="Cloudera">SUSE</indexterm>
-        <indexterm audience="Cloudera">Debian</indexterm> The relevant supported operating systems
+        <indexterm audience="hidden">software requirements</indexterm>
+        <indexterm audience="hidden">Red Hat Enterprise Linux</indexterm>
+        <indexterm audience="hidden">RHEL</indexterm>
+        <indexterm audience="hidden">CentOS</indexterm>
+        <indexterm audience="hidden">SLES</indexterm>
+        <indexterm audience="hidden">Ubuntu</indexterm>
+        <indexterm audience="hidden">SUSE</indexterm>
+        <indexterm audience="hidden">Debian</indexterm> The relevant supported operating systems
         and versions for Impala are the same as for the corresponding CDH 5 platforms. For
         details, see the <cite>Supported Operating Systems</cite> page for
         <ph audience="integrated"><xref href="rn_consolidated_pcm.xml#cdh_cm_supported_os">CDH
@@ -113,9 +113,9 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">Hive</indexterm>
-        <indexterm audience="Cloudera">MySQL</indexterm>
-        <indexterm audience="Cloudera">PostgreSQL</indexterm>
+        <indexterm audience="hidden">Hive</indexterm>
+        <indexterm audience="hidden">MySQL</indexterm>
+        <indexterm audience="hidden">PostgreSQL</indexterm>
         Impala can interoperate with data stored in Hive, and uses the same infrastructure as Hive for tracking
         metadata about schema objects such as tables and columns. The following components are prerequisites for
         Impala:
@@ -193,8 +193,8 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">Java</indexterm>
-        <indexterm audience="Cloudera">impala-dependencies.jar</indexterm>
+        <indexterm audience="hidden">Java</indexterm>
+        <indexterm audience="hidden">impala-dependencies.jar</indexterm>
         Although Impala is primarily written in C++, it does use Java to communicate with various Hadoop
         components:
       </p>
@@ -237,7 +237,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">network configuration</indexterm>
+        <indexterm audience="hidden">network configuration</indexterm>
         As part of ensuring best performance, Impala attempts to complete tasks on local data, as opposed to using
         network connections to work with remote data. To support this goal, Impala matches
         the�<b>hostname</b>�provided to each Impala daemon with the�<b>IP address</b>�of each DataNode by
@@ -265,14 +265,14 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">hardware requirements</indexterm>
-        <indexterm audience="Cloudera">capacity</indexterm>
-        <indexterm audience="Cloudera">RAM</indexterm>
-        <indexterm audience="Cloudera">memory</indexterm>
-        <indexterm audience="Cloudera">CPU</indexterm>
-        <indexterm audience="Cloudera">processor</indexterm>
-        <indexterm audience="Cloudera">Intel</indexterm>
-        <indexterm audience="Cloudera">AMD</indexterm>
+        <indexterm audience="hidden">hardware requirements</indexterm>
+        <indexterm audience="hidden">capacity</indexterm>
+        <indexterm audience="hidden">RAM</indexterm>
+        <indexterm audience="hidden">memory</indexterm>
+        <indexterm audience="hidden">CPU</indexterm>
+        <indexterm audience="hidden">processor</indexterm>
+        <indexterm audience="hidden">Intel</indexterm>
+        <indexterm audience="hidden">AMD</indexterm>
         During join operations, portions of data from each joined table are loaded into memory. Data sets can be
         very large, so ensure your hardware has sufficient memory to accommodate the joins you anticipate
         completing.
@@ -333,9 +333,9 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">impala user</indexterm>
-        <indexterm audience="Cloudera">impala group</indexterm>
-        <indexterm audience="Cloudera">root user</indexterm>
+        <indexterm audience="hidden">impala user</indexterm>
+        <indexterm audience="hidden">impala group</indexterm>
+        <indexterm audience="hidden">root user</indexterm>
         Impala creates and uses a user and group named <codeph>impala</codeph>. Do not delete this account or group
         and do not modify the account's or group's permissions and rights. Ensure no existing systems obstruct the
         functioning of these accounts and groups. For example, if you have scripts that delete user accounts not in
@@ -343,7 +343,7 @@ under the License.
       </p>
 
 <!-- Taking out because no longer applicable in CDH 5.5 and up. -->
-      <p id="impala_hdfs_group" rev="1.2" audience="Cloudera">
+      <p id="impala_hdfs_group" rev="1.2" audience="hidden">
         For the resource management feature to work (in combination with CDH 5 and the YARN and Llama components),
         the <codeph>impala</codeph> user must be a member of the <codeph>hdfs</codeph> group. This setup is
         performed automatically during a new install, but not when upgrading from earlier Impala releases to Impala

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_processes.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_processes.xml b/docs/topics/impala_processes.xml
index 6bba031..a97a644 100644
--- a/docs/topics/impala_processes.xml
+++ b/docs/topics/impala_processes.xml
@@ -33,9 +33,9 @@ under the License.
   <conbody>
 
     <p rev="1.2">
-      <indexterm audience="Cloudera">state store</indexterm>
-      <indexterm audience="Cloudera">starting services</indexterm>
-      <indexterm audience="Cloudera">services</indexterm>
+      <indexterm audience="hidden">state store</indexterm>
+      <indexterm audience="hidden">starting services</indexterm>
+      <indexterm audience="hidden">services</indexterm>
       To activate Impala if it is installed but not yet started:
     </p>
 
@@ -69,7 +69,7 @@ under the License.
     <p outputclass="toc inpage"/>
   </conbody>
 
-  <concept id="starting_via_cm" audience="Cloudera">
+  <concept id="starting_via_cm" audience="hidden">
 
     <title>Starting Impala through Cloudera Manager</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_proxy.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_proxy.xml b/docs/topics/impala_proxy.xml
index f064961..bf46a37 100644
--- a/docs/topics/impala_proxy.xml
+++ b/docs/topics/impala_proxy.xml
@@ -143,7 +143,7 @@ under the License.
           Run the load-balancing proxy server, pointing it at the configuration file that you set up.
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
           On systems managed by Cloudera Manager, on the page
           <menucascade><uicontrol>Impala</uicontrol><uicontrol>Configuration</uicontrol><uicontrol>Impala Daemon
           Default Group</uicontrol></menucascade>, specify a value for the <uicontrol>Impala Daemons Load
@@ -170,7 +170,7 @@ under the License.
 
   </concept>
 
-  <concept id="proxy_balancing" rev="CDH-33836 DOCS-349 CDH-39925 CDH-36812" audience="Cloudera">
+  <concept id="proxy_balancing" rev="CDH-33836 DOCS-349 CDH-39925 CDH-36812" audience="hidden">
     <title>Choosing the Load-Balancing Algorithm</title>
     <conbody>
       <p>
@@ -269,7 +269,7 @@ under the License.
           running the <cmdname>impalad</cmdname> daemon.
         </li>
 
-        <li rev="CDH-40363" audience="Cloudera">
+        <li rev="CDH-40363" audience="hidden">
           For a cluster managed by Cloudera Manager (5.4.2 or higher), fill in the Impala configuration setting
           <uicontrol>Impala Daemons Load Balancer</uicontrol> with the appropriate host:port combination.
           Then restart the Impala service.
@@ -320,7 +320,7 @@ under the License.
                 configuration snippet, add: <codeblock>--principal=impala/<varname>proxy_host</varname>@<varname>realm</varname>
   --be_principal=impala/<varname>actual_host</varname>@<varname>realm</varname>
   --keytab_file=<varname>path_to_merged_keytab</varname></codeblock>
-                <note audience="Cloudera">
+                <note audience="hidden">
                   <p>On a cluster managed by Cloudera Manager 5.1 (or higher),
                     when you set up Kerberos authentication using the wizard, you
                     can choose to allow Cloudera Manager to deploy the
@@ -335,7 +335,7 @@ under the License.
                 </note>
               </li>
 
-              <li audience="Cloudera">
+              <li audience="hidden">
                 On a cluster managed by Cloudera Manager, create a role group to set the configuration values from
                 the preceding step on a per-host basis.
               </li>
@@ -352,7 +352,7 @@ under the License.
             Restart Impala to make the changes take effect. Follow the appropriate steps depending on whether you use
             Cloudera Manager or not:
             <ul>
-              <li audience="Cloudera">
+              <li audience="hidden">
                 On a cluster managed by Cloudera Manager, restart the Impala service.
               </li>
 
@@ -522,11 +522,11 @@ listen impalajdbc :21051
 
       <note conref="../shared/impala_common.xml#common/proxy_jdbc_caveat"/>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         The following example shows extra steps needed for a cluster using Kerberos authentication:
       </p>
 
-<codeblock audience="Cloudera">$ klist
+<codeblock audience="hidden">$ klist
 $ impala-shell -k
 $ kinit -r 1d -kt /systest/keytabs/hdfs.keytab hdfs
 $ impala-shell -i c2104.hal.cloudera.com:21000

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_query_timeout_s.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_query_timeout_s.xml b/docs/topics/impala_query_timeout_s.xml
index 3fb40cd..e41c087 100644
--- a/docs/topics/impala_query_timeout_s.xml
+++ b/docs/topics/impala_query_timeout_s.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">QUERY_TIMEOUT_S query option</indexterm>
+      <indexterm audience="hidden">QUERY_TIMEOUT_S query option</indexterm>
       Sets the idle query timeout value for the session, in seconds. Queries that sit idle for longer than the
       timeout value are automatically cancelled. If the system administrator specified the
       <codeph>--idle_query_timeout</codeph> startup option, <codeph>QUERY_TIMEOUT_S</codeph> must be smaller than

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_rcfile.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_rcfile.xml b/docs/topics/impala_rcfile.xml
index 31d5a0c..ed5b029 100644
--- a/docs/topics/impala_rcfile.xml
+++ b/docs/topics/impala_rcfile.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">RCFile support in Impala</indexterm>
+      <indexterm audience="hidden">RCFile support in Impala</indexterm>
       Impala supports using RCFile data files.
     </p>
 
@@ -167,7 +167,7 @@ Returned 3 row(s) in 0.23s</codeblock>
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">compression</indexterm>
+        <indexterm audience="hidden">compression</indexterm>
         You may want to enable compression on existing tables. Enabling compression provides performance gains in
         most cases and is supported for RCFile tables. For example, to enable Snappy compression, you would specify
         the following additional settings when loading data through the Hive shell:
@@ -250,7 +250,7 @@ hive&gt; INSERT OVERWRITE TABLE tbl_rc PARTITION(year) SELECT * FROM tbl;</codeb
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="rcfile_data_types">
+  <concept audience="hidden" id="rcfile_data_types">
 
     <title>Data Type Considerations for RCFile Tables</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_refresh.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_refresh.xml b/docs/topics/impala_refresh.xml
index d0e3d22..8244aa4 100644
--- a/docs/topics/impala_refresh.xml
+++ b/docs/topics/impala_refresh.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">REFRESH statement</indexterm>
+      <indexterm audience="hidden">REFRESH statement</indexterm>
       To accurately respond to queries, the Impala node that acts as the coordinator (the node to which you are
       connected through <cmdname>impala-shell</cmdname>, JDBC, or ODBC) must have current metadata about those
       databases and tables that are referenced in Impala queries. If you are not familiar with the way Impala uses

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_replica_preference.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_replica_preference.xml b/docs/topics/impala_replica_preference.xml
index 3398403..610bd35 100644
--- a/docs/topics/impala_replica_preference.xml
+++ b/docs/topics/impala_replica_preference.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p rev="2.7.0">
-      <indexterm audience="Cloudera">REPLICA_PREFERENCE query option</indexterm>
+      <indexterm audience="hidden">REPLICA_PREFERENCE query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_request_pool.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_request_pool.xml b/docs/topics/impala_request_pool.xml
index d9a6210..625dc87 100644
--- a/docs/topics/impala_request_pool.xml
+++ b/docs/topics/impala_request_pool.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">REQUEST_POOL query option</indexterm>
+      <indexterm audience="hidden">REQUEST_POOL query option</indexterm>
       The pool or queue name that queries should be submitted to. Only applies when you enable the Impala admission control feature.
       Specifies the name of the pool used by requests from Impala to the resource manager.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_reservation_request_timeout.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_reservation_request_timeout.xml b/docs/topics/impala_reservation_request_timeout.xml
index c350728..6f70cfd 100644
--- a/docs/topics/impala_reservation_request_timeout.xml
+++ b/docs/topics/impala_reservation_request_timeout.xml
@@ -39,7 +39,7 @@ under the License.
     <note conref="../shared/impala_common.xml#common/llama_query_options_obsolete"/>
 
     <p>
-      <indexterm audience="Cloudera">RESERVATION_REQUEST_TIMEOUT query option</indexterm>
+      <indexterm audience="hidden">RESERVATION_REQUEST_TIMEOUT query option</indexterm>
       Maximum number of milliseconds Impala will wait for a reservation to be completely granted or denied. Used in
       conjunction with the Impala resource management feature in Impala 1.2 and higher with CDH 5.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_reserved_words.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_reserved_words.xml b/docs/topics/impala_reserved_words.xml
index 080735a..423fd43 100644
--- a/docs/topics/impala_reserved_words.xml
+++ b/docs/topics/impala_reserved_words.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">reserved words</indexterm>
+      <indexterm audience="hidden">reserved words</indexterm>
       The following are the reserved words for the current release of Impala. A reserved word is one that
       cannot be used directly as an identifier; you must quote it with backticks. For example, a statement
       <codeph>CREATE TABLE select (x INT)</codeph> fails, while <codeph>CREATE TABLE `select` (x INT)</codeph>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_resource_management.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_resource_management.xml b/docs/topics/impala_resource_management.xml
index c16f367..bc98014 100644
--- a/docs/topics/impala_resource_management.xml
+++ b/docs/topics/impala_resource_management.xml
@@ -44,7 +44,7 @@ under the License.
     <p outputclass="toc inpage"/>
   </conbody>
 
-  <concept audience="Cloudera" id="llama">
+  <concept audience="hidden" id="llama">
   <!-- Hiding the whole concept now that Llama is desupported. -->
 
     <title>The Llama Daemon</title>
@@ -315,14 +315,14 @@ under the License.
 
     <conbody>
 
-<!-- Conditionalizing some content here with audience="Cloudera" because there are already some XML comments
+<!-- Conditionalizing some content here with audience="hidden" because there are already some XML comments
      inside the list, so not practical to enclose the whole thing in XML comments. -->
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         Currently, Impala in CDH 5 has the following limitations for resource management of Impala queries:
       </p>
 
-      <ul audience="Cloudera">
+      <ul audience="hidden">
         <li>
           Table statistics are required, and column statistics are highly valuable, for Impala to produce accurate
           estimates of how much memory to request from YARN. See

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_revoke.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_revoke.xml b/docs/topics/impala_revoke.xml
index 28137ea..97a912d 100644
--- a/docs/topics/impala_revoke.xml
+++ b/docs/topics/impala_revoke.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">REVOKE statement</indexterm>
+      <indexterm audience="hidden">REVOKE statement</indexterm>
 <!-- Copied from Sentry docs. Turn into conref. I did some rewording for clarity. -->
       The <codeph>REVOKE</codeph> statement revokes roles or privileges on a specified object from groups. Only
       Sentry administrative users can revoke the role from a group. The revocation has a cascading effect. For

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_rm_initial_mem.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_rm_initial_mem.xml b/docs/topics/impala_rm_initial_mem.xml
index e63f70a..fd9f819 100644
--- a/docs/topics/impala_rm_initial_mem.xml
+++ b/docs/topics/impala_rm_initial_mem.xml
@@ -33,7 +33,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">RM_INITIAL_MEM query option</indexterm>
+      <indexterm audience="hidden">RM_INITIAL_MEM query option</indexterm>
     </p>
 
     <p>



[5/6] incubator-impala git commit: Global search/replace: audience="Cloudera" -> audience="hidden".

Posted by jr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_datetime_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_datetime_functions.xml b/docs/topics/impala_datetime_functions.xml
index 08a1036..4276c88 100644
--- a/docs/topics/impala_datetime_functions.xml
+++ b/docs/topics/impala_datetime_functions.xml
@@ -89,7 +89,7 @@ months_between
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">add_months() function</indexterm>
+          <indexterm audience="hidden">add_months() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of months.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -141,7 +141,7 @@ select now(), add_months(now(), -1);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">adddate() function</indexterm>
+          <indexterm audience="hidden">adddate() function</indexterm>
           <b>Purpose:</b> Adds a specified number of days to a <codeph>TIMESTAMP</codeph> value. Similar to
           <codeph>date_add()</codeph>, but starts with an actual <codeph>TIMESTAMP</codeph> value instead of a
           string that is converted to a <codeph>TIMESTAMP</codeph>.
@@ -179,7 +179,7 @@ select now() as right_now, adddate(now(), -15) as now_minus_15;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">current_timestamp() function</indexterm>
+          <indexterm audience="hidden">current_timestamp() function</indexterm>
           <b>Purpose:</b> Alias for the <codeph>now()</codeph> function.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -213,7 +213,7 @@ select current_timestamp() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">date_add() function</indexterm>
+          <indexterm audience="hidden">date_add() function</indexterm>
           <b>Purpose:</b> Adds a specified number of days to a <codeph>TIMESTAMP</codeph> value.
           <!-- Found this not to be true in latest release. I think the signature changed way back.
           The first argument
@@ -287,7 +287,7 @@ select date_add(cast('2016-01-31' as timestamp), interval 3 months) as 'april_31
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">date_part() function</indexterm>
+          <indexterm audience="hidden">date_part() function</indexterm>
           <b>Purpose:</b> Similar to
           <xref href="impala_datetime_functions.xml#datetime_functions/extract"><codeph>EXTRACT()</codeph></xref>,
           with the argument order reversed. Supports the same date and time units as <codeph>EXTRACT()</codeph>.
@@ -323,7 +323,7 @@ select date_part('hour',now()) as hour_of_day;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">date_sub() function</indexterm>
+          <indexterm audience="hidden">date_sub() function</indexterm>
           <b>Purpose:</b> Subtracts a specified number of days from a <codeph>TIMESTAMP</codeph> value.
           <!-- Found this not to be true in latest release. I think the signature changed way back.
           The first argument can be a string, which is automatically cast to <codeph>TIMESTAMP</codeph> if it uses the
@@ -396,7 +396,7 @@ select date_sub(cast('2016-05-31' as timestamp), interval 1 months) as 'april_31
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">datediff() function</indexterm>
+          <indexterm audience="hidden">datediff() function</indexterm>
           <b>Purpose:</b> Returns the number of days between two <codeph>TIMESTAMP</codeph> values.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -474,7 +474,7 @@ select now() as right_now, datediff(now(), now() - interval 18 hours) as 18_hour
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">day() function</indexterm>
+          <indexterm audience="hidden">day() function</indexterm>
           <b>Purpose:</b> Returns the day field from the date portion of a <codeph>TIMESTAMP</codeph>.
           The value represents the day of the month, therefore is in the range 1-31, or less for
           months without 31 days.
@@ -549,7 +549,7 @@ select day('2016-02-028');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">dayname() function</indexterm>
+          <indexterm audience="hidden">dayname() function</indexterm>
           <b>Purpose:</b> Returns the day field from a <codeph>TIMESTAMP</codeph> value, converted to the string
           corresponding to that day name. The range of return values is <codeph>'Sunday'</codeph> to
           <codeph>'Saturday'</codeph>. Used in report-generating queries, as an alternative to calling
@@ -593,7 +593,7 @@ select now() + interval 1 day as tomorrow,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">dayofweek() function</indexterm>
+          <indexterm audience="hidden">dayofweek() function</indexterm>
           <b>Purpose:</b> Returns the day field from the date portion of a <codeph>TIMESTAMP</codeph>, corresponding to the day of
           the week. The range of return values is 1 (Sunday) to 7 (Saturday).
           <p>
@@ -621,7 +621,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">dayofyear() function</indexterm>
+          <indexterm audience="hidden">dayofyear() function</indexterm>
           <b>Purpose:</b> Returns the day field from a <codeph>TIMESTAMP</codeph> value, corresponding to the day
           of the year. The range of return values is 1 (January 1) to 366 (December 31 of a leap year).
           <p>
@@ -664,7 +664,7 @@ select now() - interval 1 year as last_year,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">days_add() function</indexterm>
+          <indexterm audience="hidden">days_add() function</indexterm>
           <b>Purpose:</b> Adds a specified number of days to a <codeph>TIMESTAMP</codeph> value. Similar to
           <codeph>date_add()</codeph>, but starts with an actual <codeph>TIMESTAMP</codeph> value instead of a
           string that is converted to a <codeph>TIMESTAMP</codeph>.
@@ -692,7 +692,7 @@ select now() as right_now, days_add(now(), 31) as 31_days_later;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">days_sub() function</indexterm>
+          <indexterm audience="hidden">days_sub() function</indexterm>
           <b>Purpose:</b> Subtracts a specified number of days from a <codeph>TIMESTAMP</codeph> value. Similar to
           <codeph>date_sub()</codeph>, but starts with an actual <codeph>TIMESTAMP</codeph> value instead of a
           string that is converted to a <codeph>TIMESTAMP</codeph>.
@@ -719,7 +719,7 @@ select now() as right_now, days_sub(now(), 31) as 31_days_ago;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">extract() function</indexterm>
+          <indexterm audience="hidden">extract() function</indexterm>
           <b>Purpose:</b> Returns one of the numeric date or time fields from a <codeph>TIMESTAMP</codeph> value.
           <p>
             <b>Unit argument:</b> The <codeph>unit</codeph> string can be one of <codeph>year</codeph>,
@@ -778,7 +778,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">from_unixtime() function</indexterm>
+          <indexterm audience="hidden">from_unixtime() function</indexterm>
           <b>Purpose:</b> Converts the number of seconds from the Unix epoch to the specified time into a string in
           the local time zone.
           <p>
@@ -854,7 +854,7 @@ select from_unixtime(1392394861,"HH:mm:ss");
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">from_utc_timestamp() function</indexterm>
+          <indexterm audience="hidden">from_utc_timestamp() function</indexterm>
           <b>Purpose:</b> Converts a specified UTC timestamp value into the appropriate value for a specified time
           zone.
           <p>
@@ -921,7 +921,7 @@ select '2016-01-05' as local_datetime,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">hour() function</indexterm>
+          <indexterm audience="hidden">hour() function</indexterm>
           <b>Purpose:</b> Returns the hour field from a <codeph>TIMESTAMP</codeph> field.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -955,7 +955,7 @@ select now() + interval 12 hours as 12_hours_from_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">hours_add() function</indexterm>
+          <indexterm audience="hidden">hours_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of hours.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -982,7 +982,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">hours_sub() function</indexterm>
+          <indexterm audience="hidden">hours_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of hours.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1008,7 +1008,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">int_months_between() function</indexterm>
+          <indexterm audience="hidden">int_months_between() function</indexterm>
           <b>Purpose:</b> Returns the number of months between the date portions of two <codeph>TIMESTAMP</codeph> values,
           as an <codeph>INT</codeph> representing only the full months that passed.
           <p>
@@ -1089,7 +1089,7 @@ select int_months_between('2015-03-31', '2015-01-30');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">microseconds_add() function</indexterm>
+          <indexterm audience="hidden">microseconds_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of microseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1116,7 +1116,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">microseconds_sub() function</indexterm>
+          <indexterm audience="hidden">microseconds_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of microseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1142,7 +1142,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">millisecond() function</indexterm>
+          <indexterm audience="hidden">millisecond() function</indexterm>
           <b>Purpose:</b> Returns the millisecond portion of a <codeph>TIMESTAMP</codeph> value.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -1185,7 +1185,7 @@ select now(), millisecond(now());
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">milliseconds_add() function</indexterm>
+          <indexterm audience="hidden">milliseconds_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of milliseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1212,7 +1212,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">milliseconds_sub() function</indexterm>
+          <indexterm audience="hidden">milliseconds_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of milliseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1238,7 +1238,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">minute() function</indexterm>
+          <indexterm audience="hidden">minute() function</indexterm>
           <b>Purpose:</b> Returns the minute field from a <codeph>TIMESTAMP</codeph> value.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -1264,7 +1264,7 @@ select now() as right_now, minute(now()) as current_minute;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">minutes_add() function</indexterm>
+          <indexterm audience="hidden">minutes_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of minutes.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1290,7 +1290,7 @@ select now() as right_now, minutes_add(now(), 90) as 90_minutes_from_now;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">minutes_sub() function</indexterm>
+          <indexterm audience="hidden">minutes_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of minutes.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1316,7 +1316,7 @@ select now() as right_now, minutes_sub(now(), 90) as 90_minutes_ago;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">month() function</indexterm>
+          <indexterm audience="hidden">month() function</indexterm>
           <b>Purpose:</b> Returns the month field, represented as an integer, from the date portion of a <codeph>TIMESTAMP</codeph>.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -1342,7 +1342,7 @@ select now() as right_now, month(now()) as current_month;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">months_add() function</indexterm>
+          <indexterm audience="hidden">months_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of months.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1408,7 +1408,7 @@ with t1 as (select cast('2015-01-31' as timestamp) as jan_31)
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">months_between() function</indexterm>
+          <indexterm audience="hidden">months_between() function</indexterm>
           <b>Purpose:</b> Returns the number of months between the date portions of two <codeph>TIMESTAMP</codeph> values.
           Can include a fractional part representing extra days in addition to the full months
           between the dates. The fractional component is computed by dividing the difference in days by 31 (regardless of the month).
@@ -1586,7 +1586,7 @@ select months_between('2015-03-28 23:00:00', '2015-03-01 11:45:00');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">months_sub() function</indexterm>
+          <indexterm audience="hidden">months_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of months.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1613,7 +1613,7 @@ with t1 as (select trunc(now(), 'dd') as today)
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">nanoseconds_add() function</indexterm>
+          <indexterm audience="hidden">nanoseconds_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of nanoseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1647,7 +1647,7 @@ select now() as right_now, nanoseconds_add(now(), 1e9) as 1_second_later;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">nanoseconds_sub() function</indexterm>
+          <indexterm audience="hidden">nanoseconds_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of nanoseconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1679,7 +1679,7 @@ select now() as right_now, nanoseconds_sub(now(), 1e9) as 1_second_earlier;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">now() function</indexterm>
+          <indexterm audience="hidden">now() function</indexterm>
           <b>Purpose:</b> Returns the current date and time (in the local time zone) as a
           <codeph>TIMESTAMP</codeph> value.
           <p>
@@ -1736,7 +1736,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">second() function</indexterm>
+          <indexterm audience="hidden">second() function</indexterm>
           <b>Purpose:</b> Returns the second field from a <codeph>TIMESTAMP</codeph> value.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -1763,7 +1763,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">seconds_add() function</indexterm>
+          <indexterm audience="hidden">seconds_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of seconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1790,7 +1790,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">seconds_sub() function</indexterm>
+          <indexterm audience="hidden">seconds_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of seconds.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -1817,7 +1817,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">subdate() function</indexterm>
+          <indexterm audience="hidden">subdate() function</indexterm>
           <b>Purpose:</b> Subtracts a specified number of days from a <codeph>TIMESTAMP</codeph> value. Similar to
           <codeph>date_sub()</codeph>, but starts with an actual <codeph>TIMESTAMP</codeph> value instead of a
           string that is converted to a <codeph>TIMESTAMP</codeph>.
@@ -1855,7 +1855,7 @@ select now() as right_now, subdate(now(), -15) as now_plus_15;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">timeofday() function</indexterm>
+          <indexterm audience="hidden">timeofday() function</indexterm>
           <b>Purpose:</b> Returns a string representation of the current date and time, according to the time of the local system,
           including any time zone designation.
           <p>
@@ -1921,7 +1921,7 @@ select regexp_replace(timeofday(), '.* ([A-Z]+)$', '\\1') as current_timezone;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">timestamp_cmp() function</indexterm>
+          <indexterm audience="hidden">timestamp_cmp() function</indexterm>
           <b>Purpose:</b> Tests if one <codeph>TIMESTAMP</codeph> value is
           newer than, older than, or identical to another <codeph>TIMESTAMP</codeph>
           <p>
@@ -2015,7 +2015,7 @@ select timestamp_cmp(now(), null)
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">to_date() function</indexterm>
+          <indexterm audience="hidden">to_date() function</indexterm>
           <b>Purpose:</b> Returns a string representation of the date field from a timestamp value.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -2041,7 +2041,7 @@ select now() as right_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">to_utc_timestamp() function</indexterm>
+          <indexterm audience="hidden">to_utc_timestamp() function</indexterm>
           <b>Purpose:</b> Converts a specified timestamp value in a specified time zone into the corresponding
           value for the UTC time zone.
           <p>
@@ -2107,7 +2107,7 @@ select now() as 'Current time in California USA',
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">trunc() function</indexterm>
+          <indexterm audience="hidden">trunc() function</indexterm>
           <b>Purpose:</b> Strips off fields from a <codeph>TIMESTAMP</codeph> value.
           <p>
             <b>Unit argument:</b> The <codeph>unit</codeph> argument value is case-sensitive. This argument string
@@ -2206,7 +2206,7 @@ select now() + interval 2 weeks as 2_weeks_from_now,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">unix_timestamp() function</indexterm>
+          <indexterm audience="hidden">unix_timestamp() function</indexterm>
           <b>Purpose:</b> Returns an integer value representing the current date and time as a delta from the Unix
           epoch, or converts from a specified date and time value represented as a <codeph>TIMESTAMP</codeph> or
           <codeph>STRING</codeph>.
@@ -2313,7 +2313,7 @@ select unix_timestamp
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">weekofyear() function</indexterm>
+          <indexterm audience="hidden">weekofyear() function</indexterm>
           <b>Purpose:</b> Returns the corresponding week (1-53) from the date portion of a <codeph>TIMESTAMP</codeph>.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -2347,7 +2347,7 @@ select now() + interval 2 weeks as in_2_weeks,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">weeks_add() function</indexterm>
+          <indexterm audience="hidden">weeks_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of weeks.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -2373,7 +2373,7 @@ select now() as right_now, weeks_add(now(), 2) as week_after_next;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">weeks_sub() function</indexterm>
+          <indexterm audience="hidden">weeks_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of weeks.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -2398,7 +2398,7 @@ select now() as right_now, weeks_sub(now(), 2) as week_before_last;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">year() function</indexterm>
+          <indexterm audience="hidden">year() function</indexterm>
           <b>Purpose:</b> Returns the year field from the date portion of a <codeph>TIMESTAMP</codeph>.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -2424,7 +2424,7 @@ select now() as right_now, year(now()) as this_year;
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">years_add() function</indexterm>
+          <indexterm audience="hidden">years_add() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time plus some number of years.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>
@@ -2464,7 +2464,7 @@ select cast('2016-02-29' as timestamp) as feb_29_2016,
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">years_sub() function</indexterm>
+          <indexterm audience="hidden">years_sub() function</indexterm>
           <b>Purpose:</b> Returns the specified date and time minus some number of years.
           <p>
             <b>Return type:</b> <codeph>timestamp</codeph>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_debug_action.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_debug_action.xml b/docs/topics/impala_debug_action.xml
index ebb5fa7..72308df 100644
--- a/docs/topics/impala_debug_action.xml
+++ b/docs/topics/impala_debug_action.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DEBUG_ACTION query option</indexterm>
+      <indexterm audience="hidden">DEBUG_ACTION query option</indexterm>
       Introduces artificial problem conditions within queries. For internal Cloudera debugging and troubleshooting.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_decimal.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_decimal.xml b/docs/topics/impala_decimal.xml
index 1be0e2b..0175911 100644
--- a/docs/topics/impala_decimal.xml
+++ b/docs/topics/impala_decimal.xml
@@ -124,7 +124,7 @@ under the License.
           if all the digits of the input values were 9s and the absolute values were added together.
         </p>
 <!-- Seems like buggy output from this first query, so hiding the example for the time being. -->
-<codeblock audience="Cloudera"><![CDATA[[localhost:21000] > select 50000.5 + 12.444, precision(50000.5 + 12.444), scale(50000.5 + 12.444);
+<codeblock audience="hidden"><![CDATA[[localhost:21000] > select 50000.5 + 12.444, precision(50000.5 + 12.444), scale(50000.5 + 12.444);
 +------------------+-----------------------------+-------------------------+
 | 50000.5 + 12.444 | precision(50000.5 + 12.444) | scale(50000.5 + 12.444) |
 +------------------+-----------------------------+-------------------------+

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_delete.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_delete.xml b/docs/topics/impala_delete.xml
index a8be2f7..af20d19 100644
--- a/docs/topics/impala_delete.xml
+++ b/docs/topics/impala_delete.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DELETE statement</indexterm>
+      <indexterm audience="hidden">DELETE statement</indexterm>
       Deletes one or more rows from a Kudu table.
       Although deleting a single row or a range of rows would be inefficient for tables using HDFS
       data files, Kudu is able to perform this operation efficiently. Therefore, this statement

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_describe.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_describe.xml b/docs/topics/impala_describe.xml
index b95aa26..2548d44 100644
--- a/docs/topics/impala_describe.xml
+++ b/docs/topics/impala_describe.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DESCRIBE statement</indexterm>
+      <indexterm audience="hidden">DESCRIBE statement</indexterm>
       The <codeph>DESCRIBE</codeph> statement displays metadata about a table, such as the column names and their
       data types.
       <ph rev="2.3.0">In <keyword keyref="impala23_full"/> and higher, you can specify the name of a complex type column, which takes

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_development.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_development.xml b/docs/topics/impala_development.xml
index db1859d..41f66c0 100644
--- a/docs/topics/impala_development.xml
+++ b/docs/topics/impala_development.xml
@@ -175,21 +175,21 @@ under the License.
 
 <!-- Bunch of potential concept topics for future consideration. Major areas of Impala modelled on areas of discussion for Oracle Database, and distributed databases in general. -->
 
-  <concept id="intro_datatypes" audience="Cloudera">
+  <concept id="intro_datatypes" audience="hidden">
 
     <title>Overview of Impala SQL Data Types</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_network" audience="Cloudera">
+  <concept id="intro_network" audience="hidden">
 
     <title>Overview of Impala Network Topology</title>
 
     <conbody/>
   </concept>
 
-  <concept id="intro_cluster" audience="Cloudera">
+  <concept id="intro_cluster" audience="hidden">
 
     <title>Overview of Impala Cluster Topology</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_cached_reads.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_cached_reads.xml b/docs/topics/impala_disable_cached_reads.xml
index 886c645..20391db 100644
--- a/docs/topics/impala_disable_cached_reads.xml
+++ b/docs/topics/impala_disable_cached_reads.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DISABLE_CACHED_READS query option</indexterm>
+      <indexterm audience="hidden">DISABLE_CACHED_READS query option</indexterm>
       Prevents Impala from reading data files that are <q>pinned</q> in memory
       through the HDFS caching feature. Primarily a debugging option for
       cases where processing of HDFS cached data is concentrated on a single

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_codegen.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_codegen.xml b/docs/topics/impala_disable_codegen.xml
index 43e5297..82add52 100644
--- a/docs/topics/impala_disable_codegen.xml
+++ b/docs/topics/impala_disable_codegen.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DISABLE_CODEGEN query option</indexterm>
+      <indexterm audience="hidden">DISABLE_CODEGEN query option</indexterm>
       This is a debug option, intended for diagnosing and working around issues that cause crashes. If a query
       fails with an <q>illegal instruction</q> or other hardware-specific message, try setting
       <codeph>DISABLE_CODEGEN=true</codeph> and running the query again. If the query succeeds only when the

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_outermost_topn.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_outermost_topn.xml b/docs/topics/impala_disable_outermost_topn.xml
index ebda7ff..2bc1f12 100644
--- a/docs/topics/impala_disable_outermost_topn.xml
+++ b/docs/topics/impala_disable_outermost_topn.xml
@@ -33,7 +33,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">DISABLE_OUTERMOST_TOPN query option</indexterm>
+      <indexterm audience="hidden">DISABLE_OUTERMOST_TOPN query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_row_runtime_filtering.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_row_runtime_filtering.xml b/docs/topics/impala_disable_row_runtime_filtering.xml
index c12e167..3280084 100644
--- a/docs/topics/impala_disable_row_runtime_filtering.xml
+++ b/docs/topics/impala_disable_row_runtime_filtering.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">DISABLE_ROW_RUNTIME_FILTERING query option</indexterm>
+      <indexterm audience="hidden">DISABLE_ROW_RUNTIME_FILTERING query option</indexterm>
       The <codeph>DISABLE_ROW_RUNTIME_FILTERING</codeph> query option
       reduces the scope of the runtime filtering feature. Queries still dynamically prune
       partitions, but do not apply the filtering logic to individual rows within partitions.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_streaming_preaggregations.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_streaming_preaggregations.xml b/docs/topics/impala_disable_streaming_preaggregations.xml
index e43569d..bde6ec7 100644
--- a/docs/topics/impala_disable_streaming_preaggregations.xml
+++ b/docs/topics/impala_disable_streaming_preaggregations.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0 IMPALA-1305">
-      <indexterm audience="Cloudera">DISABLE_STREAMING_PREAGGREGATIONS query option</indexterm>
+      <indexterm audience="hidden">DISABLE_STREAMING_PREAGGREGATIONS query option</indexterm>
       Turns off the <q>streaming preaggregation</q> optimization that is available in <keyword keyref="impala25_full"/>
       and higher. This optimization reduces unnecessary work performed by queries that perform aggregation
       operations on columns with few or no duplicate values, for example <codeph>DISTINCT <varname>id_column</varname></codeph>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_disable_unsafe_spills.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_disable_unsafe_spills.xml b/docs/topics/impala_disable_unsafe_spills.xml
index d64798a..0a61966 100644
--- a/docs/topics/impala_disable_unsafe_spills.xml
+++ b/docs/topics/impala_disable_unsafe_spills.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">DISABLE_UNSAFE_SPILLS query option</indexterm>
+      <indexterm audience="hidden">DISABLE_UNSAFE_SPILLS query option</indexterm>
       Enable this option if you prefer to have queries fail when they exceed the Impala memory limit, rather than
       write temporary data to disk.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_distinct.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_distinct.xml b/docs/topics/impala_distinct.xml
index 63dba1f..710ea0c 100644
--- a/docs/topics/impala_distinct.xml
+++ b/docs/topics/impala_distinct.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DISTINCT operator</indexterm>
+      <indexterm audience="hidden">DISTINCT operator</indexterm>
       The <codeph>DISTINCT</codeph> operator in a <codeph>SELECT</codeph> statement filters the result set to
       remove duplicates:
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_data_source.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_data_source.xml b/docs/topics/impala_drop_data_source.xml
index 80ebf0d..f4784ea 100644
--- a/docs/topics/impala_drop_data_source.xml
+++ b/docs/topics/impala_drop_data_source.xml
@@ -18,7 +18,7 @@ specific language governing permissions and limitations
 under the License.
 -->
 <!DOCTYPE concept PUBLIC "-//OASIS//DTD DITA Concept//EN" "concept.dtd">
-<concept audience="Cloudera" rev="1.4.0" id="drop_data_source">
+<concept audience="hidden" rev="1.4.0" id="drop_data_source">
 
   <title>DROP DATA SOURCE Statement</title>
   <titlealts audience="PDF"><navtitle>DROP DATA SOURCE</navtitle></titlealts>
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP DATA SOURCE statement</indexterm>
+      <indexterm audience="hidden">DROP DATA SOURCE statement</indexterm>
     </p>
 
     <p conref="../shared/impala_common.xml#common/syntax_blurb"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_database.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_database.xml b/docs/topics/impala_drop_database.xml
index 78433e6..5dd1536 100644
--- a/docs/topics/impala_drop_database.xml
+++ b/docs/topics/impala_drop_database.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP DATABASE statement</indexterm>
+      <indexterm audience="hidden">DROP DATABASE statement</indexterm>
       Removes a database from the system. The physical operations involve removing the metadata for the database
       from the metastore, and deleting the corresponding <codeph>*.db</codeph> directory from HDFS.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_function.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_function.xml b/docs/topics/impala_drop_function.xml
index 297741d..6627211 100644
--- a/docs/topics/impala_drop_function.xml
+++ b/docs/topics/impala_drop_function.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP FUNCTION statement</indexterm>
+      <indexterm audience="hidden">DROP FUNCTION statement</indexterm>
       Removes a user-defined function (UDF), so that it is not available for execution during Impala
       <codeph>SELECT</codeph> or <codeph>INSERT</codeph> operations.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_role.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_role.xml b/docs/topics/impala_drop_role.xml
index 5860391..2a8484b 100644
--- a/docs/topics/impala_drop_role.xml
+++ b/docs/topics/impala_drop_role.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP ROLE statement</indexterm>
+      <indexterm audience="hidden">DROP ROLE statement</indexterm>
 <!-- Copied from Sentry docs. Turn into conref. I did some rewording for clarity. -->
       The <codeph>DROP ROLE</codeph> statement removes a role from the metastore database. Once dropped, the role
       is revoked for all users to whom it was previously assigned, and all privileges granted to that role are

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_stats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_stats.xml b/docs/topics/impala_drop_stats.xml
index 9f54d18..fe5a13e 100644
--- a/docs/topics/impala_drop_stats.xml
+++ b/docs/topics/impala_drop_stats.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p rev="2.1.0">
-      <indexterm audience="Cloudera">DROP STATS statement</indexterm>
+      <indexterm audience="hidden">DROP STATS statement</indexterm>
       Removes the specified statistics from a table or partition. The statistics were originally created by the
       <codeph>COMPUTE STATS</codeph> or <codeph>COMPUTE INCREMENTAL STATS</codeph> statement.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_table.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_table.xml b/docs/topics/impala_drop_table.xml
index 7578ae9..b89368b 100644
--- a/docs/topics/impala_drop_table.xml
+++ b/docs/topics/impala_drop_table.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP TABLE statement</indexterm>
+      <indexterm audience="hidden">DROP TABLE statement</indexterm>
       Removes an Impala table. Also removes the underlying HDFS data files for internal tables, although not for
       external tables.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_drop_view.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_drop_view.xml b/docs/topics/impala_drop_view.xml
index ba06ee7..bc70cad 100644
--- a/docs/topics/impala_drop_view.xml
+++ b/docs/topics/impala_drop_view.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">DROP VIEW statement</indexterm>
+      <indexterm audience="hidden">DROP VIEW statement</indexterm>
       Removes the specified view, which was originally created by the <codeph>CREATE VIEW</codeph> statement.
       Because a view is purely a logical construct (an alias for a query) with no physical data behind it,
       <codeph>DROP VIEW</codeph> only involves changes to metadata in the metastore database, not any data files in

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_exec_single_node_rows_threshold.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_exec_single_node_rows_threshold.xml b/docs/topics/impala_exec_single_node_rows_threshold.xml
index 54ef18d..4822712 100644
--- a/docs/topics/impala_exec_single_node_rows_threshold.xml
+++ b/docs/topics/impala_exec_single_node_rows_threshold.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">EXEC_SINGLE_NODE_ROWS_THRESHOLD query option</indexterm>
+      <indexterm audience="hidden">EXEC_SINGLE_NODE_ROWS_THRESHOLD query option</indexterm>
       This setting controls the cutoff point (in terms of number of rows scanned) below which Impala treats a query
       as a <q>small</q> query, turning off optimizations such as parallel execution and native code generation. The
       overhead for these optimizations is applicable for queries involving substantial amounts of data, but it

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_explain.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_explain.xml b/docs/topics/impala_explain.xml
index 61ea708..4c41fdb 100644
--- a/docs/topics/impala_explain.xml
+++ b/docs/topics/impala_explain.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">EXPLAIN statement</indexterm>
+      <indexterm audience="hidden">EXPLAIN statement</indexterm>
       Returns the execution plan for a statement, showing the low-level mechanisms that Impala will use to read the
       data, divide the work among nodes in the cluster, and transmit intermediate and final results across the
       network. Use <codeph>explain</codeph> followed by a complete <codeph>SELECT</codeph> query. For example:

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_explain_level.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_explain_level.xml b/docs/topics/impala_explain_level.xml
index 4d00e4d..ecd06b7 100644
--- a/docs/topics/impala_explain_level.xml
+++ b/docs/topics/impala_explain_level.xml
@@ -39,7 +39,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">EXPLAIN_LEVEL query option</indexterm>
+      <indexterm audience="hidden">EXPLAIN_LEVEL query option</indexterm>
       Controls the amount of detail provided in the output of the <codeph>EXPLAIN</codeph> statement. The basic
       output can help you identify high-level performance issues such as scanning a higher volume of data or more
       partitions than you expect. The higher levels of detail show how intermediate results flow between nodes and

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_faq.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_faq.xml b/docs/topics/impala_faq.xml
index 6988b8d..6db1b03 100644
--- a/docs/topics/impala_faq.xml
+++ b/docs/topics/impala_faq.xml
@@ -52,7 +52,7 @@ under the License.
         FAQs in this category:
       </p>
 
-      <section id="faq_tryout" audience="Cloudera">
+      <section id="faq_tryout" audience="hidden">
 
         <title>How do I try Impala out?</title>
 
@@ -73,7 +73,7 @@ under the License.
         </sectiondiv>
       </section>
 
-      <section id="faq_demo_vm" audience="Cloudera">
+      <section id="faq_demo_vm" audience="hidden">
 
         <title>Does Cloudera offer a VM for demonstrating Impala?</title>
 
@@ -473,7 +473,7 @@ through the <codeph>DEFAULT_ORDER_BY_LIMIT</codeph> query option.</ph>
               information about Impala performance optimizations and tuning techniques for queries.
             </li>
 
-            <li audience="Cloudera">
+            <li audience="hidden">
               Using Cloudera Manager, you can deploy and manage your Impala services. Cloudera Manager is the best
               way to get started with Impala on your cluster.
             </li>
@@ -647,7 +647,7 @@ through the <codeph>DEFAULT_ORDER_BY_LIMIT</codeph> query option.</ph>
         </sectiondiv>
       </section>
 
-      <section audience="Cloudera" id="faq_roadmap">
+      <section audience="hidden" id="faq_roadmap">
 
 <!-- Hidden to avoid RevRec implications. -->
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_file_formats.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_file_formats.xml b/docs/topics/impala_file_formats.xml
index 3480f15..f68e94f 100644
--- a/docs/topics/impala_file_formats.xml
+++ b/docs/topics/impala_file_formats.xml
@@ -39,8 +39,8 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">file formats</indexterm>
-      <indexterm audience="Cloudera">compression</indexterm>
+      <indexterm audience="hidden">file formats</indexterm>
+      <indexterm audience="hidden">compression</indexterm>
       Impala supports several familiar file formats used in Apache Hadoop. Impala can load and query data files
       produced by other Hadoop components such as Pig or MapReduce, and data files produced by Impala can be used
       by other components also. The following sections discuss the procedures, limitations, and performance

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_fixed_issues.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_fixed_issues.xml b/docs/topics/impala_fixed_issues.xml
index a66f9fb..db4c419 100644
--- a/docs/topics/impala_fixed_issues.xml
+++ b/docs/topics/impala_fixed_issues.xml
@@ -2003,7 +2003,7 @@ ce601ec : Enable using isilon as the underlying filesystem.
         fixes to performance and flexibility for dealing with I/O using remote reads.
         See <xref href="impala_isilon.xml#impala_isilon"/> for details on using Impala and Isilon together.
       </p>
-      <p audience="Cloudera"><b>Bug:</b> <xref href="https://jira.cloudera.com/browse/CDH-24040" scope="external" format="html">CDH-24040</xref></p>
+      <p audience="hidden"><b>Bug:</b> <xref href="https://jira.cloudera.com/browse/CDH-24040" scope="external" format="html">CDH-24040</xref></p>
       <p><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/IMPALA-1968" scope="external" format="html">IMPALA-1968</xref>,
       <xref href="https://issues.cloudera.org/browse/IMPALA-1730" scope="external" format="html">IMPALA-1730</xref></p>
       </conbody>
@@ -2098,7 +2098,7 @@ ce601ec : Enable using isilon as the underlying filesystem.
       <title>Isilon: fix custom_cluster/test_insert_inherit_permission.py</title>
       <conbody>
       <p></p>
-      <p audience="Cloudera"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
+      <p audience="hidden"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
       <p><b>Severity:</b> High</p>
       </conbody>
     </concept>
@@ -2109,7 +2109,7 @@ ce601ec : Enable using isilon as the underlying filesystem.
       <title>Isilon: add remote HDFS disk queue</title>
       <conbody>
       <p></p>
-      <p audience="Cloudera"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
+      <p audience="hidden"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
       <p><b>Severity:</b> High</p>
       </conbody>
     </concept>
@@ -2131,7 +2131,7 @@ ce601ec : Enable using isilon as the underlying filesystem.
       <title>Isilon: don't warn for multiple remote parquet blocks</title>
       <conbody>
       <p></p>
-      <p audience="Cloudera"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
+      <p audience="hidden"><b>Bug:</b> <xref href="https://issues.cloudera.org/browse/" scope="external" format="html"></xref></p>
       <p><b>Severity:</b> High</p>
       </conbody>
     </concept>
@@ -5130,7 +5130,7 @@ Bad stats:
           key.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-1188" scope="external" format="html">IMP-1188</xref>
         </p>
@@ -5974,7 +5974,7 @@ hive&gt; NULL</codeblock>
           Insert INTO TABLE SELECT &lt;constant&gt; will not insert any data and may return an error.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-231" scope="external" format="html"/> ; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6436,7 +6436,7 @@ hive&gt; NULL</codeblock>
 
       <conbody>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-474" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6551,7 +6551,7 @@ hive&gt; NULL</codeblock>
           does not propagate to Impala.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-56" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6795,7 +6795,7 @@ hive&gt; NULL</codeblock>
           impala-shell will incorrectly report that the failed metadata refresh completed successfully.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-611" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6871,7 +6871,7 @@ hive&gt; NULL</codeblock>
 
       <conbody>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-601" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6901,7 +6901,7 @@ hive&gt; NULL</codeblock>
           table will return an <codeph>unknown table</codeph> error message, even if the table is known.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-298" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6925,7 +6925,7 @@ hive&gt; NULL</codeblock>
           even if the metadata for that table is fixed.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-298" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6947,7 +6947,7 @@ hive&gt; NULL</codeblock>
           Attempting to select from these tables fails.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-581" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -6970,7 +6970,7 @@ hive&gt; NULL</codeblock>
           any of the joined tables in the WHERE clause.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-137" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -7002,7 +7002,7 @@ hive&gt; NULL</codeblock>
 
 <codeblock>SELECT * FROM (SELECT sum(col1) FROM some_table GROUP BY col1) t1 JOIN other_table ON (...);</codeblock>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-491" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -7026,7 +7026,7 @@ hive&gt; NULL</codeblock>
 
 <codeblock>INSERT OVERWRITE TABLE test SELECT * FROM test2 LIMIT 1;</codeblock>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-497" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -7050,7 +7050,7 @@ hive&gt; NULL</codeblock>
 
 <codeblock>SELECT * FROM test2 LIMIT 1;</codeblock>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-535" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -7072,7 +7072,7 @@ hive&gt; NULL</codeblock>
           Attempting to read such files does not generate a diagnostic.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-270" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
@@ -7095,7 +7095,7 @@ hive&gt; NULL</codeblock>
           exception.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b>
           <xref href="https://jira.cloudera.com/browse/IMP-522" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_grant.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_grant.xml b/docs/topics/impala_grant.xml
index 32b08b4..cdba5e0 100644
--- a/docs/topics/impala_grant.xml
+++ b/docs/topics/impala_grant.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">GRANT statement</indexterm>
+      <indexterm audience="hidden">GRANT statement</indexterm>
 <!-- Copied from Sentry docs. Turn into conref. I did some rewording for clarity. -->
       The <codeph>GRANT</codeph> statement grants roles or privileges on specified objects to groups. Only Sentry
       administrative users can grant roles to a group.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_group_concat.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_group_concat.xml b/docs/topics/impala_group_concat.xml
index 0cdeebd..ffa4084 100644
--- a/docs/topics/impala_group_concat.xml
+++ b/docs/topics/impala_group_concat.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">group_concat() function</indexterm>
+      <indexterm audience="hidden">group_concat() function</indexterm>
       An aggregate function that returns a single string representing the argument value concatenated together for
       each row of the result set. If the optional separator string is specified, the separator is added between
       each pair of concatenated values. The default separator is a comma followed by a space.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_hbase.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_hbase.xml b/docs/topics/impala_hbase.xml
index 6f7daf3..429d828 100644
--- a/docs/topics/impala_hbase.xml
+++ b/docs/topics/impala_hbase.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">HBase</indexterm>
+      <indexterm audience="hidden">HBase</indexterm>
       You can use Impala to query HBase tables. This capability allows convenient access to a storage system that
       is tuned for different kinds of workloads than the default with Impala. The default Impala tables use data
       files stored on HDFS, which are ideal for bulk loads and queries using full-table scans. In contrast, HBase
@@ -162,7 +162,7 @@ under the License.
 &lt;/property&gt;
 </codeblock>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         Currently, Cloudera Manager does not have an Impala-only override for HBase settings, so any HBase
         configuration change you make through Cloudera Manager would take affect for all HBase applications.
         Therefore, this change is not recommended on systems managed by Cloudera Manager.
@@ -630,7 +630,7 @@ set hbase_caching=1000;
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="hbase_create_new">
+  <concept audience="hidden" id="hbase_create_new">
 
     <title>Creating a New HBase Table for Impala to Use</title>
 
@@ -647,7 +647,7 @@ set hbase_caching=1000;
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="hbase_reuse_existing">
+  <concept audience="hidden" id="hbase_reuse_existing">
 
     <title>Associate Impala with an Existing HBase Table</title>
 
@@ -664,7 +664,7 @@ set hbase_caching=1000;
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="hbase_column_families">
+  <concept audience="hidden" id="hbase_column_families">
 
     <title>Map HBase Columns and Column Families to Impala Columns</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_hbase_cache_blocks.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_hbase_cache_blocks.xml b/docs/topics/impala_hbase_cache_blocks.xml
index 692d605..5bc3e16 100644
--- a/docs/topics/impala_hbase_cache_blocks.xml
+++ b/docs/topics/impala_hbase_cache_blocks.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">HBASE_CACHE_BLOCKS query option</indexterm>
+      <indexterm audience="hidden">HBASE_CACHE_BLOCKS query option</indexterm>
       Setting this option is equivalent to calling the
         <codeph>setCacheBlocks</codeph> method of the class <xref
         href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_hbase_caching.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_hbase_caching.xml b/docs/topics/impala_hbase_caching.xml
index ae8257f..db0a411 100644
--- a/docs/topics/impala_hbase_caching.xml
+++ b/docs/topics/impala_hbase_caching.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">HBASE_CACHING query option</indexterm>
+      <indexterm audience="hidden">HBASE_CACHING query option</indexterm>
       Setting this option is equivalent to calling the
         <codeph>setCaching</codeph> method of the class <xref
         href="http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/client/Scan.html"

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_hints.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_hints.xml b/docs/topics/impala_hints.xml
index 6430400..7d833f6 100644
--- a/docs/topics/impala_hints.xml
+++ b/docs/topics/impala_hints.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">hints</indexterm>
+      <indexterm audience="hidden">hints</indexterm>
       The Impala SQL dialect supports query hints, for fine-tuning the inner workings of queries. Specify hints as
       a temporary workaround for expensive queries, where missing statistics or other factors cause inefficient
       performance.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_impala_shell.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_impala_shell.xml b/docs/topics/impala_impala_shell.xml
index e833c8c..53531a3 100644
--- a/docs/topics/impala_impala_shell.xml
+++ b/docs/topics/impala_impala_shell.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">impala-shell</indexterm>
+      <indexterm audience="hidden">impala-shell</indexterm>
       You can use the Impala shell tool (<codeph>impala-shell</codeph>) to set up databases and tables, insert
       data, and issue queries. For ad hoc queries and exploration, you can submit SQL statements in an interactive
       session. To automate your work, you can specify command-line options to process a single statement or a

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_incompatible_changes.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_incompatible_changes.xml b/docs/topics/impala_incompatible_changes.xml
index 9fad426..b4cba7a 100644
--- a/docs/topics/impala_incompatible_changes.xml
+++ b/docs/topics/impala_incompatible_changes.xml
@@ -332,7 +332,7 @@ under the License.
       <note conref="../shared/impala_common.xml#common/impala_llama_obsolete"/>
 
       <ul>
-        <li rev="IMPALA-2005" audience="Cloudera">
+        <li rev="IMPALA-2005" audience="hidden">
           <p>
             If a <codeph>CREATE TABLE AS SELECT</codeph> operation fails while data is being inserted,
             the table is automatically removed. Previously, the table was left behind with no data.
@@ -1276,7 +1276,7 @@ select * from `cross`;</codeblock>
         to Impala 1.2.x in general.
       </p>
 
-      <p audience="Cloudera" conref="../shared/impala_common.xml#common/cm48_upgrade"/>
+      <p audience="hidden" conref="../shared/impala_common.xml#common/cm48_upgrade"/>
 
 <!-- <note conref="common.xml#common/cdh4_cdh5_upgrade"/> -->
     </conbody>
@@ -1318,7 +1318,7 @@ select * from `cross`;</codeblock>
         <li/>
       </ul>
 
-      <p audience="Cloudera" conref="../shared/impala_common.xml#common/cm48_upgrade"/>
+      <p audience="hidden" conref="../shared/impala_common.xml#common/cm48_upgrade"/>
 
 <!--  <note conref="common.xml#common/cdh4_cdh5_upgrade"/> -->
     </conbody>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_insert.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_insert.xml b/docs/topics/impala_insert.xml
index 834d946..ed9f78f 100644
--- a/docs/topics/impala_insert.xml
+++ b/docs/topics/impala_insert.xml
@@ -41,7 +41,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">INSERT statement</indexterm>
+      <indexterm audience="hidden">INSERT statement</indexterm>
       Impala supports inserting into tables and partitions that you create with the Impala <codeph>CREATE
       TABLE</codeph> statement, or pre-defined tables and partitions created through Hive.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_install.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_install.xml b/docs/topics/impala_install.xml
index 7daf38d..adb502c 100644
--- a/docs/topics/impala_install.xml
+++ b/docs/topics/impala_install.xml
@@ -32,15 +32,15 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">installation</indexterm>
-      <indexterm audience="Cloudera">pseudo-distributed cluster</indexterm>
-      <indexterm audience="Cloudera">cluster</indexterm>
-      <indexterm audience="Cloudera">DataNodes</indexterm>
-      <indexterm audience="Cloudera">NameNode</indexterm>
-      <indexterm audience="Cloudera">Cloudera Manager</indexterm>
-      <indexterm audience="Cloudera">impalad</indexterm>
-      <indexterm audience="Cloudera">impala-shell</indexterm>
-      <indexterm audience="Cloudera">statestored</indexterm>
+      <indexterm audience="hidden">installation</indexterm>
+      <indexterm audience="hidden">pseudo-distributed cluster</indexterm>
+      <indexterm audience="hidden">cluster</indexterm>
+      <indexterm audience="hidden">DataNodes</indexterm>
+      <indexterm audience="hidden">NameNode</indexterm>
+      <indexterm audience="hidden">Cloudera Manager</indexterm>
+      <indexterm audience="hidden">impalad</indexterm>
+      <indexterm audience="hidden">impala-shell</indexterm>
+      <indexterm audience="hidden">statestored</indexterm>
       Impala is an open-source add-on to the Cloudera Enterprise Core that returns rapid responses to
       queries.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_intro.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_intro.xml b/docs/topics/impala_intro.xml
index e3b851d..2389537 100644
--- a/docs/topics/impala_intro.xml
+++ b/docs/topics/impala_intro.xml
@@ -82,7 +82,7 @@ under the License.
 
     <conbody>
 
-      <p audience="Cloudera" conref="../shared/impala_common.xml#common/impala_overview_diagram"/>
+      <p audience="hidden" conref="../shared/impala_common.xml#common/impala_overview_diagram"/>
 
       <p conref="../shared/impala_common.xml#common/component_list"/>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_invalidate_metadata.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_invalidate_metadata.xml b/docs/topics/impala_invalidate_metadata.xml
index 7ec724a..97dfb09 100644
--- a/docs/topics/impala_invalidate_metadata.xml
+++ b/docs/topics/impala_invalidate_metadata.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">INVALIDATE METADATA statement</indexterm>
+      <indexterm audience="hidden">INVALIDATE METADATA statement</indexterm>
       Marks the metadata for one or all tables as stale. Required after a table is created through the Hive shell,
       before the table is available for Impala queries. The next time the current Impala node performs a query
       against a table whose metadata is invalidated, Impala reloads the associated metadata before the query

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_isilon.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_isilon.xml b/docs/topics/impala_isilon.xml
index c0459dd..bf93b39 100644
--- a/docs/topics/impala_isilon.xml
+++ b/docs/topics/impala_isilon.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">Isilon</indexterm>
+      <indexterm audience="hidden">Isilon</indexterm>
       You can use Impala to query data files that reside on EMC Isilon storage devices, rather than in HDFS.
       This capability allows convenient query access to a storage system where you might already be
       managing large volumes of data. The combination of the Impala query engine and Isilon storage is
@@ -102,7 +102,7 @@ CREATE TABLE d1.t2 (a TINYINT, b BOOLEAN);
       for the <cmdname>impalad</cmdname> daemon on clusters not using Cloudera Manager.
     </p>
 
-    <p audience="Cloudera">
+    <p audience="hidden">
 <!--
       For information about tasks performed on
       Isilon OneFS, see the information hub for Cloudera on the EMC Community Network:
@@ -116,7 +116,7 @@ CREATE TABLE d1.t2 (a TINYINT, b BOOLEAN);
 
     <!-- <p outputclass="toc inpage"/> -->
   </conbody>
-<concept id="isilon_cm_configs" audience="Cloudera">
+<concept id="isilon_cm_configs" audience="hidden">
 <title>Required Configurations</title>
 <conbody>
 <p>Specify the following configurations in Cloudera Manager on the <menucascade><uicontrol>Clusters</uicontrol><uicontrol><varname>Isilon Service</varname></uicontrol><uicontrol>Configuration</uicontrol></menucascade> tab:<ul id="ul_vpx_bw5_vv">

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_jdbc.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_jdbc.xml b/docs/topics/impala_jdbc.xml
index 5112e05..62fa285 100644
--- a/docs/topics/impala_jdbc.xml
+++ b/docs/topics/impala_jdbc.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">JDBC</indexterm>
+      <indexterm audience="hidden">JDBC</indexterm>
       Impala supports the standard JDBC interface, allowing access from commercial Business Intelligence tools and
       custom software written in Java or other programming languages. The JDBC driver allows you to access Impala
       from a Java program that you write, or a Business Intelligence or similar tool that uses JDBC to communicate

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_joins.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_joins.xml b/docs/topics/impala_joins.xml
index 7ddd829..ef0a67a 100644
--- a/docs/topics/impala_joins.xml
+++ b/docs/topics/impala_joins.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">joins</indexterm>
+      <indexterm audience="hidden">joins</indexterm>
       A join query is a <codeph>SELECT</codeph> statement that combines data from two or more tables,
       and returns a result set containing items from some or all of those tables. It is a way to
       cross-reference and correlate related data that is organized into multiple tables, typically

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_known_issues.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_known_issues.xml b/docs/topics/impala_known_issues.xml
index e97290b..72c9b12 100644
--- a/docs/topics/impala_known_issues.xml
+++ b/docs/topics/impala_known_issues.xml
@@ -412,7 +412,7 @@ https://issues.cloudera.org/browse/IMPALA-2144 - Don't have
 
 <!-- To do: Hiding for the moment. https://jira.cloudera.com/browse/CDH-38736 reports the issue is fixed. -->
 
-    <concept id="impala-shell_ssl_dependency" audience="Cloudera" rev="impala-shell_ssl_dependency">
+    <concept id="impala-shell_ssl_dependency" audience="hidden" rev="impala-shell_ssl_dependency">
 
       <title>impala-shell requires Python with ssl module</title>
 
@@ -572,7 +572,7 @@ impala-shell -s impala --ssl --ca_cert /path_to_truststore/truststore.pem
           in out-of-memory errors in catalogd and leading to query failures.
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Bug:</b> <xref href="https://jira.cloudera.com/browse/TSB-168" scope="external" format="html">TSB-168</xref>
         </p>
 
@@ -1377,7 +1377,7 @@ ALTER TABLE table_name SET TBLPROPERTIES('EXTERNAL'='TRUE');
           table. See <xref href="impala_tables.xml#tables"/> for the differences between internal and external tables.
         </note>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Bug:</b> <xref href="https://issues.cloudera.org/browse/CDH-41605" scope="external" format="html">CDH-41605</xref>
         </p>
 
@@ -1397,7 +1397,7 @@ ALTER TABLE table_name SET TBLPROPERTIES('EXTERNAL'='TRUE');
 
       <conbody>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b> <xref href="https://jira.cloudera.com/browse/IMP-469" scope="external" format="html"/>; KI added 0.1
           <i>Cloudera internal only</i>
         </p>
@@ -1427,7 +1427,7 @@ ALTER TABLE table_name SET TBLPROPERTIES('EXTERNAL'='TRUE');
           allowed value of type (Hive returns NULL).
         </p>
 
-        <p audience="Cloudera">
+        <p audience="hidden">
           <b>Cloudera Bug:</b> <xref href="https://jira.cloudera.com/browse/IMP-175" scope="external" format="html">IMPALA-175</xref> ; KI
           added 0.1 <i>Cloudera internal only</i>
         </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_kudu.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_kudu.xml b/docs/topics/impala_kudu.xml
index 186336b..ef90e63 100644
--- a/docs/topics/impala_kudu.xml
+++ b/docs/topics/impala_kudu.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">Kudu</indexterm>
+      <indexterm audience="hidden">Kudu</indexterm>
       You can use Impala to query Kudu tables. This capability allows convenient access to a storage system that is
       tuned for different kinds of workloads than the default with Impala. The default Impala tables use data files
       stored on HDFS, which are ideal for bulk loads and queries using full-table scans. In contrast, Kudu can do

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_langref_unsupported.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_langref_unsupported.xml b/docs/topics/impala_langref_unsupported.xml
index 25d995c..f0c326a 100644
--- a/docs/topics/impala_langref_unsupported.xml
+++ b/docs/topics/impala_langref_unsupported.xml
@@ -35,8 +35,8 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">Hive</indexterm>
-      <indexterm audience="Cloudera">HiveQL</indexterm>
+      <indexterm audience="hidden">Hive</indexterm>
+      <indexterm audience="hidden">HiveQL</indexterm>
       Impala's SQL syntax follows the SQL-92 standard, and includes many industry extensions in areas such as
       built-in functions. See <xref href="impala_porting.xml#porting"/> for a general discussion of adapting SQL
       code from a variety of database systems to Impala.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_ldap.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_ldap.xml b/docs/topics/impala_ldap.xml
index 3100088..e2f48fa 100644
--- a/docs/topics/impala_ldap.xml
+++ b/docs/topics/impala_ldap.xml
@@ -189,7 +189,7 @@ under the License.
         specify the option on the <cmdname>impalad</cmdname> command line.
       </p>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         For clusters managed by Cloudera Manager 5.4.0 and higher,
         search for the configuration field names <codeph>ldap_domain</codeph>,
         <codeph>ldap_basedn</codeph>, or <codeph>ldap_bind_pattern</codeph>,
@@ -303,7 +303,7 @@ username, for example <codeph>username@example.com</codeph>.
       </metadata>
     </prolog>
     <conbody>
-      <section id="ldap_impala_hue_cm" audience="Cloudera">
+      <section id="ldap_impala_hue_cm" audience="hidden">
         <title>Enabling LDAP for Impala in Hue Using Cloudera Manager</title>
         <p>
           <ol>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_lineage.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_lineage.xml b/docs/topics/impala_lineage.xml
index 59c0aa3..b8b76b7 100644
--- a/docs/topics/impala_lineage.xml
+++ b/docs/topics/impala_lineage.xml
@@ -40,8 +40,8 @@ under the License.
   <conbody>
 
     <p rev="2.2.0">
-      <indexterm audience="Cloudera">lineage</indexterm>
-      <indexterm audience="Cloudera">column lineage</indexterm>
+      <indexterm audience="hidden">lineage</indexterm>
+      <indexterm audience="hidden">column lineage</indexterm>
       <term>Lineage</term> is a feature in the Cloudera Navigator data
       management component that helps you track where data originated, and how
       data propagates through the system through SQL statements such as

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_literals.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_literals.xml b/docs/topics/impala_literals.xml
index 3d979d6..d0d475c 100644
--- a/docs/topics/impala_literals.xml
+++ b/docs/topics/impala_literals.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">literals</indexterm>
+      <indexterm audience="hidden">literals</indexterm>
       Each of the Impala data types has corresponding notation for literal values of that type. You specify literal
       values in SQL statements, such as in the <codeph>SELECT</codeph> list or <codeph>WHERE</codeph> clause of a
       query, or as an argument to a function call. See <xref href="impala_datatypes.xml#datatypes"/> for a complete
@@ -51,7 +51,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">numeric literals</indexterm>
+        <indexterm audience="hidden">numeric literals</indexterm>
         To write literals for the integer types (<codeph>TINYINT</codeph>, <codeph>SMALLINT</codeph>,
         <codeph>INT</codeph>, and <codeph>BIGINT</codeph>), use a sequence of digits with optional leading zeros.
       </p>
@@ -146,7 +146,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">string literals</indexterm>
+        <indexterm audience="hidden">string literals</indexterm>
         String literals are quoted using either single or double quotation marks. You can use either kind of quotes
         for string literals, even both kinds for different literals within the same statement.
       </p>
@@ -320,7 +320,7 @@ select case bool_col when true then 'yes' when false 'no' else 'null' end from t
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">NULL</indexterm>
+        <indexterm audience="hidden">NULL</indexterm>
         The notion of <codeph>NULL</codeph> values is familiar from all kinds of database systems, but each SQL
         dialect can have its own behavior and restrictions on <codeph>NULL</codeph> values. For Big Data
         processing, the precise semantics of <codeph>NULL</codeph> values are significant: any misunderstanding

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_live_progress.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_live_progress.xml b/docs/topics/impala_live_progress.xml
index 3f70d86..0c91824 100644
--- a/docs/topics/impala_live_progress.xml
+++ b/docs/topics/impala_live_progress.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p rev="2.3.0">
-      <indexterm audience="Cloudera">LIVE_PROGRESS query option</indexterm>
+      <indexterm audience="hidden">LIVE_PROGRESS query option</indexterm>
       For queries submitted through the <cmdname>impala-shell</cmdname> command,
       displays an interactive progress bar showing roughly what percentage of
       processing has been completed. When the query finishes, the progress bar is erased

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_live_summary.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_live_summary.xml b/docs/topics/impala_live_summary.xml
index d37877b..94733d2 100644
--- a/docs/topics/impala_live_summary.xml
+++ b/docs/topics/impala_live_summary.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p rev="2.3.0">
-      <indexterm audience="Cloudera">LIVE_SUMMARY query option</indexterm>
+      <indexterm audience="hidden">LIVE_SUMMARY query option</indexterm>
       For queries submitted through the <cmdname>impala-shell</cmdname> command,
       displays the same output as the <codeph>SUMMARY</codeph> command,
       with the measurements updated in real time as the query progresses.
@@ -142,7 +142,7 @@ LIVE_SUMMARY set to true
 </codeblock>
 
 <!-- Keeping this sample output that illustrates a couple of glitches in the LIVE_SUMMARY display, hidden, to help filing JIRAs. -->
-<codeblock audience="Cloudera"><![CDATA[[
+<codeblock audience="hidden"><![CDATA[[
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+
 | Operator            | #Hosts | Avg Time | Max Time | #Rows   | Est. #Rows | Peak Mem | Est. Peak Mem | Detail                |
 +---------------------+--------+----------+----------+---------+------------+----------+---------------+-----------------------+

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_load_data.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_load_data.xml b/docs/topics/impala_load_data.xml
index 5442134..8d027f3 100644
--- a/docs/topics/impala_load_data.xml
+++ b/docs/topics/impala_load_data.xml
@@ -40,7 +40,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">LOAD DATA statement</indexterm>
+      <indexterm audience="hidden">LOAD DATA statement</indexterm>
       The <codeph>LOAD DATA</codeph> statement streamlines the ETL process for an internal Impala table by moving a
       data file or all the data files in a directory from an HDFS location into the Impala data directory for that
       table.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_logging.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_logging.xml b/docs/topics/impala_logging.xml
index 40399ef..92b9c88 100644
--- a/docs/topics/impala_logging.xml
+++ b/docs/topics/impala_logging.xml
@@ -167,7 +167,7 @@ under the License.
           immediately.
         </li>
 
-        <li audience="Cloudera">
+        <li audience="hidden">
           Cloudera Manager has an Impala configuration setting that sets the <codeph>-logbuflevel</codeph> startup
           option.
         </li>
@@ -189,7 +189,7 @@ under the License.
 
     <conbody>
 
-      <p rev="upstream" audience="Cloudera"><!-- Whole paragraph can probably go. -->
+      <p rev="upstream" audience="hidden"><!-- Whole paragraph can probably go. -->
         <ph rev="upstream">Cloudera</ph> recommends installing Impala through the Cloudera Manager administration interface. To assist with
         troubleshooting, Cloudera Manager collects front-end and back-end logs together into a single view, and let
         you do a search across log data for all the managed nodes rather than examining the logs on each node
@@ -250,7 +250,7 @@ under the License.
         Linux tool or technique of choice. A value of 1 preserves only the very latest log file.
       </p>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         To set up log rotation on a system managed by Cloudera Manager 5.4.0 and higher, search for the
         <codeph>max_log_files</codeph> option name and set the appropriate value for the <userinput>Maximum Log
         Files</userinput> field for each Impala configuration category (Impala, Catalog Server, and StateStore).
@@ -459,7 +459,7 @@ I0107 08:42:12.292706 14876 logging.cc:76] Flags (see also /varz are on debug we
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">redaction</indexterm>
+        <indexterm audience="hidden">redaction</indexterm>
         <term>Log redaction</term> is a security feature that prevents sensitive information from being displayed in
         locations used by administrators for monitoring and troubleshooting, such as log files, the Cloudera Manager
         user interface, and the Impala debug web user interface. You configure regular expressions that match


[3/6] incubator-impala git commit: Global search/replace: audience="Cloudera" -> audience="hidden".

Posted by jr...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_bloom_filter_size.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_bloom_filter_size.xml b/docs/topics/impala_runtime_bloom_filter_size.xml
index 469b144..644dda7 100644
--- a/docs/topics/impala_runtime_bloom_filter_size.xml
+++ b/docs/topics/impala_runtime_bloom_filter_size.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">RUNTIME_BLOOM_FILTER_SIZE query option</indexterm>
+      <indexterm audience="hidden">RUNTIME_BLOOM_FILTER_SIZE query option</indexterm>
       Size (in bytes) of Bloom filter data structure used by the runtime filtering
       feature.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_filter_max_size.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_filter_max_size.xml b/docs/topics/impala_runtime_filter_max_size.xml
index dde24de..6349437 100644
--- a/docs/topics/impala_runtime_filter_max_size.xml
+++ b/docs/topics/impala_runtime_filter_max_size.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-3480">
-      <indexterm audience="Cloudera">RUNTIME_FILTER_MAX_SIZE query option</indexterm>
+      <indexterm audience="hidden">RUNTIME_FILTER_MAX_SIZE query option</indexterm>
       The <codeph>RUNTIME_FILTER_MAX_SIZE</codeph> query option
       adjusts the settings for the runtime filtering feature.
       This option defines the maximum size for a filter,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_filter_min_size.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_filter_min_size.xml b/docs/topics/impala_runtime_filter_min_size.xml
index 3ea0e74..9d673c2 100644
--- a/docs/topics/impala_runtime_filter_min_size.xml
+++ b/docs/topics/impala_runtime_filter_min_size.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-3480">
-      <indexterm audience="Cloudera">RUNTIME_FILTER_MIN_SIZE query option</indexterm>
+      <indexterm audience="hidden">RUNTIME_FILTER_MIN_SIZE query option</indexterm>
       The <codeph>RUNTIME_FILTER_MIN_SIZE</codeph> query option
       adjusts the settings for the runtime filtering feature.
       This option defines the minimum size for a filter,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_filter_mode.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_filter_mode.xml b/docs/topics/impala_runtime_filter_mode.xml
index 5bdaca0..7797119 100644
--- a/docs/topics/impala_runtime_filter_mode.xml
+++ b/docs/topics/impala_runtime_filter_mode.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">RUNTIME_FILTER_MODE query option</indexterm>
+      <indexterm audience="hidden">RUNTIME_FILTER_MODE query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_filter_wait_time_ms.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_filter_wait_time_ms.xml b/docs/topics/impala_runtime_filter_wait_time_ms.xml
index 1298c72..51b4806 100644
--- a/docs/topics/impala_runtime_filter_wait_time_ms.xml
+++ b/docs/topics/impala_runtime_filter_wait_time_ms.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">RUNTIME_FILTER_WAIT_TIME_MS query option</indexterm>
+      <indexterm audience="hidden">RUNTIME_FILTER_WAIT_TIME_MS query option</indexterm>
       The <codeph>RUNTIME_FILTER_WAIT_TIME_MS</codeph> query option
       adjusts the settings for the runtime filtering feature.
       It specifies a time in milliseconds that each scan node waits for

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_runtime_filtering.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_runtime_filtering.xml b/docs/topics/impala_runtime_filtering.xml
index 3ec50ac..aceecfc 100644
--- a/docs/topics/impala_runtime_filtering.xml
+++ b/docs/topics/impala_runtime_filtering.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">runtime filtering</indexterm>
+      <indexterm audience="hidden">runtime filtering</indexterm>
       <term>Runtime filtering</term> is a wide-ranging optimization feature available in
       <keyword keyref="impala25_full"/> and higher. When only a fraction of the data in a table is
       needed for a query against a partitioned table or to evaluate a join condition,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_s3.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_s3.xml b/docs/topics/impala_s3.xml
index 6011862..5a3dcf4 100644
--- a/docs/topics/impala_s3.xml
+++ b/docs/topics/impala_s3.xml
@@ -39,9 +39,9 @@ under the License.
     <note conref="../shared/impala_common.xml#common/s3_production"/>
 
     <p rev="2.2.0">
-      <indexterm audience="Cloudera">S3 with Impala</indexterm>
+      <indexterm audience="hidden">S3 with Impala</indexterm>
 
-      <indexterm audience="Cloudera">Amazon S3 with Impala</indexterm>
+      <indexterm audience="hidden">Amazon S3 with Impala</indexterm>
       You can use Impala to query data residing on the Amazon S3 filesystem. This capability allows convenient
       access to a storage system that is remotely managed, accessible from anywhere, and integrated with various
       cloud-based services. Impala can query files in any supported file format from S3. The S3 storage location
@@ -126,10 +126,10 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">fs.s3a.access.key configuration setting</indexterm>
-        <indexterm audience="Cloudera">fs.s3a.secret.key configuration setting</indexterm>
-        <indexterm audience="Cloudera">access.key configuration setting</indexterm>
-        <indexterm audience="Cloudera">secret.key configuration setting</indexterm>
+        <indexterm audience="hidden">fs.s3a.access.key configuration setting</indexterm>
+        <indexterm audience="hidden">fs.s3a.secret.key configuration setting</indexterm>
+        <indexterm audience="hidden">access.key configuration setting</indexterm>
+        <indexterm audience="hidden">secret.key configuration setting</indexterm>
         To allow Impala to access data in S3, specify values for the following configuration settings in your
         <filepath>core-site.xml</filepath> file:
       </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_s3_skip_insert_staging.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_s3_skip_insert_staging.xml b/docs/topics/impala_s3_skip_insert_staging.xml
index 809ec40..82e8ec5 100644
--- a/docs/topics/impala_s3_skip_insert_staging.xml
+++ b/docs/topics/impala_s3_skip_insert_staging.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p rev="2.6.0 IMPALA-3452 CDH-39913">
-      <indexterm audience="Cloudera">IMPALA_S3_SKIP_INSERT_STAGING query option</indexterm>
+      <indexterm audience="hidden">IMPALA_S3_SKIP_INSERT_STAGING query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_scalability.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_scalability.xml b/docs/topics/impala_scalability.xml
index 69bd08f..9f7ce3c 100644
--- a/docs/topics/impala_scalability.xml
+++ b/docs/topics/impala_scalability.xml
@@ -50,7 +50,7 @@ under the License.
 
   </conbody>
 
-  <concept audience="Cloudera" id="scalability_memory">
+  <concept audience="hidden" id="scalability_memory">
 
     <title>Overview and Guidelines for Impala Memory Usage</title>
   <prolog>
@@ -167,7 +167,7 @@ Memory Usage: Additional Notes
 
     <conbody>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         Details to fill in in future: Impact of <q>load catalog in background</q> option.
         Changing timeouts. Related Cloudera Manager settings.
       </p>
@@ -310,7 +310,7 @@ Memory Usage: Additional Notes
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="scalability_cluster_size">
+  <concept audience="hidden" id="scalability_cluster_size">
 
     <title>Scalability Considerations for Impala Cluster Size and Topology</title>
 
@@ -321,7 +321,7 @@ Memory Usage: Additional Notes
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="concurrent_connections">
+  <concept audience="hidden" id="concurrent_connections">
 
     <title>Scaling the Number of Concurrent Connections</title>
 
@@ -469,7 +469,7 @@ Memory Usage: Additional Notes
               query.
             </li>
 
-            <li audience="Cloudera">
+            <li audience="hidden">
               The <uicontrol>Impala Queries</uicontrol> dialog in Cloudera Manager. You can see the peak memory
               usage for a query, combined across all nodes in the cluster.
             </li>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_scan_node_codegen_threshold.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_scan_node_codegen_threshold.xml b/docs/topics/impala_scan_node_codegen_threshold.xml
index eb8d47d..40d1bc6 100644
--- a/docs/topics/impala_scan_node_codegen_threshold.xml
+++ b/docs/topics/impala_scan_node_codegen_threshold.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0 IMPALA-1755">
-      <indexterm audience="Cloudera">SCAN_NODE_CODEGEN_THRESHOLD query option</indexterm>
+      <indexterm audience="hidden">SCAN_NODE_CODEGEN_THRESHOLD query option</indexterm>
       The <codeph>SCAN_NODE_CODEGEN_THRESHOLD</codeph> query option
       adjusts the aggressiveness of the code generation optimization process
       when performing I/O read operations. It can help to work around performance problems

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_schedule_random_replica.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_schedule_random_replica.xml b/docs/topics/impala_schedule_random_replica.xml
index b74051e..c104990 100644
--- a/docs/topics/impala_schedule_random_replica.xml
+++ b/docs/topics/impala_schedule_random_replica.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">SCHEDULE_RANDOM_REPLICA query option</indexterm>
+      <indexterm audience="hidden">SCHEDULE_RANDOM_REPLICA query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_schema_objects.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_schema_objects.xml b/docs/topics/impala_schema_objects.xml
index ef5c54d..dc4c16f 100644
--- a/docs/topics/impala_schema_objects.xml
+++ b/docs/topics/impala_schema_objects.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">schema objects</indexterm>
+      <indexterm audience="hidden">schema objects</indexterm>
       With Impala, you work with schema objects that are familiar to database users: primarily databases, tables, views,
       and functions. The SQL syntax to work with these objects is explained in
       <xref href="impala_langref_sql.xml#langref_sql"/>. This section explains the conceptual knowledge you need to

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_select.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_select.xml b/docs/topics/impala_select.xml
index 0d68ed2..efd3e24 100644
--- a/docs/topics/impala_select.xml
+++ b/docs/topics/impala_select.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SELECT statement</indexterm>
+      <indexterm audience="hidden">SELECT statement</indexterm>
       The <codeph>SELECT</codeph> statement performs queries, retrieving data from one or more tables and producing
       result sets consisting of rows and columns.
     </p>
@@ -208,7 +208,7 @@ LIMIT <i>expression</i> [OFFSET <i>expression</i>]
     </p>
   </conbody>
 
-  <concept id="where" audience="Cloudera">
+  <concept id="where" audience="hidden">
 
 <!-- WHERE hidden for the moment until there's the chance to add some reasonably comprehensive content
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_seq_compression_mode.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_seq_compression_mode.xml b/docs/topics/impala_seq_compression_mode.xml
index 1f2b40b..09b6fd5 100644
--- a/docs/topics/impala_seq_compression_mode.xml
+++ b/docs/topics/impala_seq_compression_mode.xml
@@ -33,7 +33,7 @@ under the License.
   <conbody>
 
     <p rev="2.5.0">
-      <indexterm audience="Cloudera">RM_INITIAL_MEM query option</indexterm>
+      <indexterm audience="hidden">RM_INITIAL_MEM query option</indexterm>
     </p>
 
     <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_seqfile.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_seqfile.xml b/docs/topics/impala_seqfile.xml
index 63d6587..7143530 100644
--- a/docs/topics/impala_seqfile.xml
+++ b/docs/topics/impala_seqfile.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SequenceFile support in Impala</indexterm>
+      <indexterm audience="hidden">SequenceFile support in Impala</indexterm>
       Impala supports using SequenceFile data files.
     </p>
 
@@ -161,7 +161,7 @@ Returned 3 row(s) in 0.23s</codeblock>
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">compression</indexterm>
+        <indexterm audience="hidden">compression</indexterm>
         You may want to enable compression on existing tables. Enabling compression provides performance gains in
         most cases and is supported for SequenceFile tables. For example, to enable Snappy compression, you would
         specify the following additional settings when loading data through the Hive shell:
@@ -225,7 +225,7 @@ hive&gt; INSERT OVERWRITE TABLE tbl_seq PARTITION(year) SELECT * FROM tbl;</code
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="seqfile_data_types">
+  <concept audience="hidden" id="seqfile_data_types">
 
     <title>Data Type Considerations for SequenceFile Tables</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_set.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_set.xml b/docs/topics/impala_set.xml
index 026c6df..c3efd90 100644
--- a/docs/topics/impala_set.xml
+++ b/docs/topics/impala_set.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">SET statement</indexterm>
+      <indexterm audience="hidden">SET statement</indexterm>
       Specifies values for query options that control the runtime behavior of other statements within the same
       session.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_shell_commands.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_shell_commands.xml b/docs/topics/impala_shell_commands.xml
index 253edf4..a7b4c75 100644
--- a/docs/topics/impala_shell_commands.xml
+++ b/docs/topics/impala_shell_commands.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">impala-shell</indexterm>
+      <indexterm audience="hidden">impala-shell</indexterm>
       Use the following commands within <codeph>impala-shell</codeph> to pass requests to the
       <codeph>impalad</codeph> daemon that the shell is connected to. You can enter a command interactively at the
       prompt, or pass it as the argument to the <codeph>-q</codeph> option of <codeph>impala-shell</codeph>. Most

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_show.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_show.xml b/docs/topics/impala_show.xml
index 2ece7cf..c5a7740 100644
--- a/docs/topics/impala_show.xml
+++ b/docs/topics/impala_show.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SHOW statement</indexterm>
+      <indexterm audience="hidden">SHOW statement</indexterm>
       The <codeph>SHOW</codeph> statement is a flexible way to get information about different types of Impala
       objects.
     </p>
@@ -60,7 +60,7 @@ SHOW GRANT ROLE <varname>role_name</varname></ph>
 <!-- SHOW ROLE GRANT { USER <varname>user_name</varname> | GROUP <varname>group_name</varname> | ROLE <varname>role_name</varname> } -->
 
 <!-- Extracted from the previous codeblock because even hidden content produces blank lines.
-<ph audience="Cloudera" rev="1.4.0">SHOW DATA SOURCES [LIKE '<varname>source_name</varname>]</ph>
+<ph audience="hidden" rev="1.4.0">SHOW DATA SOURCES [LIKE '<varname>source_name</varname>]</ph>
 -->
 
 <!-- Some suggestion there would be this syntax for 1.4, but it's not in the builds:
@@ -1255,7 +1255,7 @@ show functions in _impala_builtins like '*week*';
     </conbody>
   </concept>
 
-  <concept rev="someday" audience="Cloudera" id="show_data_sources">
+  <concept rev="someday" audience="hidden" id="show_data_sources">
 
     <title>SHOW DATA SOURCES Statement (CDH x.y and later only)</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_ssl.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_ssl.xml b/docs/topics/impala_ssl.xml
index 7375cfd..8f1e248 100644
--- a/docs/topics/impala_ssl.xml
+++ b/docs/topics/impala_ssl.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SSL</indexterm>
+      <indexterm audience="hidden">SSL</indexterm>
       Impala supports TLS/SSL network encryption, between Impala and client programs, and between the Impala-related daemons running on
       different nodes in the cluster. This feature is important when you also use other features such as Kerberos authentication or Sentry
       authorization, where credentials are being transmitted back and forth.
@@ -56,7 +56,7 @@ under the License.
 
   </conbody>
 
-  <concept id="concept_gnk_2tt_qp" audience="Cloudera">
+  <concept id="concept_gnk_2tt_qp" audience="hidden">
 
     <title>Using Cloudera Manager</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_stddev.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_stddev.xml b/docs/topics/impala_stddev.xml
index 4471ff3..29e1510 100644
--- a/docs/topics/impala_stddev.xml
+++ b/docs/topics/impala_stddev.xml
@@ -37,9 +37,9 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">stddev() function</indexterm>
-      <indexterm audience="Cloudera">stddev_samp() function</indexterm>
-      <indexterm audience="Cloudera">stddev_pop() function</indexterm>
+      <indexterm audience="hidden">stddev() function</indexterm>
+      <indexterm audience="hidden">stddev_samp() function</indexterm>
+      <indexterm audience="hidden">stddev_pop() function</indexterm>
       An aggregate function that
       <xref href="http://en.wikipedia.org/wiki/Standard_deviation" scope="external" format="html">standard
       deviation</xref> of a set of numbers.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_string_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_string_functions.xml b/docs/topics/impala_string_functions.xml
index acb9298..048daf4 100644
--- a/docs/topics/impala_string_functions.xml
+++ b/docs/topics/impala_string_functions.xml
@@ -76,7 +76,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">ascii() function</indexterm>
+          <indexterm audience="hidden">ascii() function</indexterm>
           <b>Purpose:</b> Returns the numeric ASCII code of the first character of the argument.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -93,7 +93,7 @@ under the License.
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">btrim() function</indexterm>
+          <indexterm audience="hidden">btrim() function</indexterm>
           <b>Purpose:</b> Removes all instances of one or more characters
           from the start and end of a <codeph>STRING</codeph> value.
           By default, removes only spaces.
@@ -151,8 +151,8 @@ select concat('[',btrim('xyhelxyzlozyzzxx','xyz'),']');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">char_length() function</indexterm>
-          <indexterm audience="Cloudera">character_length() function</indexterm>
+          <indexterm audience="hidden">char_length() function</indexterm>
+          <indexterm audience="hidden">character_length() function</indexterm>
           <b>Purpose:</b> Returns the length in characters of the argument string. Aliases for the
           <codeph>length()</codeph> function.
           <p>
@@ -169,7 +169,7 @@ select concat('[',btrim('xyhelxyzlozyzzxx','xyz'),']');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">chr() function</indexterm>
+          <indexterm audience="hidden">chr() function</indexterm>
           <b>Purpose:</b> Returns a character specified by a decimal code point value.
           The interpretation and display of the resulting character depends on your system locale.
           Because consistent processing of Impala string values is only guaranteed
@@ -208,7 +208,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">concat() function</indexterm>
+          <indexterm audience="hidden">concat() function</indexterm>
           <b>Purpose:</b> Returns a single string representing all the argument values joined together.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -225,7 +225,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">concat_ws() function</indexterm>
+          <indexterm audience="hidden">concat_ws() function</indexterm>
           <b>Purpose:</b> Returns a single string representing the second and following argument values joined
           together, delimited by a specified separator.
           <p>
@@ -243,7 +243,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">find_in_set() function</indexterm>
+          <indexterm audience="hidden">find_in_set() function</indexterm>
           <b>Purpose:</b> Returns the position (starting from 1) of the first occurrence of a specified string
           within a comma-separated string. Returns <codeph>NULL</codeph> if either argument is
           <codeph>NULL</codeph>, 0 if the search string is not found, or 0 if the search string contains a comma.
@@ -261,7 +261,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">group_concat() function</indexterm>
+          <indexterm audience="hidden">group_concat() function</indexterm>
           <b>Purpose:</b> Returns a single string representing the argument value concatenated together for each
           row of the result set. If the optional separator string is specified, the separator is added between each
           pair of concatenated values.
@@ -290,7 +290,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">initcap() function</indexterm>
+          <indexterm audience="hidden">initcap() function</indexterm>
           <b>Purpose:</b> Returns the input string with the first letter capitalized.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -306,7 +306,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">instr() function</indexterm>
+          <indexterm audience="hidden">instr() function</indexterm>
           <b>Purpose:</b> Returns the position (starting from 1) of the first occurrence of a substring within a
           longer string.
           <p>
@@ -323,7 +323,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">length() function</indexterm>
+          <indexterm audience="hidden">length() function</indexterm>
           <b>Purpose:</b> Returns the length in characters of the argument string.
           <p>
             <b>Return type:</b> <codeph>int</codeph>
@@ -339,7 +339,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">locate() function</indexterm>
+          <indexterm audience="hidden">locate() function</indexterm>
           <b>Purpose:</b> Returns the position (starting from 1) of the first occurrence of a substring within a
           longer string, optionally after a particular position.
           <p>
@@ -356,7 +356,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">lower() function</indexterm>
+          <indexterm audience="hidden">lower() function</indexterm>
           <b>Purpose:</b> Returns the argument string converted to all-lowercase.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -374,7 +374,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">lpad() function</indexterm>
+          <indexterm audience="hidden">lpad() function</indexterm>
           <b>Purpose:</b> Returns a string of a specified length, based on the first argument string. If the
           specified string is too short, it is padded on the left with a repeating sequence of the characters from
           the pad string. If the specified string is too long, it is truncated on the right.
@@ -392,7 +392,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">ltrim() function</indexterm>
+          <indexterm audience="hidden">ltrim() function</indexterm>
           <b>Purpose:</b> Returns the argument string with any leading spaces removed from the left side.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -408,7 +408,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">parse_url() function</indexterm>
+          <indexterm audience="hidden">parse_url() function</indexterm>
           <b>Purpose:</b> Returns the portion of a URL corresponding to a specified part. The part argument can be
           <codeph>'PROTOCOL'</codeph>, <codeph>'HOST'</codeph>, <codeph>'PATH'</codeph>, <codeph>'REF'</codeph>,
           <codeph>'AUTHORITY'</codeph>, <codeph>'FILE'</codeph>, <codeph>'USERINFO'</codeph>, or
@@ -436,7 +436,7 @@ SELECT chr(97);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">regexp_extract() function</indexterm>
+          <indexterm audience="hidden">regexp_extract() function</indexterm>
           <b>Purpose:</b> Returns the specified () group from a string based on a regular expression pattern. Group
           0 refers to the entire extracted string, while group 1, 2, and so on refers to the first, second, and so
           on <codeph>(...)</codeph> portion.
@@ -501,7 +501,7 @@ Returned 1 row(s) in 0.11s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">regexp_like() function</indexterm>
+          <indexterm audience="hidden">regexp_like() function</indexterm>
           <b>Purpose:</b> Returns <codeph>true</codeph> or <codeph>false</codeph> to indicate
           whether the source string contains anywhere inside it the regular expression given by the pattern.
           The optional third argument consists of letter flags that change how the match is performed,
@@ -607,7 +607,7 @@ select regexp_like('foooooobar','fx*y*o*b');
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">regexp_replace() function</indexterm>
+          <indexterm audience="hidden">regexp_replace() function</indexterm>
           <b>Purpose:</b> Returns the initial argument with the regular expression pattern replaced by the final
           argument string.
           <p>
@@ -664,7 +664,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">repeat() function</indexterm>
+          <indexterm audience="hidden">repeat() function</indexterm>
           <b>Purpose:</b> Returns the argument string repeated a specified number of times.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -680,7 +680,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">reverse() function</indexterm>
+          <indexterm audience="hidden">reverse() function</indexterm>
           <b>Purpose:</b> Returns the argument string with characters in reversed order.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -696,7 +696,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">rpad() function</indexterm>
+          <indexterm audience="hidden">rpad() function</indexterm>
           <b>Purpose:</b> Returns a string of a specified length, based on the first argument string. If the
           specified string is too short, it is padded on the right with a repeating sequence of the characters from
           the pad string. If the specified string is too long, it is truncated on the right.
@@ -714,7 +714,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">rtrim() function</indexterm>
+          <indexterm audience="hidden">rtrim() function</indexterm>
           <b>Purpose:</b> Returns the argument string with any trailing spaces removed from the right side.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -730,7 +730,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">space() function</indexterm>
+          <indexterm audience="hidden">space() function</indexterm>
           <b>Purpose:</b> Returns a concatenated string of the specified number of spaces. Shorthand for
           <codeph>repeat(' ',<varname>n</varname>)</codeph>.
           <p>
@@ -747,7 +747,7 @@ Returned 1 row(s) in 0.12s</codeblock>
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">split_part() function</indexterm>
+          <indexterm audience="hidden">split_part() function</indexterm>
           <b>Purpose:</b> Returns the nth field within a delimited string.
           The fields are numbered starting from 1.
           The delimiter can consist of multiple characters, not just a
@@ -839,7 +839,7 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">strleft() function</indexterm>
+          <indexterm audience="hidden">strleft() function</indexterm>
           <b>Purpose:</b> Returns the leftmost characters of the string. Shorthand for a call to
           <codeph>substr()</codeph> with 2 arguments.
           <p>
@@ -856,7 +856,7 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">strright() function</indexterm>
+          <indexterm audience="hidden">strright() function</indexterm>
           <b>Purpose:</b> Returns the rightmost characters of the string. Shorthand for a call to
           <codeph>substr()</codeph> with 2 arguments.
           <p>
@@ -874,7 +874,7 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">substr() function</indexterm>
+          <indexterm audience="hidden">substr() function</indexterm>
           <b>Purpose:</b> Returns the portion of the string starting at a specified point, optionally with a
           specified maximum length. The characters in the string are indexed starting at 1.
           <p>
@@ -891,7 +891,7 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">translate() function</indexterm>
+          <indexterm audience="hidden">translate() function</indexterm>
           <b>Purpose:</b> Returns the input string with a set of characters replaced by another set of characters.
           <p>
             <b>Return type:</b> <codeph>string</codeph>
@@ -907,7 +907,7 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">trim() function</indexterm>
+          <indexterm audience="hidden">trim() function</indexterm>
           <b>Purpose:</b> Returns the input string with both leading and trailing spaces removed. The same as
           passing the string through both <codeph>ltrim()</codeph> and <codeph>rtrim()</codeph>.
           <p>
@@ -928,8 +928,8 @@ select split_part('one\|/two\|/three','\|/',3);
         </dt>
 
         <dd>
-          <indexterm audience="Cloudera">upper() function</indexterm>
-          <indexterm audience="Cloudera">ucase() function</indexterm>
+          <indexterm audience="hidden">upper() function</indexterm>
+          <indexterm audience="hidden">ucase() function</indexterm>
           <b>Purpose:</b> Returns the argument string converted to all-uppercase.
           <p>
             <b>Return type:</b> <codeph>string</codeph>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_subqueries.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_subqueries.xml b/docs/topics/impala_subqueries.xml
index 4c6ec4c..b8190bf 100644
--- a/docs/topics/impala_subqueries.xml
+++ b/docs/topics/impala_subqueries.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">subqueries</indexterm>
+      <indexterm audience="hidden">subqueries</indexterm>
       A <term>subquery</term> is a query that is nested within another query. Subqueries let queries on one table
       dynamically adapt based on the contents of another table. This technique provides great flexibility and
       expressive power for SQL queries.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_sum.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_sum.xml b/docs/topics/impala_sum.xml
index 5570396..1d81d4a 100644
--- a/docs/topics/impala_sum.xml
+++ b/docs/topics/impala_sum.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">sum() function</indexterm>
+      <indexterm audience="hidden">sum() function</indexterm>
       An aggregate function that returns the sum of a set of numbers. Its single argument can be numeric column, or
       the numeric result of a function or expression applied to the column value. Rows with a <codeph>NULL</codeph>
       value for the specified column are ignored. If the table is empty, or all the values supplied to

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_support_start_over.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_support_start_over.xml b/docs/topics/impala_support_start_over.xml
index 9036e90..629b627 100644
--- a/docs/topics/impala_support_start_over.xml
+++ b/docs/topics/impala_support_start_over.xml
@@ -32,7 +32,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SUPPORT_START_OVER query option</indexterm>
+      <indexterm audience="hidden">SUPPORT_START_OVER query option</indexterm>
       Leave this setting at its default value.
       It is a read-only setting, tested by some client applications such as Hue.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_sync_ddl.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_sync_ddl.xml b/docs/topics/impala_sync_ddl.xml
index 45caa8f..7dd007b 100644
--- a/docs/topics/impala_sync_ddl.xml
+++ b/docs/topics/impala_sync_ddl.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SYNC_DDL query option</indexterm>
+      <indexterm audience="hidden">SYNC_DDL query option</indexterm>
       When enabled, causes any DDL operation such as <codeph>CREATE TABLE</codeph> or <codeph>ALTER TABLE</codeph>
       to return only when the changes have been propagated to all other Impala nodes in the cluster by the Impala
       catalog service. That way, if you issue a subsequent <codeph>CONNECT</codeph> statement in

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_tables.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_tables.xml b/docs/topics/impala_tables.xml
index 500ccca..5041111 100644
--- a/docs/topics/impala_tables.xml
+++ b/docs/topics/impala_tables.xml
@@ -96,7 +96,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">internal tables</indexterm>
+        <indexterm audience="hidden">internal tables</indexterm>
         The default kind of table produced by the <codeph>CREATE TABLE</codeph> statement is known as an internal
         table. (Its counterpart is the external table, produced by the <codeph>CREATE EXTERNAL TABLE</codeph>
         syntax.)
@@ -166,7 +166,7 @@ under the License.
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">external tables</indexterm>
+        <indexterm audience="hidden">external tables</indexterm>
         The syntax <codeph>CREATE EXTERNAL TABLE</codeph> sets up an Impala table that points at existing data
         files, potentially in HDFS locations outside the normal Impala data directories.. This operation saves the
         expense of importing the data into a new table when you already have the data files in a known location in

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_trouble_bad_results.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_trouble_bad_results.xml b/docs/topics/impala_trouble_bad_results.xml
index 99512ce..30bc7fb 100644
--- a/docs/topics/impala_trouble_bad_results.xml
+++ b/docs/topics/impala_trouble_bad_results.xml
@@ -33,9 +33,9 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">incorrect results</indexterm>
-      <indexterm audience="Cloudera">bad results</indexterm>
-      <indexterm audience="Cloudera">wrong results</indexterm>
+      <indexterm audience="hidden">incorrect results</indexterm>
+      <indexterm audience="hidden">bad results</indexterm>
+      <indexterm audience="hidden">wrong results</indexterm>
     </p>
 
     <p outputclass="toc inpage"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_trouble_memory.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_trouble_memory.xml b/docs/topics/impala_trouble_memory.xml
index 28e7868..cd0cdda 100644
--- a/docs/topics/impala_trouble_memory.xml
+++ b/docs/topics/impala_trouble_memory.xml
@@ -34,8 +34,8 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">memory errors</indexterm>
-      <indexterm audience="Cloudera">out-of-memory errors</indexterm>
+      <indexterm audience="hidden">memory errors</indexterm>
+      <indexterm audience="hidden">out-of-memory errors</indexterm>
     </p>
 
     <p outputclass="toc inpage"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_trouble_query_fail.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_trouble_query_fail.xml b/docs/topics/impala_trouble_query_fail.xml
index 2ba172e..128d2d8 100644
--- a/docs/topics/impala_trouble_query_fail.xml
+++ b/docs/topics/impala_trouble_query_fail.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">query errors</indexterm>
+      <indexterm audience="hidden">query errors</indexterm>
     </p>
 
     <p outputclass="toc inpage"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_trouble_sql.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_trouble_sql.xml b/docs/topics/impala_trouble_sql.xml
index 16042ab..f7f6e50 100644
--- a/docs/topics/impala_trouble_sql.xml
+++ b/docs/topics/impala_trouble_sql.xml
@@ -34,8 +34,8 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">SQL errors</indexterm>
-      <indexterm audience="Cloudera">syntax errors</indexterm>
+      <indexterm audience="hidden">SQL errors</indexterm>
+      <indexterm audience="hidden">syntax errors</indexterm>
     </p>
 
     <p outputclass="toc inpage"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_trouble_startup.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_trouble_startup.xml b/docs/topics/impala_trouble_startup.xml
index af05254..416ba62 100644
--- a/docs/topics/impala_trouble_startup.xml
+++ b/docs/topics/impala_trouble_startup.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">startup errors</indexterm>
+      <indexterm audience="hidden">startup errors</indexterm>
     </p>
 
     <p outputclass="toc inpage"/>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_troubleshooting.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_troubleshooting.xml b/docs/topics/impala_troubleshooting.xml
index 348f32d..3a9e2bb 100644
--- a/docs/topics/impala_troubleshooting.xml
+++ b/docs/topics/impala_troubleshooting.xml
@@ -34,7 +34,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">troubleshooting</indexterm>
+      <indexterm audience="hidden">troubleshooting</indexterm>
       Troubleshooting for Impala requires being able to diagnose and debug problems
       with performance, network connectivity, out-of-memory conditions, disk space usage,
       and crash or hang conditions in any of the Impala-related daemons.
@@ -342,7 +342,7 @@ $ sudo dd if=/dev/sdd bs=1M of=/dev/null count=1k
         </tgroup>
       </table>
 
-      <p audience="Cloudera">
+      <p audience="hidden">
         Some or all of these settings might also be useful.
 <codeblock>NUM_SCANNER_THREADS: 0
 ABORT_ON_DEFAULT_LIMIT_EXCEEDED: 0
@@ -363,7 +363,7 @@ MEM_LIMIT: 0
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="core_dumps">
+  <concept audience="hidden" id="core_dumps">
 
     <title>Enabling Core Dumps for Impala</title>
 
@@ -394,7 +394,7 @@ MEM_LIMIT: 0
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="io_throughput">
+  <concept audience="hidden" id="io_throughput">
     <title>Verifying I/O Throughput</title>
     <conbody>
       <p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_truncate_table.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_truncate_table.xml b/docs/topics/impala_truncate_table.xml
index e7752a2..3cf5ec3 100644
--- a/docs/topics/impala_truncate_table.xml
+++ b/docs/topics/impala_truncate_table.xml
@@ -37,7 +37,7 @@ under the License.
   <conbody>
 
     <p rev="2.3.0">
-      <indexterm audience="Cloudera">TRUNCATE TABLE statement</indexterm>
+      <indexterm audience="hidden">TRUNCATE TABLE statement</indexterm>
       Removes the data from an Impala table while leaving the table itself.
     </p>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_tutorial.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_tutorial.xml b/docs/topics/impala_tutorial.xml
index 2b161ed..d4f8422 100644
--- a/docs/topics/impala_tutorial.xml
+++ b/docs/topics/impala_tutorial.xml
@@ -462,7 +462,7 @@ Copyright (c) 2012 Cloudera, Inc. All rights reserved.
       </conbody>
     </concept>
 
-    <concept audience="Cloudera" id="tutorial_quickstart_vm">
+    <concept audience="hidden" id="tutorial_quickstart_vm">
 
       <title>Setting Up Impala with the QuickStart VM</title>
 
@@ -475,7 +475,7 @@ Copyright (c) 2012 Cloudera, Inc. All rights reserved.
       </conbody>
     </concept>
 
-    <concept audience="Cloudera" id="tutorial_hdfs">
+    <concept audience="hidden" id="tutorial_hdfs">
 
       <title>Exploring the HDFS Directory Structure</title>
 
@@ -489,7 +489,7 @@ Copyright (c) 2012 Cloudera, Inc. All rights reserved.
       </conbody>
     </concept>
 
-    <concept audience="Cloudera" id="tutorial_external_table">
+    <concept audience="hidden" id="tutorial_external_table">
 
       <title/>
 
@@ -1517,7 +1517,7 @@ Returned 10 row(s) in 0.39s</codeblock>
       </conbody>
     </concept>
 
-    <concept audience="Cloudera" id="tut_connect">
+    <concept audience="hidden" id="tut_connect">
 
       <title>Connecting to Impala through impala-shell</title>
 
@@ -1578,7 +1578,7 @@ Returned 10 row(s) in 0.39s</codeblock>
     </concept>
   </concept>
 
-  <concept audience="Cloudera" id="tut_mem_limit">
+  <concept audience="hidden" id="tut_mem_limit">
 
     <title>Effects of Memory Limits on Impala Queries</title>
   <prolog>
@@ -2492,7 +2492,7 @@ need to be restricted to a date range of 1995 and higher.
     </conbody>
   </concept>
 
-  <concept audience="Cloudera" id="tut_synthesizing">
+  <concept audience="hidden" id="tut_synthesizing">
 
     <title>Synthesizing Data</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_txtfile.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_txtfile.xml b/docs/topics/impala_txtfile.xml
index f23d4d7..5d3f315 100644
--- a/docs/topics/impala_txtfile.xml
+++ b/docs/topics/impala_txtfile.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">Text support in Impala</indexterm>
+      <indexterm audience="hidden">Text support in Impala</indexterm>
       Impala supports using text files as the storage format for input and output. Text files are a
       convenient format to use for interchange with other applications or scripts that produce or read delimited
       text files, such as CSV or TSV with commas or tabs for delimiters.
@@ -439,9 +439,9 @@ INSERT INTO csv SELECT * FROM other_file_format_table;</codeblock>
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">LZO support in Impala</indexterm>
+        <indexterm audience="hidden">LZO support in Impala</indexterm>
 
-        <indexterm audience="Cloudera">compression</indexterm>
+        <indexterm audience="hidden">compression</indexterm>
         Impala supports using text data files that employ LZO compression. <ph rev="upstream">Cloudera</ph> recommends compressing
         text data files when practical. Impala queries are usually I/O-bound; reducing the amount of data read from
         disk typically speeds up a query, despite the extra CPU work to uncompress the data in memory.
@@ -483,11 +483,11 @@ INSERT INTO csv SELECT * FROM other_file_format_table;</codeblock>
         <ol>
           <li>
             <b>Prepare your systems to work with LZO by downloading and installing the appropriate libraries:</b>
-            <p audience="Cloudera">
+            <p audience="hidden">
               <b>On systems managed by Cloudera Manager using parcels:</b>
             </p>
 
-            <p audience="Cloudera">
+            <p audience="hidden">
               See the setup instructions for the LZO parcel in the Cloudera Manager documentation for
               <xref href="http://www.cloudera.com/documentation/enterprise/latest/topics/cm_ig_install_gpl_extras.html" scope="external" format="html">Cloudera
               Manager 5</xref>.
@@ -725,13 +725,13 @@ hive&gt; INSERT INTO TABLE lzo_t SELECT col1, col2 FROM uncompressed_text_table;
     <conbody>
 
       <p>
-        <indexterm audience="Cloudera">gzip support in Impala</indexterm>
+        <indexterm audience="hidden">gzip support in Impala</indexterm>
 
-        <indexterm audience="Cloudera">bzip2 support in Impala</indexterm>
+        <indexterm audience="hidden">bzip2 support in Impala</indexterm>
 
-        <indexterm audience="Cloudera">Snappy support in Impala</indexterm>
+        <indexterm audience="hidden">Snappy support in Impala</indexterm>
 
-        <indexterm audience="Cloudera">compression</indexterm>
+        <indexterm audience="hidden">compression</indexterm>
         In Impala 2.0 and later, Impala supports using text data files that employ gzip, bzip2, or Snappy
         compression. These compression types are primarily for convenience within an existing ETL pipeline rather
         than maximum performance. Although it requires less I/O to read compressed text than the equivalent
@@ -815,7 +815,7 @@ $ hdfs dfs -ls 'hdfs://127.0.0.1:8020/user/hive/warehouse/file_formats.db/csv_co
 
   </concept>
 
-  <concept audience="Cloudera" id="txtfile_data_types">
+  <concept audience="hidden" id="txtfile_data_types">
 
     <title>Data Type Considerations for Text Tables</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_udf.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_udf.xml b/docs/topics/impala_udf.xml
index 46316ab..7e529a7 100644
--- a/docs/topics/impala_udf.xml
+++ b/docs/topics/impala_udf.xml
@@ -982,7 +982,7 @@ within UDAs, you can return without specifying a value.
             href="https://github.com/cloudera/impala-udf-samples/blob/master/uda-sample.h"
             scope="external" format="html" /></p>
 
-<codeblock audience="Cloudera">#ifndef SAMPLES_UDA_H
+<codeblock audience="hidden">#ifndef SAMPLES_UDA_H
 #define SAMPLES_UDA_H
 
 #include &lt;impala_udf/udf.h&gt;
@@ -1084,7 +1084,7 @@ StringVal ToStringVal(FunctionContext* context, const T&amp; val);
             scope="external" format="html" />
         </p>
 
-<codeblock audience="Cloudera">#include "uda-sample.h"
+<codeblock audience="hidden">#include "uda-sample.h"
 #include &lt;assert.h&gt;
 #include &lt;sstream&gt;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_update.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_update.xml b/docs/topics/impala_update.xml
index c69c917..3f78a17 100644
--- a/docs/topics/impala_update.xml
+++ b/docs/topics/impala_update.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">UPDATE statement</indexterm>
+      <indexterm audience="hidden">UPDATE statement</indexterm>
       Updates one or more rows from a Kudu table.
       Although updating a single row or a range of rows would be inefficient for tables using HDFS
       data files, Kudu is able to perform this operation efficiently. Therefore, this statement

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_upgrading.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_upgrading.xml b/docs/topics/impala_upgrading.xml
index 3fea37b..676ea0b 100644
--- a/docs/topics/impala_upgrading.xml
+++ b/docs/topics/impala_upgrading.xml
@@ -47,7 +47,7 @@ under the License.
           upgrade from CDH 4 to the latest CDH 5, which also has the latest Impala.
         </li>
 
-        <li rev="ver" audience="Cloudera">
+        <li rev="ver" audience="hidden">
           When you upgrade Impala, also upgrade Cloudera Manager if necessary:
           <ul>
             <li>
@@ -92,7 +92,7 @@ under the License.
     <p outputclass="toc inpage"/>
   </conbody>
 
-  <concept id="upgrade_cm_parcels" audience="Cloudera">
+  <concept id="upgrade_cm_parcels" audience="hidden">
 
     <title>Upgrading Impala through Cloudera Manager - Parcels</title>
   <prolog>
@@ -170,7 +170,7 @@ sudo apt-get purge <varname>pkg_names</varname> # Ubuntu, Debian</codeblock>
     </conbody>
   </concept>
 
-  <concept id="upgrade_cm_pkgs" audience="Cloudera">
+  <concept id="upgrade_cm_pkgs" audience="hidden">
 
     <title>Upgrading Impala through Cloudera Manager - Packages</title>
   <prolog>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_upsert.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_upsert.xml b/docs/topics/impala_upsert.xml
index c8fcb3b..d4e880f 100644
--- a/docs/topics/impala_upsert.xml
+++ b/docs/topics/impala_upsert.xml
@@ -38,7 +38,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">UPSERT statement</indexterm>
+      <indexterm audience="hidden">UPSERT statement</indexterm>
       Acts as a combination of the <codeph>INSERT</codeph>
       and <codeph>UPDATE</codeph> statements.
       For each row processed by the <codeph>UPSERT</codeph>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_use.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_use.xml b/docs/topics/impala_use.xml
index 6bb56f5..11bccd8 100644
--- a/docs/topics/impala_use.xml
+++ b/docs/topics/impala_use.xml
@@ -35,7 +35,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">USE statement</indexterm>
+      <indexterm audience="hidden">USE statement</indexterm>
       Switches the current session to a specified database. The <term>current database</term> is where any
       <codeph>CREATE TABLE</codeph>, <codeph>INSERT</codeph>, <codeph>SELECT</codeph>, or other statements act when
       you specify a table or other object name, without prefixing it with a database name. The new current database

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_v_cpu_cores.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_v_cpu_cores.xml b/docs/topics/impala_v_cpu_cores.xml
index 612cd64..b2b68b1 100644
--- a/docs/topics/impala_v_cpu_cores.xml
+++ b/docs/topics/impala_v_cpu_cores.xml
@@ -39,7 +39,7 @@ under the License.
     <note conref="../shared/impala_common.xml#common/llama_query_options_obsolete"/>
 
     <p>
-      <indexterm audience="Cloudera">V_CPU_CORES query option</indexterm>
+      <indexterm audience="hidden">V_CPU_CORES query option</indexterm>
       The number of per-host virtual CPU cores to request from YARN. If set, the query option overrides the
       automatic estimate from Impala.
       Used in conjunction with the Impala resource management feature in Impala 1.2 and higher and CDH 5.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_varchar.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_varchar.xml b/docs/topics/impala_varchar.xml
index f36687d..4226c3a 100644
--- a/docs/topics/impala_varchar.xml
+++ b/docs/topics/impala_varchar.xml
@@ -36,7 +36,7 @@ under the License.
   <conbody>
 
     <p rev="2.0.0">
-      <indexterm audience="Cloudera">VARCHAR data type</indexterm>
+      <indexterm audience="hidden">VARCHAR data type</indexterm>
       A variable-length character type, truncated during processing if necessary to fit within the specified
       length.
     </p>

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_variance.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_variance.xml b/docs/topics/impala_variance.xml
index 84f8d1d..299165c 100644
--- a/docs/topics/impala_variance.xml
+++ b/docs/topics/impala_variance.xml
@@ -37,11 +37,11 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">variance() function</indexterm>
-      <indexterm audience="Cloudera">variance_samp() function</indexterm>
-      <indexterm audience="Cloudera">variance_pop() function</indexterm>
-      <indexterm audience="Cloudera">var_samp() function</indexterm>
-      <indexterm audience="Cloudera">var_pop() function</indexterm>
+      <indexterm audience="hidden">variance() function</indexterm>
+      <indexterm audience="hidden">variance_samp() function</indexterm>
+      <indexterm audience="hidden">variance_pop() function</indexterm>
+      <indexterm audience="hidden">var_samp() function</indexterm>
+      <indexterm audience="hidden">var_pop() function</indexterm>
       An aggregate function that returns the
       <xref href="http://en.wikipedia.org/wiki/Variance" scope="external" format="html">variance</xref> of a set of
       numbers. This is a mathematical property that signifies how far the values spread apart from the mean. The

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_webui.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_webui.xml b/docs/topics/impala_webui.xml
index 612b882..57d247c 100644
--- a/docs/topics/impala_webui.xml
+++ b/docs/topics/impala_webui.xml
@@ -35,8 +35,8 @@ under the License.
   <conbody>
 
     <p id="webui_intro">
-      <indexterm audience="Cloudera">web UI</indexterm>
-      <indexterm audience="Cloudera">debug UI</indexterm>
+      <indexterm audience="hidden">web UI</indexterm>
+      <indexterm audience="hidden">debug UI</indexterm>
       Each of the Impala daemons (<cmdname>impalad</cmdname>, <cmdname>statestored</cmdname>,
       and <cmdname>catalogd</cmdname>) includes a built-in web server that displays
       diagnostic and status information:
@@ -104,14 +104,14 @@ under the License.
         suspect is having problems.
       </p>
 
-      <note audience="Cloudera">
+      <note audience="hidden">
         To get a convenient picture of the health of all Impala nodes in a cluster, use the Cloudera Manager
         interface, which collects the low-level operational information from all Impala nodes, and presents a
         unified view of the entire cluster.
       </note>
     </conbody>
 
-    <concept audience="Cloudera" id="webui_impalad_disabling">
+    <concept audience="hidden" id="webui_impalad_disabling">
 
       <title>Turning off the Web UI for impalad</title>
 
@@ -351,7 +351,7 @@ under the License.
     </concept>
   </concept>
 
-  <concept audience="Cloudera" id="webui_statestored">
+  <concept audience="hidden" id="webui_statestored">
 
     <title>Debug Web UI for statestored</title>
 
@@ -360,7 +360,7 @@ under the License.
       <p></p>
     </conbody>
 
-    <concept audience="Cloudera" id="webui_statestored_disabling">
+    <concept audience="hidden" id="webui_statestored_disabling">
 
       <title>Turning off the Web UI for statestored</title>
 
@@ -536,7 +536,7 @@ under the License.
     </concept>
   </concept>
 
-  <concept audience="Cloudera" id="webui_catalogd">
+  <concept audience="hidden" id="webui_catalogd">
 
     <title>Debug Web UI for catalogd</title>
 
@@ -545,7 +545,7 @@ under the License.
       <p></p>
     </conbody>
 
-    <concept audience="Cloudera" id="webui_catalogd_disabling">
+    <concept audience="hidden" id="webui_catalogd_disabling">
 
       <title>Turning off the Web UI for impalad</title>
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/8377b994/docs/topics/impala_window_functions.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_window_functions.xml b/docs/topics/impala_window_functions.xml
index c696267..9716403 100644
--- a/docs/topics/impala_window_functions.xml
+++ b/docs/topics/impala_window_functions.xml
@@ -30,7 +30,7 @@ under the License.
   <conbody>
 
     <p>
-      <indexterm audience="Cloudera">window functions</indexterm>
+      <indexterm audience="hidden">window functions</indexterm>
       Window functions are a special category of built-in functions. They produce one output value for each input
       row, like scalar functions such as <codeph>length()</codeph> or <codeph>substr()</codeph>. Yet like aggregate
       functions, they also examine the contents of multiple input rows to compute each output value.