You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by zt...@apache.org on 2021/12/15 03:23:12 UTC

[hawq] branch ztao updated: HAWQ-1822. Enable heap table

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

ztao1987 pushed a commit to branch ztao
in repository https://gitbox.apache.org/repos/asf/hawq.git


The following commit(s) were added to refs/heads/ztao by this push:
     new 4f67d22  HAWQ-1822. Enable heap table
4f67d22 is described below

commit 4f67d22e1cd242a4be2660d0eda20e8cbe1c926f
Author: ztao1987 <zh...@gmail.com>
AuthorDate: Wed Dec 15 11:22:12 2021 +0800

    HAWQ-1822. Enable heap table
---
 contrib/magma/magma.c              |  2 +-
 src/backend/commands/indexcmds.c   |  4 ++--
 src/backend/commands/tablecmds.c   | 16 +++++++++-------
 src/backend/parser/analyze.c       | 15 ++++++++++++++-
 src/backend/postmaster/identity.c  | 23 +++++++++++++++++++++++
 src/backend/tcop/utility.c         | 12 ++++++++----
 src/backend/utils/cache/relcache.c |  3 ++-
 src/backend/utils/misc/guc.c       | 21 +++++++++++++++++++++
 src/include/catalog/pg_class.h     |  2 +-
 src/include/postmaster/identity.h  |  1 +
 src/include/utils/guc.h            |  9 +++++++++
 11 files changed, 91 insertions(+), 17 deletions(-)

