You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2020/03/09 07:38:54 UTC

[impala] branch master updated: IMPALA-7784: Use unescaped string in partition pruning + fix duplicatedly unescaping strings

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9672d94  IMPALA-7784: Use unescaped string in partition pruning + fix duplicatedly unescaping strings
9672d94 is described below

commit 9672d945963e1ca3c8699340f92d7d6ce1d91c9f
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Sun Feb 23 17:05:59 2020 +0800

    IMPALA-7784: Use unescaped string in partition pruning + fix duplicatedly unescaping strings
    
    String values from external systems (HDFS, Hive, Kudu, etc.) are already
    unescaped, the same as string values in Thrift objects deserialized in
    coordinators. We should mark needsUnescaping_ as false in creating
    StringLiterals for these values (in LiteralExpr#create()).
    
    When comparing StringLiterals in partition pruning, we should also use
    the unescaped values if needsUnescaping_ is true.
    
    Tests:
     - Add tests for partition pruning on unescaped strings.
     - Add test coverage for all existing code paths using
       LiteralExpr#create().
     - Run core tests
    
    Change-Id: Iea8070f16a74f9aeade294504f2834abb8b3b38f
    Reviewed-on: http://gerrit.cloudera.org:8080/15278
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../org/apache/impala/analysis/LiteralExpr.java    |  18 ++--
 .../org/apache/impala/analysis/StringLiteral.java  |  10 +-
 .../org/apache/impala/catalog/FeCatalogUtils.java  |   2 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  |   2 +-
 .../java/org/apache/impala/catalog/KuduColumn.java |   3 +-
 .../apache/impala/analysis/LiteralExprTest.java    |   8 +-
 .../catalog/CatalogObjectToFromThriftTest.java     |  11 ++-
 testdata/bin/compute-table-stats.sh                |   2 +-
 .../functional/functional_schema_template.sql      |  14 +++
 .../datasets/functional/schema_constraints.csv     |   2 +
 .../queries/PlannerTest/partition-pruning.test     | 105 +++++++++++++++++++++
 .../queries/QueryTest/kudu_describe.test           |  22 +++++
 .../queries/QueryTest/show-stats.test              |  14 +++
 tests/metadata/test_recover_partitions.py          |  28 ++++++
 14 files changed, 217 insertions(+), 24 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java b/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
index a258c3e..7f9012e 100644
--- a/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/LiteralExpr.java
@@ -60,10 +60,11 @@ public abstract class LiteralExpr extends Expr implements Comparable<LiteralExpr
   }
 
   /**
-   * Returns an analyzed literal of 'type'. Returns null for types that do not have a
-   * LiteralExpr subclass, e.g. TIMESTAMP.
+   * Creates an analyzed literal of 'type' from an unescaped string value. Returns null
+   * for types that do not have a LiteralExpr subclass, e.g. TIMESTAMP.
    */
