You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by wl...@apache.org on 2016/10/31 03:58:49 UTC

[1/2] incubator-hawq git commit: Author: Chunling Wang Date: Mon Oct 31 09:20:24 2016 +0800

Repository: incubator-hawq
Updated Branches:
  refs/heads/master 7ab73156f -> 45adcbdb3


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/tools/bin/hawqregister
----------------------------------------------------------------------
diff --git a/tools/bin/hawqregister b/tools/bin/hawqregister
index 492b151..67a188d 100755
--- a/tools/bin/hawqregister
+++ b/tools/bin/hawqregister
@@ -123,6 +123,25 @@ def register_yaml_dict_check(D, table_column_num, src_tablename):
         logger.error('Column number of table in yaml file is not equals to the column number of table %s.' % src_tablename)
         sys.exit(1)
 
+def ispartition(yml_file):
+    import yaml
+    try:
+        with open(yml_file, 'r') as f:
+            params = yaml.load(f)
+    except yaml.scanner.ScannerError as e:
+        print e
+        sys.exit(1)
+
+    if params['FileFormat'].lower() == 'parquet':
+        Format = 'Parquet'
+    else: #AO format
+        Format = 'AO'
+    Format_FileLocations = '%s_FileLocations' % Format
+    if params.get(Format_FileLocations):
+        partitionby = params.get(Format_FileLocations).get('PartitionBy')
+        if partitionby:
+            return True
+    return False
 
 class FailureHandler(object):
     def __init__(self, conn):
@@ -256,6 +275,23 @@ class GpRegisterAccessor(object):
         rows = self.exec_query(query)
         return rows[0]['attrnums']
 
+    def get_partition_info(self, tablename):
+        ''' Get partition information from pg_partitions, return a constraint-tablename dictionary '''
+        query = "SELECT partitiontablename, partitionboundary FROM pg_partitions WHERE tablename = '%s'" % tablename
+        return self.exec_query(query)
+
+    def get_partitionby(self, tablename):
+        query = "SELECT partitionschemaname, partitiontablename, partitionname, partitiontype, parentpartitiontablename, partitionboundary FROM pg_partitions WHERE tablename='%s';" % tablename
+        parition_type = self.exec_query(query)[0]['partitiontype']
+        query = "SELECT columnname, partitionlevel FROM pg_partition_columns WHERE tablename='%s' ORDER BY position_in_partition_key;" % tablename
+        partition_columnname = self.exec_query(query)[0]['columnname']
+        partitionby = 'PARTITION BY %s (%s)' % (parition_type, partition_columnname)
+        return partitionby
+
+    def get_partition_num(self, tablename):
+        query = "SELECT partitionschemaname from pg_partitions WHERE tablename='%s';" % tablename
+        return len(self.exec_query(query))
+
     def get_bucket_number(self, tablename):
         query = "select oid from pg_class where relname = '%s';" % tablename.split('.')[-1].lower()
         rows = self.exec_query(query)
@@ -307,7 +343,13 @@ class HawqRegister(object):
         self.mode = self._init_mode(options.force, options.repair)
         self.srcfiles = []
         self.dstfiles = []
-        self._init()
+        self.files_same_path = []
+        self.sizes_same_path = []
+        self.segnos_same_path = []
+        self.tupcounts_same_path = []
+        self.varblockcounts_same_path = []
+        self.eofuncompresseds_same_path = []
+        self.segnos_same_path = []
 
     def _init_mode(self, force, repair):
         def table_existed():
@@ -374,7 +416,8 @@ class HawqRegister(object):
 
     def _set_yml_data(self, file_format, files, sizes, tupcounts, eofuncompresseds, varblockcounts, tablename, schema, distribution_policy, file_locations,\
                       bucket_number, partitionby, partitions_constraint, partitions_name, partitions_compression_level,\
-                      partitions_compression_type, partitions_checksum, partitions_filepaths, partitions_filesizes, encoding):
+                      partitions_compression_type, partitions_checksum, partitions_filepaths, partitions_filesizes, \
+                      partitions_tupcounts, partitions_eofuncompresseds, partitions_varblockcounts, encoding):
         self.file_format = file_format
         self.files = files
         self.sizes = sizes
@@ -394,6 +437,9 @@ class HawqRegister(object):
         self.partitions_checksum = partitions_checksum
         self.partitions_filepaths = partitions_filepaths
         self.partitions_filesizes = partitions_filesizes
+        self.partitions_tupcounts = partitions_tupcounts
+        self.partitions_eofuncompresseds = partitions_eofuncompresseds
+        self.partitions_varblockcounts = partitions_varblockcounts
         self.encoding = encoding
 
     def _option_parser_yml(self, yml_file):
@@ -414,6 +460,9 @@ class HawqRegister(object):
         partitions_checksum = []
         partitions_compression_level = []
         partitions_compression_type = []
+        partitions_tupcounts = []
+        partitions_eofuncompresseds = []
+        partitions_varblockcounts = []
         files, sizes, tupcounts, eofuncompresseds, varblockcounts = [], [], [], [], []
 
         if params['FileFormat'].lower() == 'parquet':
@@ -422,9 +471,6 @@ class HawqRegister(object):
             Format = 'AO'
         Format_FileLocations = '%s_FileLocations' % Format
         partitionby = params.get(Format_FileLocations).get('PartitionBy')
-        if partitionby:
-            logger.info('Partition table is not supported in current release of hawq register.')
-            sys.exit(0)
         if params.get(Format_FileLocations).get('Partitions') and len(params[Format_FileLocations]['Partitions']):
             partitions_checksum = [d['Checksum'] for d in params[Format_FileLocations]['Partitions']]
             partitions_compression_level = [d['CompressionLevel'] for d in params[Format_FileLocations]['Partitions']]
