You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bi...@apache.org on 2018/10/10 18:06:17 UTC

[1/3] impala git commit: IMPALA-7684: Fix Admission result printed in the query profile

Repository: impala
Updated Branches:
  refs/heads/master 6568e6e11 -> 8ca79261a


IMPALA-7684: Fix Admission result printed in the query profile

With this patch the right result is printed for the admission result
when the query is admitted immediately vs when it is admitted after
being queued.

Testing:
Added coverage for checking the result for these two cases to an
existing test.

Change-Id: I410993a555d9590cca42902fbfa1fe7aa883be06
Reviewed-on: http://gerrit.cloudera.org:8080/11634
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 9ac874d1be84a591c79fe2ca3ba48b9cbe7c7c70
Parents: 6568e6e
Author: Bikramjeet Vig <bi...@cloudera.com>
Authored: Tue Oct 9 14:56:14 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Oct 10 03:00:27 2018 +0000

----------------------------------------------------------------------
 be/src/scheduling/admission-controller.cc         | 4 ++--
 tests/custom_cluster/test_admission_controller.py | 4 ++++
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9ac874d1/be/src/scheduling/admission-controller.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/admission-controller.cc b/be/src/scheduling/admission-controller.cc
index a23fce7..e2b5415 100644
--- a/be/src/scheduling/admission-controller.cc
+++ b/be/src/scheduling/admission-controller.cc
@@ -626,7 +626,7 @@ Status AdmissionController::SubmitForAdmission(QuerySchedule* schedule,
         return Status::CANCELLED;
       }
       VLOG_QUERY << "Admitted query id=" << PrintId(schedule->query_id());
-      AdmitQuery(schedule, true);
+      AdmitQuery(schedule, false);
       VLOG_RPC << "Final: " << stats->DebugString();
       return Status::OK();
     }
@@ -997,7 +997,7 @@ void AdmissionController::DequeueLoop() {
           continue;
         }
         DCHECK_ENUM_EQ(outcome, AdmissionOutcome::ADMITTED);
-        AdmitQuery(schedule, false);
+        AdmitQuery(schedule, true);
       }
       pools_for_updates_.insert(pool_name);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/9ac874d1/tests/custom_cluster/test_admission_controller.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_admission_controller.py b/tests/custom_cluster/test_admission_controller.py
index 24e2ce3..56d2b0b 100644
--- a/tests/custom_cluster/test_admission_controller.py
+++ b/tests/custom_cluster/test_admission_controller.py
@@ -758,6 +758,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
     # Setup to queue a query.
     sleep_query_handle = self.client.execute_async("select sleep(10000)")
     self.client.wait_for_admission_control(sleep_query_handle)
+    self.__wait_for_change_to_profile(sleep_query_handle,
+                                      "Admission result: Admitted immediately")
     queued_query_handle = self.client.execute_async("select 1")
     self.__wait_for_change_to_profile(queued_query_handle, "Admission result: Queued")
 
@@ -802,6 +804,8 @@ class TestAdmissionController(TestAdmissionControllerBase, HS2TestSuite):
     # Now change the config back to a reasonable value.
     config.set_config_value(pool_name, config_str, 0)
     self.client.wait_for_finished_timeout(queued_query_handle, 20)
+    self.__wait_for_change_to_profile(queued_query_handle,
+                                      "Admission result: Admitted (queued)")
     self.close_query(queued_query_handle)
 
   def __wait_for_change_to_profile(self, query_handle, search_string, timeout=20):


[2/3] impala git commit: IMPALA-7676: DESCRIBE on table should require VIEW_METADATA privilege

Posted by bi...@apache.org.
IMPALA-7676: DESCRIBE on table should require VIEW_METADATA privilege