-  public static LiteralExpr create(String value, Type type) throws AnalysisException {
+  public static LiteralExpr createFromUnescapedStr(String value, Type type)
+      throws AnalysisException {
     if (!type.isValid()) {
       throw new UnsupportedFeatureException("Invalid literal type: " + type.toSql());
     }
@@ -87,7 +88,7 @@ public abstract class LiteralExpr extends Expr implements Comparable<LiteralExpr
       case STRING:
       case VARCHAR:
       case CHAR:
-        e = new StringLiteral(value);
+        e = new StringLiteral(value, type, false);
         break;
       case DATE:
         e = new DateLiteral(value);
@@ -124,7 +125,7 @@ public abstract class LiteralExpr extends Expr implements Comparable<LiteralExpr
       LiteralExpr result = null;
       switch (exprNode.node_type) {
         case FLOAT_LITERAL:
-          result = LiteralExpr.create(
+          result = LiteralExpr.createFromUnescapedStr(
               Double.toString(exprNode.float_literal.value), colType);
           break;
         case DECIMAL_LITERAL:
@@ -140,14 +141,15 @@ public abstract class LiteralExpr extends Expr implements Comparable<LiteralExpr
               exprNode.date_literal.date_string);
           break;
         case INT_LITERAL:
-          result = LiteralExpr.create(
+          result = LiteralExpr.createFromUnescapedStr(
               Long.toString(exprNode.int_literal.value), colType);
           break;
         case STRING_LITERAL:
-          result = LiteralExpr.create(exprNode.string_literal.value, colType);
+          result = LiteralExpr.createFromUnescapedStr(
+              exprNode.string_literal.value, colType);
           break;
         case BOOL_LITERAL:
-          result =  LiteralExpr.create(
+          result =  LiteralExpr.createFromUnescapedStr(
               Boolean.toString(exprNode.bool_literal.value), colType);
           break;
         case NULL_LITERAL:
diff --git a/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java b/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
index 3cf2ab2..37fdd45 100644
--- a/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
+++ b/fe/src/main/java/org/apache/impala/analysis/StringLiteral.java
@@ -38,7 +38,10 @@ public class StringLiteral extends LiteralExpr {
   private final String value_;
   public static int MAX_STRING_LEN = Integer.MAX_VALUE;
 
-  // Indicates whether this value needs to be unescaped in toThrift().
+  // Indicates whether this value needs to be unescaped in toThrift() or comparison.
+  // TODO: Add enum to distinguish the sources, e.g. double/single quoted SQL, HMS view,
+  //  or the result of an evaluated const expression. So we know whether we need
+  //  unescaping more clearly.
   private final boolean needsUnescaping_;
 
   public StringLiteral(String value) {
@@ -215,7 +218,10 @@ public class StringLiteral extends LiteralExpr {
     int ret = super.compareTo(o);
     if (ret != 0) return ret;
     StringLiteral other = (StringLiteral) o;
-    return value_.compareTo(other.getStringValue());
+    String thisValue = needsUnescaping_? getUnescapedValue() : value_;
+    String otherValue = other.needsUnescaping_?
+        other.getUnescapedValue() : other.getStringValue();
+    return thisValue.compareTo(otherValue);
   }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
index 8898fa1..bfe245e 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeCatalogUtils.java
@@ -235,7 +235,7 @@ public abstract class FeCatalogUtils {
         keyValues.add(NullLiteral.create(type));
       } else {
         try {
-          keyValues.add(LiteralExpr.create(partitionKey, type));
+          keyValues.add(LiteralExpr.createFromUnescapedStr(partitionKey, type));
         } catch (Exception ex) {
           LOG.warn(String.format(
               "Failed to create literal expression: type: %s, value: '%s'",
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 c1decbb..1e3bf65 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -1770,7 +1770,7 @@ public class HdfsTable extends Table implements FeFsTable {
     String value = URLDecoder.decode(partName[1], StandardCharsets.UTF_8.name());
     if (!value.equals(getNullPartitionKeyValue())) {
       try {
-        expr = LiteralExpr.create(value, type);
+        expr = LiteralExpr.createFromUnescapedStr(value, type);
         // Skip large value which exceeds the MAX VALUE of specified Type.
         if (expr instanceof NumericLiteral) {
           if (NumericLiteral.isOverflow(((NumericLiteral) expr).getValue(), type)) {
diff --git a/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java b/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
index 246b0e3..03dcd96 100644
--- a/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
+++ b/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
@@ -78,7 +78,8 @@ public class KuduColumn extends Column {
     if (defaultValue != null) {
       Type defaultValueType = type.isTimestamp() ? Type.BIGINT : type;
       try {
-        defaultValueExpr = LiteralExpr.create(defaultValue.toString(), defaultValueType);
+        defaultValueExpr = LiteralExpr.createFromUnescapedStr(defaultValue.toString(),
+            defaultValueType);
       } catch (AnalysisException e) {
         throw new ImpalaRuntimeException(String.format("Error parsing default value: " +
             "'%s'", defaultValue), e);
diff --git a/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java b/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
index 3862443..88d1aa3 100644
--- a/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/LiteralExprTest.java
@@ -24,10 +24,6 @@ import static org.junit.Assert.fail;
 import org.apache.impala.catalog.ScalarType;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.FrontendTestBase;
-import org.apache.impala.common.ImpalaException;
-import org.apache.impala.common.SqlCastException;
-import org.apache.impala.rewrite.ExprRewriteRule;
-import org.apache.impala.rewrite.FoldConstantsRule;
 import org.junit.Test;
 
 /**
@@ -76,7 +72,7 @@ public class LiteralExprTest extends FrontendTestBase {
   private void testLiteralExprPositive(String value, Type type) {
     LiteralExpr expr = null;
     try {
-      expr = LiteralExpr.create(value, type);
+      expr = LiteralExpr.createFromUnescapedStr(value, type);
     } catch (Exception e) {
       fail("\nFailed to create LiteralExpr of type: " + type.toString() +
           " from: " + value + " due to " + e.getMessage() + "\n");
@@ -90,7 +86,7 @@ public class LiteralExprTest extends FrontendTestBase {
     boolean failure = false;
     LiteralExpr expr = null;
     try {
-      expr = LiteralExpr.create(value, type);
+      expr = LiteralExpr.createFromUnescapedStr(value, type);
     } catch (Exception e) {
       failure = true;
     }
diff --git a/fe/src/test/java/org/apache/impala/catalog/CatalogObjectToFromThriftTest.java b/fe/src/test/java/org/apache/impala/catalog/CatalogObjectToFromThriftTest.java
index e6c47b6..2da922e 100644
--- a/fe/src/test/java/org/apache/impala/catalog/CatalogObjectToFromThriftTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/CatalogObjectToFromThriftTest.java
@@ -237,10 +237,13 @@ public class CatalogObjectToFromThriftTest {
     // Create a dummy partition with an invalid decimal type.
     try {
       new HdfsPartition(hdfsTable, part.toHmsPartition(),
-        Lists.newArrayList(LiteralExpr.create("11.1", ScalarType.createDecimalType(1, 0)),
-            LiteralExpr.create("11.1", ScalarType.createDecimalType(1, 0))),
-        null, new ArrayList<>(),
-        TAccessLevel.READ_WRITE);
+          Lists.newArrayList(
+              LiteralExpr.createFromUnescapedStr(
+                  "11.1", ScalarType.createDecimalType(1, 0)),
+              LiteralExpr.createFromUnescapedStr(
+                  "11.1", ScalarType.createDecimalType(1, 0))),
+          null, new ArrayList<>(),
+          TAccessLevel.READ_WRITE);
       fail("Expected metadata to be malformed.");
     } catch (SqlCastException e) {
       Assert.assertTrue(e.getMessage().contains(
diff --git a/testdata/bin/compute-table-stats.sh b/testdata/bin/compute-table-stats.sh
index d7e8cf8..31c585c 100755
--- a/testdata/bin/compute-table-stats.sh
+++ b/testdata/bin/compute-table-stats.sh
@@ -34,7 +34,7 @@ COMPUTE_STATS_SCRIPT="${IMPALA_HOME}/tests/util/compute_table_stats.py --impalad
 ${COMPUTE_STATS_SCRIPT} --db_names=functional\
     --table_names="alltypes,alltypesagg,alltypesaggmultifilesnopart,alltypesaggnonulls,
     alltypessmall,alltypestiny,jointbl,dimtbl,stringpartitionkey,nulltable,nullrows,
-    date_tbl,chars_medium"
+    date_tbl,chars_medium,part_strings_with_quotes"
 
 # We cannot load HBase on s3 and isilon yet.
 if [ "${TARGET_FILESYSTEM}" = "hdfs" ]; then
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index a003434..f502df7 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -2617,6 +2617,20 @@ FROM {db_name}.{table_name};
 ---- DATASET
 functional
 ---- BASE_TABLE_NAME
+part_strings_with_quotes
+---- COLUMNS
+i int
+---- PARTITION_COLUMNS
+p string
+---- LOAD
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (p="\"") VALUES (1);
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (p='\'') VALUES (2);
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (p="\\\"") VALUES (3);
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} PARTITION (p='\\\'') VALUES (4);
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
 manynulls
 ---- COLUMNS
 id int
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index 7c63d76..b644a91 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -284,3 +284,5 @@ table_name:bucketed_table, constraint:exclude, table_format:text/lzo/block
 # them each time when running the test. Developers may run this test many times locally.
 table_name:uncomp_src_alltypes, constraint:restrict_to, table_format:orc/def/block
 table_name:uncomp_src_decimal_tbl, constraint:restrict_to, table_format:orc/def/block
+
+table_name:part_strings_with_quotes, constraint:restrict_to, table_format:text/none/none
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
index 4d4403e..67b0e75 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
@@ -21,3 +21,108 @@ PLAN-ROOT SINK
    tuple-ids=0 row-size=16B cardinality=1
    in pipelines: 00(GETNEXT)
 ====
+# IMPALA-7784: Test partition pruning on unescaped string values
+select * from functional.part_strings_with_quotes where p = "\"" and p=concat("", '"')
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+00:SCAN HDFS [functional.part_strings_with_quotes]
+   partition predicates: p = '"'
+   HDFS partitions=1/4 files=1 size=2B
+   stored statistics:
+     table: rows=4 size=8B
+     partitions: 1/1 rows=1
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=4
+   mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
+   tuple-ids=0 row-size=16B cardinality=1
+   in pipelines: 00(GETNEXT)
+====
+# IMPALA-7784: Test partition pruning on unescaped string values
+select * from functional.part_strings_with_quotes where p = '\''
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+00:SCAN HDFS [functional.part_strings_with_quotes]
+   partition predicates: p = '\''
+   HDFS partitions=1/4 files=1 size=2B
+   stored statistics:
+     table: rows=4 size=8B
+     partitions: 1/1 rows=1
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=4
+   mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
+   tuple-ids=0 row-size=16B cardinality=1
+   in pipelines: 00(GETNEXT)
+====
+# IMPALA-7784: Test partition pruning on unescaped string values
+select * from functional.part_strings_with_quotes where p = "\\\""
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+00:SCAN HDFS [functional.part_strings_with_quotes]
+   partition predicates: p = '\\"'
+   HDFS partitions=1/4 files=1 size=2B
+   stored statistics:
+     table: rows=4 size=8B
+     partitions: 1/1 rows=1
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=4
+   mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
+   tuple-ids=0 row-size=16B cardinality=1
+   in pipelines: 00(GETNEXT)
+====
+# IMPALA-7784: Test partition pruning on unescaped string values
+select * from functional.part_strings_with_quotes where p = '\\\''
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+00:SCAN HDFS [functional.part_strings_with_quotes]
+   partition predicates: p = '\\\''
+   HDFS partitions=1/4 files=1 size=2B
+   stored statistics:
+     table: rows=4 size=8B
+     partitions: 1/1 rows=1
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=4
+   mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
+   tuple-ids=0 row-size=16B cardinality=1
+   in pipelines: 00(GETNEXT)
+====
+# IMPALA-7784: Test partition pruning on unescaped string values
+select * from functional.part_strings_with_quotes where p in ("\"", "\\\"")
+---- PLAN
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: functional.part_strings_with_quotes.i, functional.part_strings_with_quotes.p
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+00:SCAN HDFS [functional.part_strings_with_quotes]
+   partition predicates: p IN ('"', '\\"')
+   HDFS partitions=2/4 files=2 size=4B
+   stored statistics:
+     table: rows=4 size=8B
+     partitions: 2/2 rows=2
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=2
+   mem-estimate=16.00MB mem-reservation=8.00KB thread-reservation=1
+   tuple-ids=0 row-size=16B cardinality=2
+   in pipelines: 00(GETNEXT)
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
index 8b8e80f..4884e64 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
@@ -65,3 +65,25 @@ NAME,TYPE,COMMENT,PRIMARY_KEY,NULLABLE,DEFAULT_VALUE,ENCODING,COMPRESSION,BLOCK_
 ---- TYPES
 STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING
 ====
+---- QUERY
+# IMPALA-7781: Test unescaped default column values
+CREATE TABLE IF NOT EXISTS unescaped_str_defaults (
+  id int,
+  s1 string default "\"",
+  s2 string default '\'',
+  s3 string default "\\\"",
+  s4 string default '\\\'',
+  primary key(id)
+) STORED AS KUDU;
+DESCRIBE unescaped_str_defaults;
+---- LABELS
+NAME,TYPE,COMMENT,PRIMARY_KEY,NULLABLE,DEFAULT_VALUE,ENCODING,COMPRESSION,BLOCK_SIZE
+---- RESULTS
+'id','int','','true','false','','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'s1','string','','false','true','"','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'s2','string','','false','true','''','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'s3','string','','false','true','\\"','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'s4','string','','false','true','\\''','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+---- TYPES
+STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/show-stats.test b/testdata/workloads/functional-query/queries/QueryTest/show-stats.test
index 73cdfb1..22b38bc 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/show-stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/show-stats.test
@@ -172,3 +172,17 @@ COLUMN, TYPE, #DISTINCT VALUES, #NULLS, MAX SIZE, AVG SIZE
 ---- TYPES
 STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ====
+---- QUERY
+# IMPALA-7784: Show unescaped partition string values. Note that in our test framework,
+# two single quotes inside a single-quoted string is escaped to a single quote.
+show partitions functional.part_strings_with_quotes
+---- LABELS
+p, #ROWS, #FILES, SIZE, BYTES CACHED, CACHE REPLICATION, FORMAT, INCREMENTAL STATS, LOCATION
+---- RESULTS
+'"',1,1,regex:.+B,'NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/part_strings_with_quotes/p=%22'
+'''',1,1,regex:.+B,'NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/part_strings_with_quotes/p=%27'
+'\\"',1,1,regex:.+B,'NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/part_strings_with_quotes/p=%5C%22'
+'\\''',1,1,regex:.+B,'NOT CACHED','NOT CACHED','TEXT','false','$NAMENODE/test-warehouse/part_strings_with_quotes/p=%5C%27'
+'Total',4,4,'8B','0B','','','',''
+---- TYPES
+STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
diff --git a/tests/metadata/test_recover_partitions.py b/tests/metadata/test_recover_partitions.py
index 36a23ea..dcd393e 100644
--- a/tests/metadata/test_recover_partitions.py
+++ b/tests/metadata/test_recover_partitions.py
@@ -354,6 +354,27 @@ class TestRecoverPartitions(ImpalaTestSuite):
         FQ_TBL_NAME
 
   @SkipIfLocal.hdfs_client
+  def test_unescaped_string_partition(self, vector, unique_database):
+    """IMPALA-7784: Test that RECOVER PARTITIONS correctly parses unescaped string
+       values"""
+    tbl_name = "test_unescaped_string_partition"
+    fq_tbl_name = unique_database + "." + tbl_name
+    tbl_location = self.__get_fs_location(unique_database, tbl_name)
+
+    self.execute_query_expect_success(
+        self.client, "CREATE TABLE %s (i int) PARTITIONED BY (p string)" % fq_tbl_name)
+    self.create_fs_partition(tbl_location, 'p=\"', "file_000", "1")
+    self.create_fs_partition(tbl_location, 'p=\'', "file_000", "2")
+    self.create_fs_partition(tbl_location, 'p=\\\"', "file_000", "3")
+    self.create_fs_partition(tbl_location, 'p=\\\'', "file_000", "4")
+    self.execute_query_expect_success(
+        self.client, "ALTER TABLE %s RECOVER PARTITIONS" % fq_tbl_name)
+    result = self.execute_query_expect_success(
+        self.client, "SHOW PARTITIONS %s" % fq_tbl_name)
+    assert self.count_partition(result.data) == 4
+    self.verify_partitions(['\"', '\'', '\\\"', '\\\''], result.data)
+
+  @SkipIfLocal.hdfs_client
   @SkipIfS3.empty_directory
   def test_empty_directory(self, vector, unique_database):
     """Explicitly test how empty directories are handled when partitions are recovered."""
@@ -433,3 +454,10 @@ class TestRecoverPartitions(ImpalaTestSuite):
   def count_value(self, value, lines):
     """Count the number of lines that contain value."""
     return len(filter(lambda line: line.find(value) != -1, lines))
+
+  def verify_partitions(self, expected_parts, lines):
+    """Check if all partition values are expected"""
+    values = [line.split('\t')[0] for line in lines]
+    assert len(values) == len(expected_parts) + 1
+    for p in expected_parts:
+      assert p in values