@@ -435,6 +481,9 @@ class HawqRegister(object):
                 for pfile in partitions_files:
                     partitions_filepaths.append([params['DFS_URL'] + item['path'] for item in pfile])
                     partitions_filesizes.append([item['size'] for item in pfile])
+                    partitions_tupcounts.append([item['tupcount'] if item.has_key('tupcount') else -1 for item in pfile])
+                    partitions_eofuncompresseds.append([item['eofuncompressed'] if item.has_key('eofuncompressed') else -1 for item in pfile])
+                    partitions_varblockcounts.append([item['varblockcount'] if item.has_key('varblockcount') else -1 for item in pfile])
             partitions_name = [d['Name'] for d in params[Format_FileLocations]['Partitions']]
         if len(params[Format_FileLocations]['Files']):
             for ele in params[Format_FileLocations]['Files']:
@@ -446,7 +495,7 @@ class HawqRegister(object):
 
         encoding = params['Encoding']
         bucketNum = params['Bucketnum'] if params['Distribution_Policy'].startswith('DISTRIBUTED BY') else 6
-        self._set_yml_data(Format, files, sizes, tupcounts, eofuncompresseds, varblockcounts, params['TableName'], params['%s_Schema' % Format], params['Distribution_Policy'], params[Format_FileLocations], bucketNum, partitionby, partitions_constraint, partitions_name, partitions_compression_level, partitions_compression_type, partitions_checksum, partitions_filepaths, partitions_filesizes, encoding)
+        self._set_yml_data(Format, files, sizes, tupcounts, eofuncompresseds, varblockcounts, params['TableName'], params['%s_Schema' % Format], params['Distribution_Policy'], params[Format_FileLocations], bucketNum, partitionby, partitions_constraint, partitions_name, partitions_compression_level, partitions_compression_type, partitions_checksum, partitions_filepaths, partitions_filesizes, partitions_tupcounts, partitions_eofuncompresseds, partitions_varblockcounts, encoding)
 
 
     # check conflicting distributed policy
@@ -528,13 +577,7 @@ class HawqRegister(object):
                     self.failure_handler.rollback()
                     sys.exit(1)
 
-    def _init(self):
-        self.files_same_path = []
-        self.sizes_same_path = []
-        self.tupcounts_same_path = []
-        self.varblockcounts_same_path = []
-        self.eofuncompresseds_same_path = []
-        self.segnos_same_path = []
+    def prepare(self):
         if self.yml:
             self._option_parser_yml(options.yml_config)
             self.filepath = self.files[0][:self.files[0].rfind('/')] if self.files else ''
@@ -673,6 +716,27 @@ class HawqRegister(object):
         if self.file_format == 'Parquet':
             self._check_parquet_format(self.files)
 
+    def test_set_move_files_in_hdfs(self):
+        ''' Output of print shoud be:
+        self.files_update = ['1', '2', '3']
+        self.files_same_path = ['5', '6', 'a']
+        self.srcfiles=['5', '6', 'a', '1', '2', '3']
+        self.dstfiles=['5', '6', '4', '7' , '8', '9']
+        '''
+        self.firstsegno = 4
+        self.files_update = ['1', '2', '3', '5', '6', 'a']
+        self.sizes_update = [1, 2, 3, 4, 5, 6]
+        self.files_append = ['1', '2', '3']
+        self.tupcounts_update = [1, 2, 3, 4, 5, 6]
+        self.eofuncompresseds_update = [1, 2, 3, 4, 5, 6]
+        self.varblockcounts_update = [1, 2, 3, 4, 5, 6]
+        self.tabledir = ''
+        self._set_move_files_in_hdfs()
+        print self.files_update
+        print self.files_same_path
+        print self.srcfiles
+        print self.dstfiles
+
     def _check_files_and_table_in_same_hdfs_cluster(self, filepath, tabledir):
         '''Check whether all the files refered by 'filepath' and the location corresponding to the table are in the same hdfs cluster'''
         if not filepath:
@@ -915,6 +979,88 @@ class HawqRegister(object):
         logger.info('Hawq Register Succeed.')
 
 