IMPALA-6479 broke the DESCRIBE's privilege model by changing the
privilege from VIEW_METADATA to ANY in order to support column-level
privileges in DESCRIBE. This caused an issue where having non-
VIEW_METADATA privilege, such as CREATE privilege on a particular
database allows executing a DESCRIBE statement on all tables in the
database. This behavior is also inconsistent with Hive's DESCRIBE
and Impala's DESCRIBE DATABASE privilege models. Although there is not
any security risk for this particular issue since having non-
VIEW METADATA on a particular database always returns an empty result,
fixing this issue will make the behavior consistent with Hive and also
DESCRIBE DATABASE in Impala. This patch fixes the issue by changing the
privilege requirement back from ANY to VIEW_METADATA.

Testing:
- Ran all FE tests

Change-Id: I283e30ebff6d61e779a4cec8284cae0ccb90cc49
Reviewed-on: http://gerrit.cloudera.org:8080/11617
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 53decbc9fcfe117470df09eb27b361195d704a06
Parents: 9ac874d
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Mon Oct 8 12:18:13 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed Oct 10 09:03:10 2018 +0000

----------------------------------------------------------------------
 .../org/apache/impala/analysis/Analyzer.java    | 27 ++++++++--
 .../impala/analysis/DescribeTableStmt.java      |  5 +-
 .../impala/analysis/DropTableOrViewStmt.java    |  3 +-
 .../apache/impala/analysis/PartitionDef.java    |  4 +-
 .../impala/analysis/PartitionSpecBase.java      |  3 +-
 .../apache/impala/analysis/AuditingTest.java    | 12 ++---
 .../impala/analysis/AuthorizationStmtTest.java  | 55 ++++++++++----------
 7 files changed, 67 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index f848cc8..0672cfb 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -2413,21 +2413,25 @@ public class Analyzer {
    * Returns the Table with the given name from the 'loadedTables' map in the global
    * analysis state. Throws an AnalysisException if the table or the db does not exist.
    * Throws a TableLoadingException if the registered table failed to load.
-   * Always registers privilege request(s) for the table at the given privilege level(s),
+   * When addColumnLevelPrivilege is set to true, always registers privilege request(s)
+   * for the columns at the given table.
+   * When addColumnLevelPrivilege is set to false, always registers privilege request(s)
+   * for the table at the given privilege level(s),
    * regardless of the state of the table (i.e. whether it exists, is loaded, etc.).
    * If addAccessEvent is true adds access event(s) for successfully loaded tables. When
    * multiple privileges are specified, all those privileges will be required for the
    * authorization check.
    */
   public FeTable getTable(TableName tableName, boolean addAccessEvent,
-      Privilege... privilege) throws AnalysisException, TableLoadingException {
+      boolean addColumnPrivilege, Privilege... privilege)
+      throws AnalysisException, TableLoadingException {
     Preconditions.checkNotNull(tableName);
     Preconditions.checkNotNull(privilege);
     tableName = getFqTableName(tableName);
     for (Privilege priv : privilege) {
-      if (priv == Privilege.ANY) {
+      if (priv == Privilege.ANY || addColumnPrivilege) {
         registerPrivReq(new PrivilegeRequestBuilder()
-            .any().onAnyColumn(tableName.getDb(), tableName.getTbl()).toRequest());
+            .allOf(priv).onAnyColumn(tableName.getDb(), tableName.getTbl()).toRequest());
       } else {
         registerPrivReq(new PrivilegeRequestBuilder()
             .allOf(priv).onTable(tableName.getDb(), tableName.getTbl()).toRequest());
@@ -2458,7 +2462,20 @@ public class Analyzer {
   public FeTable getTable(TableName tableName, Privilege... privilege)
       throws AnalysisException {
     try {
-      return getTable(tableName, true, privilege);
+      return getTable(tableName, true, false, privilege);
+    } catch (TableLoadingException e) {
+      throw new AnalysisException(e);
+    }
+  }
+
+  /**
+   * Sets the addColumnPrivilege to true to add column-level privilege(s) for a given
+   * table instead of table-level privilege(s).
+   */
+  public FeTable getTable(TableName tableName, boolean addColumnPrivilege,
+      Privilege... privilege) throws AnalysisException {
+    try {
+      return getTable(tableName, true, addColumnPrivilege, privilege);
     } catch (TableLoadingException e) {
       throw new AnalysisException(e);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
index 1e70468..e731dfc 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.impala.analysis.Path.PathType;
 import org.apache.impala.authorization.Privilege;
+import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.PrivilegeRequestBuilder;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.catalog.StructType;
@@ -111,8 +112,10 @@ public class DescribeTableStmt extends StatementBase {
     }
 
     table_ = path_.getRootTable();
+
     // Register authorization and audit events.
-    analyzer.getTable(table_.getTableName(), Privilege.ANY);
+    analyzer.getTable(table_.getTableName(), /* add column-level privilege */ true,
+        Privilege.VIEW_METADATA);
 
     // Describing a table.
     if (path_.destTable() != null) return;

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
index 2d53794..1da6c68 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
@@ -100,7 +100,8 @@ public class DropTableOrViewStmt extends StatementBase {
   public void analyze(Analyzer analyzer) throws AnalysisException {
     dbName_ = analyzer.getTargetDbName(tableName_);
     try {
-      FeTable table = analyzer.getTable(tableName_, true, Privilege.DROP);
+      FeTable table = analyzer.getTable(tableName_, /* add access event */ true,
+          /* add column-level privilege */ false, Privilege.DROP);
       Preconditions.checkNotNull(table);
       if (table instanceof FeView && dropTable_) {
         throw new AnalysisException(String.format(

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/main/java/org/apache/impala/analysis/PartitionDef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PartitionDef.java b/fe/src/main/java/org/apache/impala/analysis/PartitionDef.java
index 7b4f837..b66d779 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PartitionDef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PartitionDef.java
@@ -83,7 +83,9 @@ public class PartitionDef implements ParseNode {
 
     FeTable table;
     try {
-      table = analyzer.getTable(partitionSpec_.getTableName(), false, Privilege.ALTER);
+      table = analyzer.getTable(partitionSpec_.getTableName(),
+          /* add access event */ false, /* add column-level privilege */ false,
+          Privilege.ALTER);
     } catch (TableLoadingException e) {
       throw new AnalysisException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/main/java/org/apache/impala/analysis/PartitionSpecBase.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PartitionSpecBase.java b/fe/src/main/java/org/apache/impala/analysis/PartitionSpecBase.java
index ccdb266..c4829a2 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PartitionSpecBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PartitionSpecBase.java
@@ -74,7 +74,8 @@ public abstract class PartitionSpecBase implements ParseNode {
     // be audited outside of the PartitionSpec.
     FeTable table;
     try {
-      table = analyzer.getTable(tableName_, false, privilegeRequirement_);
+      table = analyzer.getTable(tableName_, /* add access event */ false,
+          /* add column-level privilege */ false, privilegeRequirement_);
     } catch (TableLoadingException e) {
       throw new AnalysisException(e.getMessage(), e);
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
index 944fadb..836020b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
@@ -319,20 +319,20 @@ public class AuditingTest extends FrontendTestBase {
     Set<TAccessEvent> accessEvents =
         AnalyzeAccessEvents("describe functional.alltypesagg");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional.alltypesagg", TCatalogObjectType.TABLE, "ANY")));
+        "functional.alltypesagg", TCatalogObjectType.TABLE, "VIEW_METADATA")));
 
     accessEvents = AnalyzeAccessEvents("describe formatted functional.alltypesagg");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional.alltypesagg", TCatalogObjectType.TABLE, "ANY")));
+        "functional.alltypesagg", TCatalogObjectType.TABLE, "VIEW_METADATA")));
 
     accessEvents = AnalyzeAccessEvents("describe functional.complex_view");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional.complex_view", TCatalogObjectType.VIEW, "ANY")));
+        "functional.complex_view", TCatalogObjectType.VIEW, "VIEW_METADATA")));
 
     accessEvents = AnalyzeAccessEvents(
         "describe functional.allcomplextypes.int_array_col");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional.allcomplextypes", TCatalogObjectType.TABLE, "ANY")));
+        "functional.allcomplextypes", TCatalogObjectType.TABLE, "VIEW_METADATA")));
   }
 
   @Test
@@ -459,12 +459,12 @@ public class AuditingTest extends FrontendTestBase {
     // Describe
     accessEvents = AnalyzeAccessEvents("describe functional_kudu.testtbl");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "ANY")));
+        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "VIEW_METADATA")));
 
     // Describe formatted
     accessEvents = AnalyzeAccessEvents("describe formatted functional_kudu.testtbl");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "ANY")));