diff --git a/contrib/magma/magma.c b/contrib/magma/magma.c
index b7d6758..c732b85 100644
--- a/contrib/magma/magma.c
+++ b/contrib/magma/magma.c
@@ -409,7 +409,7 @@ Datum magma_protocol_blocklocation(PG_FUNCTION_ARGS) {
   MagmaTablePtr table = MagmaClientC_FetchTable(client, useClientCacheDirectly);
   magma_check_result(&client);
 
-  elog(LOG, "magma_protocol_blocklocation pass fetch table");
+  elog(DEBUG3, "magma_protocol_blocklocation pass fetch table");
 
   /*
    * Step 3. map ranges to block locations
diff --git a/src/backend/commands/indexcmds.c b/src/backend/commands/indexcmds.c
index 345de0b..464a82e 100644
--- a/src/backend/commands/indexcmds.c
+++ b/src/backend/commands/indexcmds.c
@@ -755,8 +755,8 @@ DefineIndex(Oid relationId,
         }
         else
         {
-        	/* magma and native orc support index */
-        	if (!(RelationIsOrc(rel) || RelationIsMagmaTable2(relationId)))
+                /* Native orc, heap table and magma support index */
+                if (!RelationIsOrc(rel) && !RelationIsHeap(rel) && !RelationIsMagmaTable(relationId))
         	{
         		ereport(ERROR, (errcode(ERRCODE_CDB_FEATURE_NOT_YET), errmsg("Cannot support DefineIndex")));
         	}
diff --git a/src/backend/commands/tablecmds.c b/src/backend/commands/tablecmds.c
index a92eaa6..a21d803 100644
--- a/src/backend/commands/tablecmds.c
+++ b/src/backend/commands/tablecmds.c
@@ -499,8 +499,8 @@ DefineRelation(CreateStmt *stmt, char relkind, char relstorage,
     Oid reloid = 0;
     Assert(stmt->base.relation->schemaname == NULL || strlen(stmt->base.relation->schemaname)>0);
 
-    /* forbid create non-system table on tablespace pg_default */
-    if((!IsBootstrapProcessingMode()) && (isPgDefaultTablespace(stmt->base.tablespacename)))
+    /* forbid create non-system table on tablespace pg_default when enable_pg_default_for_non_system_table is not true */
+    if(!enable_pg_default_for_non_system_table && (!IsBootstrapProcessingMode()) && (isPgDefaultTablespace(stmt->base.tablespacename)))
     {
     	ereport(ERROR,
     					(errcode(ERRCODE_CDB_FEATURE_NOT_YET),
@@ -605,11 +605,13 @@ DefineRelation_int(CreateStmt *stmt,
 		 * dispatch.
 		 */
 	  if (pg_strcasecmp("pg_default", stmt->base.tablespacename) == 0)
-	        ereport(ERROR,
-	            (errcode(ERRCODE_UNDEFINED_OBJECT),
-	             errmsg("can not create ao format table on tablespace \"%s\" ",
-	                stmt->base.tablespacename),
-	             errOmitLocation(true)));
+	        if (!enable_pg_default_for_non_system_table) {  
+                        ereport(ERROR,
+                            (errcode(ERRCODE_UNDEFINED_OBJECT),
+                             errmsg("can not create ao format table on tablespace \"%s\" ",
+                                stmt->base.tablespacename),
+                             errOmitLocation(true)));
+                }
 
 		tablespaceId = get_tablespace_oid(stmt->base.tablespacename);
 		if (!OidIsValid(tablespaceId))
diff --git a/src/backend/parser/analyze.c b/src/backend/parser/analyze.c
index ce75706..15226b0 100644
--- a/src/backend/parser/analyze.c
+++ b/src/backend/parser/analyze.c
@@ -3138,7 +3138,20 @@ static void transformDistributedBy(ParseState *pstate, CreateStmtContext *cxt,
     *policyp = NULL;
     return;
   }
-
+  
+  /*
+   * Currently heap table only exists in hawq's master, so there is no
+   * policy information.
+   */
+  if (enable_heap_table_on_master) {
+    bool appendonly;
+    bool hasAppendOnly = GetRelOpt_appendonly_fromOptions(options, &appendonly);
+    if (hasAppendOnly && !appendonly) {
+      *policyp = NULL;
+      return;
+    }
+  }
+  
   policy = (GpPolicy *)palloc(sizeof(GpPolicy) +
                               maxattrs * sizeof(policy->attrs[0]));
   policy->ptype = POLICYTYPE_PARTITIONED;
diff --git a/src/backend/postmaster/identity.c b/src/backend/postmaster/identity.c
index 94d202f..2e5ed19 100644
--- a/src/backend/postmaster/identity.c
+++ b/src/backend/postmaster/identity.c
@@ -535,6 +535,29 @@ int GetRelOpt_bucket_num_fromOptions(List *options, int default_val)
 		}
 	return ((bucketnum>0)?bucketnum : (default_val));
 }
+
+/*
+ * If appendonly is set, return true, otherwise return false.
+ */
+bool GetRelOpt_appendonly_fromOptions(List *options, bool *appendonly)
+{
+  ListCell   *cell;
+  /* Scan list to see if "appendonly" was included */
+  if(options)
+    foreach(cell, options)
+    {
+      DefElem    *def = (DefElem *) lfirst(cell);
+
+      if (pg_strcasecmp(def->defname, "appendonly") == 0)
+      {
+        bool need_free_arg = false;
+        *appendonly = defGetString(def, &need_free_arg) == "true";
+        return true;
+      }
+    }
+  return false;
+}
+
 /** 
   *	Read Relation Option from catalog Relation
   */
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 89d20e0..1cd6e2b 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -417,20 +417,24 @@ QueryIsReadOnly(Query *parsetree)
 
 /*
  * CanCreateIndex: can support create index
- * So far, magma table and native orc could support index
+ * So far, magma table, native orc and heap table could support index
  */
 void CanSupportIndex(IndexStmt *stmt, Oid relid)
 {
 	/* 1. upgrade mode should support index operation */
 	if (gp_upgrade_mode) return;
-
+	
 	bool supportIndex = false;
 	Relation rel = heap_open(relid, AccessShareLock);
 	bool nativeOrc = RelationIsOrc(rel);
+	bool isHeap = RelationIsHeap(rel);
 	heap_close(rel, AccessShareLock);
+	
+	/* 2. heap table supports index */
+	if (isHeap) return;
 
 	/*
-	 * 2. deal magma table and native orc
+	 * 3. deal magma table and native orc
 	 * for "stmt->magma", deal with special partition situation, oushu issue #1049
 	 * its ugly, but there is no elegant way now
 	 */
@@ -446,7 +450,7 @@ void CanSupportIndex(IndexStmt *stmt, Oid relid)
 	if (supportIndex)
 	{
 		/*
-		 * 3. magma/native orc index cant support the accessory conditions
+		 * 4. magma/native orc index cant support the accessory conditions
 		 */
 		if (stmt->options) {
 			ereport(ERROR,
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 521d2b6..a9f62ac 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1301,9 +1301,10 @@ RelationBuildDesc(Oid targetRelId, bool insertIt)
 
     /*
      * initialize Greenplum Database partitioning info
+     * currently heap table only exists in hawq's master, so there is no rd_cdbpolicy.
      */
     if (relation->rd_rel->relkind == RELKIND_RELATION &&
-        !IsSystemRelation(relation))
+        !IsSystemRelation(relation) && !RelationIsHeap(relation))
         relation->rd_cdbpolicy = GpPolicyFetch(CacheMemoryContext, targetRelId);
 
     relation->rd_cdbDefaultStatsWarningIssued = false;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 20f8f1c..25f8714 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -773,6 +773,8 @@ int hawq_rm_nvseg_for_analyze_nopart_perquery_perseg_limit;
 int hawq_rm_nvseg_for_analyze_part_perquery_perseg_limit;
 int hawq_rm_nvseg_for_analyze_nopart_perquery_limit;
 int hawq_rm_nvseg_for_analyze_part_perquery_limit;
+bool enable_heap_table_on_master;
+bool enable_pg_default_for_non_system_table;
 double	  optimizer_cost_threshold;
 double  optimizer_nestloop_factor;
 double  locality_upper_bound;
@@ -4581,6 +4583,25 @@ static struct config_bool ConfigureNamesBool[] =
     &hawq_init_with_hdfs,
     true, NULL, NULL
   },
+  
+        {
+                {"enable_heap_table_on_master", PGC_USERSET, DEVELOPER_OPTIONS,
+                        gettext_noop("Enable heap tables on the master."),
+                        NULL,
+                        GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE
+                },
+                &enable_heap_table_on_master,
+                false, NULL, NULL
+        },
+        {
+	        {"enable_pg_default_for_non_system_table", PGC_USERSET, DEVELOPER_OPTIONS,
+                        gettext_noop("Enable pg_default for non-system tables."),
+                        NULL,
+                        GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE
+                },
+                &enable_pg_default_for_non_system_table,
+                false, NULL, NULL
+        },
 
 	/* End-of-list marker */
 	{
diff --git a/src/include/catalog/pg_class.h b/src/include/catalog/pg_class.h
index 59614df..da4ac2f 100644
--- a/src/include/catalog/pg_class.h
+++ b/src/include/catalog/pg_class.h
@@ -246,7 +246,7 @@ DESCR("");
 #define		  RELSTORAGE_EXTERNAL 'x'
 #define		  RELSTORAGE_FOREIGN 'f'
 
-static inline bool relstorage_is_buffer_pool(char c)
+static inline bool relstorage_is_heap(char c)
 {
 	return (c == RELSTORAGE_HEAP);
 }
diff --git a/src/include/postmaster/identity.h b/src/include/postmaster/identity.h
index 7aed5e1..d98e8f0 100644
--- a/src/include/postmaster/identity.h
+++ b/src/include/postmaster/identity.h
@@ -94,6 +94,7 @@ extern int GetUserDefinedFunctionVsegNum(void);
 extern int GetAnalyzeVSegNum(void);
 extern int	 GetCopyFromVSegNum(void);
 extern int GetRelOpt_bucket_num_fromOptions(List *options, int default_val);
+extern bool GetRelOpt_appendonly_fromOptions(List *options, bool *appendonly);
 extern int GetRelOpt_bucket_num_fromRel(Relation relation, int default_val);
 extern int GetRelOpt_bucket_num_fromRangeVar(const RangeVar* rel_rv, int default_val);
 extern int GetDefaultPartitionNum(void);
diff --git a/src/include/utils/guc.h b/src/include/utils/guc.h
index 369b704..a788deb 100644
--- a/src/include/utils/guc.h
+++ b/src/include/utils/guc.h
@@ -322,6 +322,15 @@ extern int hawq_rm_nvseg_for_analyze_nopart_perquery_perseg_limit;
 extern int hawq_rm_nvseg_for_analyze_part_perquery_perseg_limit;
 extern int hawq_rm_nvseg_for_analyze_nopart_perquery_limit;
 extern int hawq_rm_nvseg_for_analyze_part_perquery_limit;
+
+/////////////////////////////////////////////////////////////////////////
+/*
+ * These guc and related code are temporary, maybe will be delete afterwords.
+ */
+extern bool enable_heap_table_on_master;
+extern bool enable_pg_default_for_non_system_table;
+/////////////////////////////////////////////////////////////////////////
+
 extern bool allow_file_count_bucket_num_mismatch;
 extern bool enable_pg_stat_activity_history;