+class HawqRegisterPartition(HawqRegister):
+    def __init__(self, options, table, utility_conn, conn, failure_handler):
+        HawqRegister.__init__(self, options, table, utility_conn, conn, failure_handler)
+
+    def _get_partition_info(self):
+        dic = {}
+        for ele in self.accessor.get_partition_info(self.dst_table_name.split('.')[-1]):
+            dic[ele['partitionboundary']] = ele['partitiontablename']
+        return dic
+
+    def _check_partitionby(self):
+        def get_partitionby():
+            return self.accessor.get_partitionby(self.dst_table_name.split('.')[-1])
+
+        if self.partitionby != get_partitionby():
+            logger.error('PartitionBy of %s is not consistent with previous partitionby.' % self.tablename)
+            self.failure_handler.rollback()
+            sys.exit(1)
+
+    def _check_partition_num(self):
+        def get_partition_num():
+            return self.accessor.get_partition_num(self.dst_table_name.split('.')[-1])
+
+        if get_partition_num() < len(self.partitions_name):
+            logger.error('Partition Number of %s is not consistent with previous partition number.' % self.tablename)
+            self.failure_handler.rollback()
+            sys.exit(1)
+
+    def _check_duplicate_constraint(self):
+        partitions_constraint = sorted(self.partitions_constraint)
+        for k, _ in enumerate(partitions_constraint):
+            if k < len(partitions_constraint) - 1 and partitions_constraint[k] == partitions_constraint[k+1]:
+                logger.error('Partition Constraint "%s" in table %s is duplicated' % (partitions_constraint[k], self.tablename))
+                self.failure_handler.rollback()
+                sys.exit(1)
+
+    def prepare(self):
+        if self.yml:
+            self._option_parser_yml(options.yml_config)
+        else:
+            if self._is_folder(self.filepath) and self.filesize:
+                logger.error('-e option is only supported with single file case.')
+                sys.exit(1)
+            self.file_format = 'Parquet'
+            self._check_hash_type() # Usage1 only support randomly distributed table
+        for k, pn in enumerate(self.partitions_name):
+            self.tablename = pn
+            self.files = self.partitions_filepaths[k]
+            self.sizes = self.partitions_filesizes[k]
+            if self.yml:
+                self.filepath = self.files[0][:self.files[0].rfind('/')] if self.files else ''
+                self._check_file_not_folder()
+        if self.yml:
+            self._check_database_encoding()
+            if self.mode != 'repair':
+                if not self._create_table() and self.mode != 'force':
+                    self.mode = 'usage2_table_exist'
+                    self._check_partitionby()
+                    self._check_partition_num()
+        partitions = self._get_partition_info()
+        self.queries = "set allow_system_table_mods='dml';"
+        self.queries += "begin transaction;"
+        self._check_duplicate_constraint()
+        for k, pn in enumerate(self.partitions_name):
+            self.constraint = self.partitions_constraint[k]
+            if not partitions.has_key(self.constraint):
+                logger.error('Partition Constraint "%s" is not in table %s' % (self.constraint, self.tablename))
+                self.failure_handler.rollback()
+                sys.exit(1)
+            self.tablename = partitions[self.constraint]
+            self.files = self.partitions_filepaths[k]
+            self.sizes = self.partitions_filesizes[k]
+            self.tupcounts = self.partitions_tupcounts[k]
+            self.eofuncompresseds = self.partitions_eofuncompresseds[k]
+            self.varblockcounts = self.partitions_varblockcounts[k]
+            self._do_check()
+            self._prepare_register()
+        self.queries += "end transaction;"
+
+    def register(self):
+        HawqRegister.register(self)
+
 def main(options, args):
     def connectdb(options):
         '''
@@ -939,10 +1085,41 @@ def main(options, args):
 
     failure_handler = FailureHandler(conn)
     # register
-    ins = HawqRegister(options, args[0], utility_conn, conn, failure_handler)
+    if options.yml_config and ispartition(options.yml_config):
+        ins = HawqRegisterPartition(options, args[0], utility_conn, conn, failure_handler)
+    else:
+        ins = HawqRegister(options, args[0], utility_conn, conn, failure_handler)
+    ins.prepare()
     ins.register()
     conn.close()
 
+def test(options, args):
+    def connectdb(options):
+        '''
+        Trying to connect database, return a connection object.
+        If failed to connect, raise a pg.InternalError
+        '''
+        url = dbconn.DbURL(hostname=options.host, port=options.port,
+                           dbname=options.database, username=options.user)
+        logger.info('try to connect database %s:%s %s' % (url.pghost, url.pgport, url.pgdb))
+        utility_conn = pg.connect(dbname=url.pgdb, host=url.pghost, port=url.pgport,
+                                  user=url.pguser, passwd=url.pgpass, opt='-c gp_session_role=utility')
+        conn = pg.connect(dbname=url.pgdb, host=url.pghost, port=url.pgport,
+                          user=url.pguser, passwd=url.pgpass)
+        return utility_conn, conn
+
+    # connect db
+    try:
+        utility_conn, conn = connectdb(options)
+    except pg.InternalError:
+        logger.error('Fail to connect to database, this script can only be run when database is up.')
+        return 1
+
+    failure_handler = FailureHandler(conn)
+    # register
+    ins = HawqRegister(options, args[0], utility_conn, conn, failure_handler)
+    ins.test_set_move_files_in_hdfs()
+
 
 if __name__ == '__main__':
     parser = option_parser()


[2/2] incubator-hawq git commit: Author: Chunling Wang Date: Mon Oct 31 09:20:24 2016 +0800

Posted by wl...@apache.org.
Author: Chunling Wang <wa...@126.com>
Date:   Mon Oct 31 09:20:24 2016 +0800

HAWQ-1035. Support partition table register.


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

Branch: refs/heads/master
Commit: 45adcbdb3fe6bbe227d985f8ff10ddee6c8eb720
Parents: 7ab7315
Author: Wen Lin <wl...@pivotal.io>
Authored: Mon Oct 31 11:38:02 2016 +0800
Committer: Wen Lin <wl...@pivotal.io>
Committed: Mon Oct 31 11:38:02 2016 +0800

----------------------------------------------------------------------
 .../backup_usage2case1/bucket0_tpl.yml          |  31 --
 .../contain_error_symbol_tpl.yml                |  31 --
 .../backup_usage2case1/error_blockszie_tpl.yml  |  31 --
 .../backup_usage2case1/error_checksum_tpl.yml   |  31 --
 .../error_compresslevel_tpl.yml                 |  31 --
 .../error_compresstype_tpl.yml                  |  31 --
 .../backup_usage2case1/error_encoding_tpl.yml   |  31 --
 .../backup_usage2case1/error_pagesize_tpl.yml   |  31 --
 .../error_rowgroupsize_tpl.yml                  |  31 --
 .../backup_usage2case1/error_schema_tpl.yml     |  33 ---
 .../backup_usage2case1/float_eof_tpl.yml        |  31 --
 .../backup_usage2case1/larger_eof_tpl.yml       |  31 --
 .../backup_usage2case1/minus_eof_tpl.yml        |  31 --
 .../wrong_distributed_policy_tpl.yml            |  31 --
 .../constraint_not_exists_in_catalog_tpl.yml    |  98 +++++++
 .../different_partition_policy_tpl.yml          |  98 +++++++
 .../duplicate_partition_constraint_tpl.yml      |  98 +++++++
 .../partition/force_mode_normal_tpl.yml         | 290 +++++++++++++++++++
 .../partition/sub_set_catalog_table_tpl.yml     |  79 +++++
 .../table_exists_list_partition_tpl.yml         |  98 +++++++
 .../table_exists_range_partition_tpl.yml        |  98 +++++++
 .../table_exists_table_file_not_exists_tpl.yml  |  98 +++++++
 .../table_not_exists_list_partition_tpl.yml     |  98 +++++++
 .../table_not_exists_range_partition_tpl.yml    |  98 +++++++
 ...ble_not_exists_table_file_not_exists_tpl.yml |  98 +++++++
 .../feature/ManagementTool/test_hawq_register.h |   3 +
 .../test_hawq_register_partition.cpp            | 215 ++++++++++++++
 tools/bin/hawqregister                          | 205 ++++++++++++-
 28 files changed, 1660 insertions(+), 450 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/bucket0_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/bucket0_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/bucket0_tpl.yml
deleted file mode 100644
index b407317..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/bucket0_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 0
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/contain_error_symbol_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/contain_error_symbol_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/contain_error_symbol_tpl.yml
deleted file mode 100644
index fbec3a3..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/contain_error_symbol_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/*/../1
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_blockszie_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_blockszie_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_blockszie_tpl.yml
deleted file mode 100644
index 90c8c61..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_blockszie_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32767
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_checksum_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_checksum_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_checksum_tpl.yml
deleted file mode 100644
index e21c511..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_checksum_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: true
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_compresslevel_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_compresslevel_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_compresslevel_tpl.yml
deleted file mode 100644
index 46a8cfb..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_compresslevel_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 1
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_compresstype_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_compresstype_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_compresstype_tpl.yml
deleted file mode 100644
index 77400d1..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_compresstype_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: SNAPPY
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_encoding_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_encoding_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_encoding_tpl.yml
deleted file mode 100644
index b98a04c..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_encoding_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: Unicode
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_pagesize_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_pagesize_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_pagesize_tpl.yml
deleted file mode 100644
index 4c2da07..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_pagesize_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_rowgroupsize_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_rowgroupsize_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_rowgroupsize_tpl.yml
deleted file mode 100644
index 4c2da07..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_rowgroupsize_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/error_schema_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/error_schema_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/error_schema_tpl.yml
deleted file mode 100644
index 77f61c9..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/error_schema_tpl.yml
+++ /dev/null
@@ -1,33 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-- name: j
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/float_eof_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/float_eof_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/float_eof_tpl.yml
deleted file mode 100644
index 193f973..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/float_eof_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216.0
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/larger_eof_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/larger_eof_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/larger_eof_tpl.yml
deleted file mode 100644
index 7055310..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/larger_eof_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 2016
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/minus_eof_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/minus_eof_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/minus_eof_tpl.yml
deleted file mode 100644
index 32d60f1..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/minus_eof_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: -1
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (i)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/backup_usage2case1/wrong_distributed_policy_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/backup_usage2case1/wrong_distributed_policy_tpl.yml b/src/test/feature/ManagementTool/backup_usage2case1/wrong_distributed_policy_tpl.yml
deleted file mode 100644
index 55ad2c2..0000000
--- a/src/test/feature/ManagementTool/backup_usage2case1/wrong_distributed_policy_tpl.yml
+++ /dev/null
@@ -1,31 +0,0 @@
-AO_FileLocations:
-  Blocksize: 32768
-  Checksum: false
-  CompressionLevel: 0
-  CompressionType: null
-  Files:
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
-    size: 176
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
-    size: 168
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
-    size: 192
-  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
-    size: 216
-AO_Schema:
-- name: i
-  type: int4
-Bucketnum: 6
-DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
-  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
-  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
-DFS_URL: hdfs://localhost:8020
-Distribution_Policy: DISTRIBUTED BY (j)
-Encoding: UTF8
-FileFormat: AO
-TableName: public.t
-Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/constraint_not_exists_in_catalog_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/constraint_not_exists_in_catalog_tpl.yml b/src/test/feature/ManagementTool/partition/constraint_not_exists_in_catalog_tpl.yml
new file mode 100644
index 0000000..c25d7c9
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/constraint_not_exists_in_catalog_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION group2 VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/different_partition_policy_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/different_partition_policy_tpl.yml b/src/test/feature/ManagementTool/partition/different_partition_policy_tpl.yml
new file mode 100644
index 0000000..739c872
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/different_partition_policy_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (year)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/duplicate_partition_constraint_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/duplicate_partition_constraint_tpl.yml b/src/test/feature/ManagementTool/partition/duplicate_partition_constraint_tpl.yml
new file mode 100644
index 0000000..4de77fb
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/duplicate_partition_constraint_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/force_mode_normal_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/force_mode_normal_tpl.yml b/src/test/feature/ManagementTool/partition/force_mode_normal_tpl.yml
new file mode 100644
index 0000000..a0a75c7
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/force_mode_normal_tpl.yml
@@ -0,0 +1,290 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/1
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/2
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/3
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/4
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/5
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD@/6
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/1
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/2
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/3
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/4
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/5
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  - eofuncompressed: 0
+    path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW@/6
+    size: 0
+    tupcount: 0
+    varblockcount: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - eofuncompressed: 432
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/1
+      size: 432
+      tupcount: 16
+      varblockcount: 1
+    - eofuncompressed: 408
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/2
+      size: 408
+      tupcount: 15
+      varblockcount: 1
+    - eofuncompressed: 464
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/3
+      size: 464
+      tupcount: 17
+      varblockcount: 1
+    - eofuncompressed: 408
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/4
+      size: 408
+      tupcount: 15
+      varblockcount: 1
+    - eofuncompressed: 464
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/5
+      size: 464
+      tupcount: 17
+      varblockcount: 1
+    - eofuncompressed: 536
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD1@/6
+      size: 536
+      tupcount: 20
+      varblockcount: 1
+    - eofuncompressed: 280
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/1
+      size: 280
+      tupcount: 10
+      varblockcount: 1
+    - eofuncompressed: 224
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/2
+      size: 224
+      tupcount: 8
+      varblockcount: 1
+    - eofuncompressed: 224
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/3
+      size: 224
+      tupcount: 8
+      varblockcount: 1
+    - eofuncompressed: 256
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/4
+      size: 256
+      tupcount: 9
+      varblockcount: 1
+    - eofuncompressed: 200
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/5
+      size: 200
+      tupcount: 7
+      varblockcount: 1
+    - eofuncompressed: 224
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW1@/6
+      size: 224
+      tupcount: 8
+      varblockcount: 1
+    Name: nt_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/1
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/2
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/3
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/4
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/5
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD2@/6
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/1
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/2
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/3
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/4
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/5
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW2@/6
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    Name: nt_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/1
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/2
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/3
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/4
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/5
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_OLD3@/6
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/1
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/2
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/3
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/4
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/5
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    - eofuncompressed: 0
+      path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID_NEW3@/6
+      size: 0
+      tupcount: 0
+      varblockcount: 0
+    Name: nt_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.nt
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/sub_set_catalog_table_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/sub_set_catalog_table_tpl.yml b/src/test/feature/ManagementTool/partition/sub_set_catalog_table_tpl.yml
new file mode 100644
index 0000000..5cba94e
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/sub_set_catalog_table_tpl.yml
@@ -0,0 +1,79 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_exists_list_partition_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_exists_list_partition_tpl.yml b/src/test/feature/ManagementTool/partition/table_exists_list_partition_tpl.yml
new file mode 100644
index 0000000..184a675
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_exists_list_partition_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_exists_range_partition_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_exists_range_partition_tpl.yml b/src/test/feature/ManagementTool/partition/table_exists_range_partition_tpl.yml
new file mode 100644
index 0000000..557e3e7
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_exists_range_partition_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY range (id)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls START (1) END (41) EVERY (40) WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 256
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 176
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 176
+    Name: t_1_prt_group1
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys START (41) END (81) EVERY (40) WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 224
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 176
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 256
+    Name: t_1_prt_group2
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 72
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 96
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 120
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 72
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 120
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 152
+    Name: t_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.t
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_exists_table_file_not_exists_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_exists_table_file_not_exists_tpl.yml b/src/test/feature/ManagementTool/partition/table_exists_table_file_not_exists_tpl.yml
new file mode 100644
index 0000000..ae74f0d
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_exists_table_file_not_exists_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/7
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_not_exists_list_partition_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_not_exists_list_partition_tpl.yml b/src/test/feature/ManagementTool/partition/table_not_exists_list_partition_tpl.yml
new file mode 100644
index 0000000..184a675
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_not_exists_list_partition_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_not_exists_range_partition_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_not_exists_range_partition_tpl.yml b/src/test/feature/ManagementTool/partition/table_not_exists_range_partition_tpl.yml
new file mode 100644
index 0000000..557e3e7
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_not_exists_range_partition_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY range (id)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls START (1) END (41) EVERY (40) WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 256
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 176
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 176
+    Name: t_1_prt_group1
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys START (41) END (81) EVERY (40) WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 152
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 224
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 176
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 200
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 256
+    Name: t_1_prt_group2
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 72
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 96
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 120
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/4
+      size: 72
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 120
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 152
+    Name: t_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.t
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/partition/table_not_exists_table_file_not_exists_tpl.yml
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/partition/table_not_exists_table_file_not_exists_tpl.yml b/src/test/feature/ManagementTool/partition/table_not_exists_table_file_not_exists_tpl.yml
new file mode 100644
index 0000000..ae74f0d
--- /dev/null
+++ b/src/test/feature/ManagementTool/partition/table_not_exists_table_file_not_exists_tpl.yml
@@ -0,0 +1,98 @@
+AO_FileLocations:
+  Blocksize: 32768
+  Checksum: false
+  CompressionLevel: 0
+  CompressionType: null
+  Files:
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/1
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/2
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/3
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/4
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/5
+    size: 0
+  - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID@/6
+    size: 0
+  PartitionBy: PARTITION BY list (gender)
+  Partitions:
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION girls VALUES('F') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/1
+      size: 432
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/2
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/3
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/4
+      size: 408
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/5
+      size: 464
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID1@/6
+      size: 536
+    Name: rank_1_prt_girls
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: PARTITION boys VALUES('M') WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/4
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID2@/6
+      size: 0
+    Name: rank_1_prt_boys
+  - Blocksize: 32768
+    Checksum: false
+    CompressionLevel: 0
+    CompressionType: null
+    Constraint: DEFAULT PARTITION other  WITH (appendonly=true)
+    Files:
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/1
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/2
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/3
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/7
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/5
+      size: 0
+    - path: /hawq_default/16385/@DATABASE_OID@/@TABLE_OID3@/6
+      size: 0
+    Name: rank_1_prt_other
+AO_Schema:
+- name: id
+  type: int4
+- name: rank
+  type: int4
+- name: year
+  type: int4
+- name: gender
+  type: bpchar
+- name: count
+  type: int4
+Bucketnum: 6
+DBVersion: PostgreSQL 8.2.15 (Greenplum Database 4.2.0 build 1) (HAWQ 2.0.1.0 build
+  dev) on x86_64-apple-darwin14.5.0, compiled by GCC Apple LLVM version 6.1.0 (clang-602.0.53)
+  (based on LLVM 3.6.0svn) compiled on Jul  6 2016 10:22:33
+DFS_URL: hdfs://@PORT@
+Distribution_Policy: DISTRIBUTED BY (id)
+Encoding: UTF8
+FileFormat: AO
+TableName: public.rank
+Version: 1.0.0

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/test_hawq_register.h
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/test_hawq_register.h b/src/test/feature/ManagementTool/test_hawq_register.h
index dce1c8c..c5f6658 100644
--- a/src/test/feature/ManagementTool/test_hawq_register.h
+++ b/src/test/feature/ManagementTool/test_hawq_register.h
@@ -95,6 +95,9 @@ class TestHawqRegister : public ::testing::Test {
         void runYamlCaseTableNotExists(std::string casename, std::string ymlname, int expectederror, int checknum);
         void runYamlCaseTableExists(std::string casename, std::string ymlname, int isexpectederror, int checknum);
         void runYamlCaseForceMode(std::string casename, std::string ymlname, int isexpectederror, int rows, int checknum, bool samepath);
+        void runYamlCaseTableNotExistsPartition(std::string casename, std::string ymlname, int expectederror, int checknum, int islistpartition);
+        void runYamlCaseTableExistsPartition(std::string casename, std::string ymlname, int isexpectederror, int checknum, int islistpartition);
+        void runYamlCaseForceModePartition(std::string casename, std::string ymlname, int isexpectederror, int rows, int checknum);
 
     private:
         std::unique_ptr<hawq::test::PSQL> conn;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/45adcbdb/src/test/feature/ManagementTool/test_hawq_register_partition.cpp
----------------------------------------------------------------------
diff --git a/src/test/feature/ManagementTool/test_hawq_register_partition.cpp b/src/test/feature/ManagementTool/test_hawq_register_partition.cpp
new file mode 100644
index 0000000..3decf6e
--- /dev/null
+++ b/src/test/feature/ManagementTool/test_hawq_register_partition.cpp
@@ -0,0 +1,215 @@
+#include <string>
+
+#include "lib/command.h"
+#include "lib/sql_util.h"
+#include "lib/string_util.h"
+#include "lib/hdfs_config.h"
+#include "lib/file_replace.h"
+#include "test_hawq_register.h"
+
+#include "gtest/gtest.h"
+
+using std::string;
+using hawq::test::SQLUtility;
+using hawq::test::Command;
+using hawq::test::HdfsConfig;
+
+void TestHawqRegister::runYamlCaseTableExistsPartition(std::string casename, std::string ymlname, int isexpectederror=1, int checknum=100, int islistpartition=1) {
+    SQLUtility util;
+    string test_root(util.getTestRootPath());
+    util.execute("drop table if exists t;");
+    util.execute("drop table if exists nt;");
+    
+    if (islistpartition > 0) {
+        util.execute("CREATE TABLE t (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY LIST (gender) ( PARTITION girls VALUES ('F'), PARTITION boys VALUES ('M'), DEFAULT PARTITION other );");
+        util.execute("CREATE TABLE nt (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY LIST (gender) ( PARTITION girls VALUES ('F'), PARTITION boys VALUES ('M'), DEFAULT PARTITION other );");
+    } else {
+        util.execute("CREATE TABLE t (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY RANGE (id) (PARTITION girls START (1) END (41) EVERY(40), PARTITION boys START (41) END (81) EVERY(40), DEFAULT PARTITION other);");
+        util.execute("CREATE TABLE nt (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY RANGE (id) (PARTITION girls START (1) END (41) EVERY(40), PARTITION boys START (41) END (81) EVERY(40), DEFAULT PARTITION other);");
+    }
+    util.execute("insert into t select generate_series(1, 100), 1, 1, 'F', 1;");
+    util.execute("insert into nt select generate_series(1, 100), 1, 1, 'F', 1;");
+    util.query("select * from t;", 100);
+    util.query("select * from nt;", 100);
+    string t_yml(hawq::test::stringFormat("%s/ManagementTool/%s.yml", test_root.c_str(), ymlname.c_str()));
+    string t_yml_tpl(hawq::test::stringFormat("%s/ManagementTool/%s_tpl.yml", test_root.c_str(), ymlname.c_str()));
+    hawq::test::FileReplace frep;
+    std::unordered_map<std::string, std::string> strs_src_dst;
+    strs_src_dst["@DATABASE_OID@"]= getDatabaseOid();
+    strs_src_dst["@TABLE_OID@"]= getTableOid("t");
+    strs_src_dst["@TABLE_OID1@"]= getTableOid("t_1_prt_girls");
+    strs_src_dst["@TABLE_OID2@"]= getTableOid("t_1_prt_boys");
+    strs_src_dst["@TABLE_OID3@"]= getTableOid("t_1_prt_other");
+    hawq::test::HdfsConfig hc;
+    string hdfs_prefix;
+    hc.getNamenodeHost(hdfs_prefix);
+    strs_src_dst["@PORT@"]= hdfs_prefix;
+    frep.replace(t_yml_tpl, t_yml, strs_src_dst);
+    EXPECT_EQ(isexpectederror, Command::getCommandStatus(hawq::test::stringFormat("hawq register -d %s -c %s testhawqregister_%s.nt", HAWQ_DB, t_yml.c_str(), casename.c_str())));
+    if (isexpectederror > 0) {
+        util.query("select * from t;", 100);
+        util.query("select * from nt;", 100);
+    } else {
+        util.query("select * from nt;", checknum);
+    }
+    
+    EXPECT_EQ(0, Command::getCommandStatus(hawq::test::stringFormat("rm -rf %s", t_yml.c_str())));
+    util.execute("drop table t;");
+    util.execute("drop table nt;");
+}
+
+void TestHawqRegister::runYamlCaseTableNotExistsPartition(std::string casename, std::string ymlname, int isexpectederror=1, int checknum=100, int islistpartition=1) {
+    SQLUtility util;
+    string test_root(util.getTestRootPath());
+    util.execute("drop table if exists t;");
+    util.execute("drop table if exists nt;");
+    
+    if (islistpartition > 0) {
+        util.execute("CREATE TABLE t (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY LIST (gender) ( PARTITION girls VALUES ('F'), PARTITION boys VALUES ('M'), DEFAULT PARTITION other );");
+    } else {
+        util.execute("CREATE TABLE t (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY RANGE (id) (PARTITION girls START (1) END (41) EVERY(40), PARTITION boys START (41) END (81) EVERY(40), DEFAULT PARTITION other);");
+    }
+    util.execute("insert into t select generate_series(1, 100), 1, 1, 'F', 1;");
+    util.query("select * from t;", 100);
+    string t_yml(hawq::test::stringFormat("%s/ManagementTool/%s.yml", test_root.c_str(), ymlname.c_str()));
+    string t_yml_tpl(hawq::test::stringFormat("%s/ManagementTool/%s_tpl.yml", test_root.c_str(), ymlname.c_str()));
+    hawq::test::FileReplace frep;
+    std::unordered_map<std::string, std::string> strs_src_dst;
+    strs_src_dst["@DATABASE_OID@"]= getDatabaseOid();
+    strs_src_dst["@TABLE_OID@"]= getTableOid("t");
+    strs_src_dst["@TABLE_OID1@"]= getTableOid("t_1_prt_girls");
+    strs_src_dst["@TABLE_OID2@"]= getTableOid("t_1_prt_boys");
+    strs_src_dst["@TABLE_OID3@"]= getTableOid("t_1_prt_other");
+    hawq::test::HdfsConfig hc;
+    string hdfs_prefix;
+    hc.getNamenodeHost(hdfs_prefix);
+    strs_src_dst["@PORT@"]= hdfs_prefix;
+    frep.replace(t_yml_tpl, t_yml, strs_src_dst);
+    EXPECT_EQ(isexpectederror, Command::getCommandStatus(hawq::test::stringFormat("hawq register -d %s -c %s testhawqregister_%s.nt", HAWQ_DB, t_yml.c_str(), casename.c_str())));
+    if (isexpectederror > 0) {
+        util.query("select * from t;", 100);
+        util.query("select * from pg_class where relname = 'nt';", 0);
+    } else {
+        util.query("select * from nt;", checknum);
+    }
+    
+    EXPECT_EQ(0, Command::getCommandStatus(hawq::test::stringFormat("rm -rf %s", t_yml.c_str())));
+    util.execute("drop table t;");
+    util.execute("drop table if exists nt;");
+}
+
+void TestHawqRegister::runYamlCaseForceModePartition(std::string casename, std::string ymlname, int isexpectederror = 1, int rows = 50, int checknum = 200) {
+    SQLUtility util;
+    string test_root(util.getTestRootPath());
+    string t_yml_tpl(hawq::test::stringFormat("%s/ManagementTool/%s_tpl.yml", test_root.c_str(), ymlname.c_str()));
+    string t_yml(hawq::test::stringFormat("%s/ManagementTool/%s.yml", test_root.c_str(), ymlname.c_str()));
+    util.execute("drop table if exists t;");
+    util.execute("drop table if exists nt;");
+
+    util.execute("CREATE TABLE nt (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY LIST (gender) ( PARTITION girls VALUES ('F'), PARTITION boys VALUES ('M'), DEFAULT PARTITION other );");
+    util.execute("insert into nt select generate_series(1, 100), 1, 1, 'F', 1;");
+    // get pg_aoseg.pg_xxxseg_xxx table
+    std::string reloid1_1_1 = getTableOid("nt_1_prt_girls");
+    std::string reloid1_2_1 = getTableOid("nt_1_prt_boys");
+    std::string reloid1_3_1 = getTableOid("nt_1_prt_other");
+    string result1_1_1 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_1_1.c_str()));
+    string result1_2_1 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_2_1.c_str()));
+    string result1_3_1 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_3_1.c_str()));
+    util.execute("insert into nt select generate_series(1, 100), 1, 1, 'F', 1;");
+    util.query("select * from nt;", 200);
+
+    // hawq register --force -d hawq_feature_test -c t_new_#.yml nt_usage2_case2_#
+    util.execute("CREATE TABLE t (id int, rank int, year int, gender char(1), count int ) DISTRIBUTED BY (id) PARTITION BY LIST (gender) ( PARTITION girls VALUES ('F'), PARTITION boys VALUES ('M'), DEFAULT PARTITION other );");
+    util.execute(hawq::test::stringFormat("insert into t select generate_series(1, %s), 1, 1, 'F', 1;", std::to_string(rows).c_str()));
+    // get pg_aoseg.pg_xxxseg_xxx table
+    std::string reloid1_1_2 = getTableOid("t_1_prt_girls");
+    std::string reloid1_2_2 = getTableOid("t_1_prt_boys");
+    std::string reloid1_3_2 = getTableOid("t_1_prt_other");
+    string result1_1_2 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_1_2.c_str()));
+    string result1_2_2 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_2_2.c_str()));
+    string result1_3_2 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid1_3_2.c_str()));
+    util.query("select * from t;", rows);
+    hawq::test::FileReplace frep;
+    std::unordered_map<std::string, std::string> strs_src_dst;
+    strs_src_dst["@DATABASE_OID@"]= getDatabaseOid();
+    strs_src_dst["@TABLE_OID_OLD@"]= getTableOid("nt");
+    strs_src_dst["@TABLE_OID_NEW@"]= getTableOid("t");
+    strs_src_dst["@TABLE_OID_OLD1@"]= getTableOid("nt_1_prt_girls");
+    strs_src_dst["@TABLE_OID_OLD2@"]= getTableOid("nt_1_prt_boys");
+    strs_src_dst["@TABLE_OID_OLD3@"]= getTableOid("nt_1_prt_other");
+    strs_src_dst["@TABLE_OID_NEW1@"]= getTableOid("t_1_prt_girls");
+    strs_src_dst["@TABLE_OID_NEW2@"]= getTableOid("t_1_prt_boys");
+    strs_src_dst["@TABLE_OID_NEW3@"]= getTableOid("t_1_prt_other");
+    string hdfs_prefix;
+    hawq::test::HdfsConfig hc;
+    hc.getNamenodeHost(hdfs_prefix);
+    strs_src_dst["@PORT@"]= hdfs_prefix;
+    frep.replace(t_yml_tpl, t_yml, strs_src_dst);
+    
+    EXPECT_EQ(isexpectederror, Command::getCommandStatus(hawq::test::stringFormat("hawq register --force -d %s -c %s testhawqregister_%s.nt", HAWQ_DB, t_yml.c_str(), casename.c_str())));
+    util.query("select * from nt;", checknum);
+
+    // check pg_aoseg.pg_xxxseg_xxx table
+    std::string reloid2_1 = getTableOid("nt_1_prt_girls");
+    std::string reloid2_2 = getTableOid("nt_1_prt_boys");
+    std::string reloid2_3 = getTableOid("nt_1_prt_other");
+    string result2_1 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid2_1.c_str()));
+    string result2_2 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid2_2.c_str()));
+    string result2_3 = util.getQueryResultSetString(hawq::test::stringFormat("select eof, tupcount, varblockcount, eofuncompressed from pg_aoseg.pg_aoseg_%s order by segno;", reloid2_3.c_str()));
+
+    result1_1_1.substr(0, result1_1_1.length() - 1);
+    result1_2_1.substr(0, result1_1_1.length() - 1);
+    result1_3_1.substr(0, result1_1_1.length() - 1);
+    EXPECT_EQ(result1_1_1 + result1_1_2, result2_1);
+    EXPECT_EQ(result1_2_1 + result1_2_2, result2_2);
+    EXPECT_EQ(result1_3_1 + result1_3_2, result2_3);
+
+    EXPECT_EQ(0, Command::getCommandStatus(hawq::test::stringFormat("rm -rf %s", t_yml.c_str())));
+    util.execute("drop table t;");
+    util.execute("drop table nt;");
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableNotExistsListPartition) {
+    runYamlCaseTableNotExistsPartition("testpartitiontablenotexistslistpartition", "partition/table_not_exists_list_partition", 0, 100);
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableNotExistsRangePartition) {
+    runYamlCaseTableNotExistsPartition("testpartitiontablenotexistsrangepartition", "partition/table_not_exists_range_partition", 0, 100, 0);
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableExistsListPartition) {
+    runYamlCaseTableExistsPartition("testpartitiontableexistslistpartition", "partition/table_exists_list_partition", 0, 200);
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableExistsRangePartition) {
+    runYamlCaseTableExistsPartition("testpartitiontableexistsrangepartition", "partition/table_exists_range_partition", 0, 200, 0);
+}
+
+TEST_F(TestHawqRegister, TestPartitionForceModeNormal) {
+    runYamlCaseForceModePartition("testpartitionforcemodenormal", "partition/force_mode_normal", 0, 50, 150); 
+}
+
+TEST_F(TestHawqRegister, TestPartitionSubSetCatalogTable) {
+    runYamlCaseTableExistsPartition("testpartitionsubsetcatalogtable", "partition/sub_set_catalog_table", 0, 200);
+}
+
+TEST_F(TestHawqRegister, TestPartitionDifferentPartitionPolicy) {
+    runYamlCaseTableExistsPartition("testpartitiondifferentpartitionpolicy", "partition/different_partition_policy");
+}
+
+TEST_F(TestHawqRegister, TestPartitionConstraintNotExistsInCatalog) {
+    runYamlCaseTableExistsPartition("testpartitionconstraintnotexistsincatalog", "partition/constraint_not_exists_in_catalog");
+}
+
+TEST_F(TestHawqRegister, TestPartitionDuplicatePartitionConstraint) {
+    runYamlCaseTableExistsPartition("testpartitionduplicatepartitionconstraint", "partition/duplicate_partition_constraint");
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableNotExistsTableFileNotExists) {
+    runYamlCaseTableNotExistsPartition("testpartitiontablenotexiststablefilenotexists", "partition/table_not_exists_table_file_not_exists");
+}
+
+TEST_F(TestHawqRegister, TestPartitionTableExistsTableFileNotExists) {
+    runYamlCaseTableExistsPartition("testpartitiontableexiststablefilenotexists", "partition/table_exists_table_file_not_exists");
+}
+