+        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "VIEW_METADATA")));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/53decbc9/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
index 8be63b9..02ae21b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
@@ -25,7 +25,6 @@ import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.User;
 import org.apache.impala.catalog.AuthorizationException;
-import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.impala.catalog.Role;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Type;
@@ -1220,18 +1219,18 @@ public class AuthorizationStmtTest extends FrontendTestBase {
               onTable("functional", "alltypes", privilege));
     }
     authzTest
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onServer(allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onDatabase("functional",allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onTable("functional", "alltypes", allExcept(viewMetadataPrivileges())))
         // In this test, since we only have column level privileges on "id", then
         // only the "id" column should show and the others should not.
         .okDescribe(tableName, describeOutput(style).includeStrings(new String[]{"id"})
             .excludeStrings(ALLTYPES_COLUMNS_WITHOUT_ID), onColumn("functional",
             "alltypes", "id", TPrivilegeLevel.SELECT))
-        .error(accessError("functional.alltypes"));
+        .error(accessError("functional.alltypes"))
+        .error(accessError("functional.alltypes"),
+            onServer(allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes"),
+            onDatabase("functional", allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes"),
+            onTable("functional", "alltypes", allExcept(viewMetadataPrivileges())));
 
     // Describe table extended.
     tableName = new TTableName("functional", "alltypes");
@@ -1248,21 +1247,19 @@ public class AuthorizationStmtTest extends FrontendTestBase {
           .okDescribe(tableName, describeOutput(style).includeStrings(checkStrings),
               onTable("functional", "alltypes", privilege));
     }
-    // Describe table without VIEW_METADATA privilege should not show all columns and
-    // location.
     authzTest
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onServer(allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onDatabase("functional", allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onTable("functional", "alltypes", allExcept(viewMetadataPrivileges())))
         // Location should not appear with only column level auth.
         .okDescribe(tableName, describeOutput(style).includeStrings(new String[]{"id"})
             .excludeStrings((String[]) ArrayUtils.addAll(ALLTYPES_COLUMNS_WITHOUT_ID,
-            new String[]{"Location:"})), onColumn("functional", "alltypes", "id",
+                new String[]{"Location:"})), onColumn("functional", "alltypes", "id",
             TPrivilegeLevel.SELECT))
-        .error(accessError("functional.alltypes"));
+        .error(accessError("functional.alltypes"))
+        .error(accessError("functional.alltypes"),
+            onServer(allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes"),
+            onDatabase("functional.alltypes", allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes"),
+            onTable("functional", "alltypes", allExcept(viewMetadataPrivileges())));
 
     // Describe view.
     tableName = new TTableName("functional", "alltypes_view");
@@ -1278,11 +1275,13 @@ public class AuthorizationStmtTest extends FrontendTestBase {
               onTable("functional", "alltypes_view", privilege));
     }
     authzTest
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
+        .error(accessError("functional.alltypes_view"))
+        .error(accessError("functional.alltypes_view"),
             onServer(allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
+        .error(accessError("functional.alltypes_view"),
             onDatabase("functional",allExcept(viewMetadataPrivileges())))
-        .error(accessError("functional.alltypes_view"));
+        .error(accessError("functional.alltypes_view"),
+            onTable("functional", "alltypes_view", allExcept(viewMetadataPrivileges())));
 
     // Describe view extended.
     tableName = new TTableName("functional", "alltypes_view");
@@ -1301,15 +1300,17 @@ public class AuthorizationStmtTest extends FrontendTestBase {
               onTable("functional", "alltypes_view", privilege));
     }
     authzTest
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onServer(allExcept(viewMetadataPrivileges())))
-        .okDescribe(tableName, describeOutput(style).excludeStrings(ALLTYPES_COLUMNS),
-            onDatabase("functional",allExcept(viewMetadataPrivileges())))
-        .error(accessError("functional.alltypes_view"));
+        .error(accessError("functional.alltypes_view"))
+        .error(accessError("functional.alltypes_view"), onServer(
+            allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes_view"), onDatabase("functional",
+            allExcept(viewMetadataPrivileges())))
+        .error(accessError("functional.alltypes_view"), onTable("functional", "alltypes",
+            allExcept(viewMetadataPrivileges())));
 
     // Describe specific column on a table.
     authzTest = authorize("describe functional.allcomplextypes.int_struct_col");
-    for (TPrivilegeLevel privilege: TPrivilegeLevel.values()) {
+    for (TPrivilegeLevel privilege: viewMetadataPrivileges()) {
       authzTest.ok(onServer(privilege))
           .ok(onDatabase("functional", privilege))
           .ok(onTable("functional", "allcomplextypes", privilege));


[3/3] impala git commit: [DOCS] Copy edited the Cancelling a Query section

Posted by bi...@apache.org.
[DOCS] Copy edited the Cancelling a Query section

Change-Id: I7a06c94b397c872636d2ecf68cc6a2e193a3b047
Reviewed-on: http://gerrit.cloudera.org:8080/11637
Tested-by: Impala Public Jenkins <im...@cloudera.com>
Reviewed-by: Bikramjeet Vig <bi...@cloudera.com>


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

Branch: refs/heads/master
Commit: 8ca79261a336741f7b537809d2f5de422744d8b3
Parents: 53decbc
Author: Alex Rodoni <ar...@cloudera.com>
Authored: Tue Oct 9 16:31:06 2018 -0700
Committer: Alex Rodoni <ar...@cloudera.com>
Committed: Wed Oct 10 18:04:21 2018 +0000

----------------------------------------------------------------------
 docs/topics/impala_timeouts.xml | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/8ca79261/docs/topics/impala_timeouts.xml
----------------------------------------------------------------------
diff --git a/docs/topics/impala_timeouts.xml b/docs/topics/impala_timeouts.xml
index 0ae394e..4ee95e6 100644
--- a/docs/topics/impala_timeouts.xml
+++ b/docs/topics/impala_timeouts.xml
@@ -204,11 +204,9 @@ Trying to re-register with state-store</codeblock>
     <title>Cancelling a Query</title>
 
     <conbody>
-
-      <p> Sometimes, an Impala query might run for an unexpectedly long time,
+      <p> Occasionally, an Impala query might run for an unexpectedly long time,
         tying up resources in the cluster. This section describes the options to
-        terminate the runaway queries.</p>
-
+        terminate such runaway queries.</p>
     </conbody>
     <concept id="cancel_query_with_query_option">
       <title>Setting a Time Limit on Query Execution</title>
@@ -216,11 +214,11 @@ Trying to re-register with state-store</codeblock>
         <p>An Impala administrator can set a default value of the
             <codeph>EXEC_TIME_LIMIT_S</codeph> query option for a resource pool.
           If a user accidentally runs a large query that executes for longer
-          than the limit, it will be automatically killed after the time limit
-          expires and frees up resources. </p>
-        <p>Users can override the default value per query or per session if they
-          do not want the default <codeph>EXEC_TIME_LIMIT_S</codeph> value to
-          apply to a specific query or a session. See <xref
+          than the limit, it will be automatically terminated after the time
+          limit expires to free up resources. </p>
+        <p>You can override the default value per query or per session if you do
+          not want to apply the default <codeph>EXEC_TIME_LIMIT_S</codeph> value
+          to a specific query or a session. See <xref
             href="impala_exec_time_limit_s.xml#exec_time_limit_s"/> for the
           details of the query option.</p>
       </conbody>