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/11/18 01:51:43 UTC

[hawq] 02/02: HAWQ-1815. native orc supports udt

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

commit c82cf9f2a2f61859918f34401c6cad26cd912490
Author: ztao1987 <zh...@gmail.com>
AuthorDate: Thu Nov 18 09:45:04 2021 +0800

    HAWQ-1815. native orc supports udt
---
 contrib/orc/orc.c                     |  66 +++++++-------
 src/backend/access/orc/orcam.c        | 158 ++++++++++++++++++++++------------
 src/backend/utils/hawq_type_mapping.c |  61 ++++++-------
 src/include/utils/hawq_type_mapping.h |   2 +
 4 files changed, 175 insertions(+), 112 deletions(-)

diff --git a/contrib/orc/orc.c b/contrib/orc/orc.c
index 465e4f3..b58943d 100644
--- a/contrib/orc/orc.c
+++ b/contrib/orc/orc.c
@@ -337,36 +337,44 @@ Datum orc_validate_encodings(PG_FUNCTION_ARGS)
  * void
  * orc_validate_datatypes(TupleDesc tupDesc)
  */
-Datum orc_validate_datatypes(PG_FUNCTION_ARGS)
-{
-	PlugStorageValidator psv = (PlugStorageValidator) (fcinfo->context);
-	TupleDesc tup_desc = psv->tuple_desc;
-
-	for (int i = 0; i < tup_desc->natts; ++i)
-	{
-		int32_t datatype =
-				(int32_t) (((Form_pg_attribute) (tup_desc->attrs[i]))->atttypid);
-		int4 	typmod = ((Form_pg_attribute) (tup_desc->attrs[i]))->atttypmod;
-
-		if (checkORCUnsupportedDataType(datatype))
-		{
-			ereport(ERROR,
-					(errcode(ERRCODE_SYNTAX_ERROR), errmsg("unsupported data types %s for columns of external ORC table is specified.", TypeNameToString(makeTypeNameFromOid(datatype, -1))), errOmitLocation(true)));
-		}
-		if (HAWQ_TYPE_NUMERIC == datatype)
-		{
-			int4 tmp_typmod = typmod - VARHDRSZ;
-			int precision = (tmp_typmod >> 16) & 0xffff;
-			int scale = tmp_typmod & 0xffff;
-			if (precision < 1 || 38 < precision)
-				ereport(ERROR,
-						(errcode(ERRCODE_INVALID_PARAMETER_VALUE), errmsg("ORC DECIMAL precision must be between 1 and 38")));
-			if (scale == 0)
-				ereport(NOTICE, (errmsg("Using a scale of zero for ORC DECIMAL")));
-		}
-	}
+Datum orc_validate_datatypes(PG_FUNCTION_ARGS) {
+  PlugStorageValidator psv = (PlugStorageValidator)(fcinfo->context);
+  TupleDesc tup_desc = psv->tuple_desc;
+
+  for (int i = 0; i < tup_desc->natts; ++i) {
+    int32_t datatype =
+        (int32_t)(((Form_pg_attribute)(tup_desc->attrs[i]))->atttypid);
+    int4 typmod = ((Form_pg_attribute)(tup_desc->attrs[i]))->atttypmod;
+
+    if (checkORCUnsupportedDataType(datatype)) {
+      ereport(ERROR,
+              (errcode(ERRCODE_SYNTAX_ERROR),
+               errmsg("unsupported data types %s for columns of external ORC "
+                      "table is specified.",
+                      TypeNameToString(makeTypeNameFromOid(datatype, -1))),
+               errOmitLocation(true)));
+    }
+    if (HAWQ_TYPE_NUMERIC == datatype) {
+      int4 tmp_typmod = typmod - VARHDRSZ;
+      int precision = (tmp_typmod >> 16) & 0xffff;
+      int scale = tmp_typmod & 0xffff;
+      if (precision < 1 || 38 < precision)
+        ereport(ERROR,
+                (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+                 errmsg("ORC DECIMAL precision must be between 1 and 38")));
+      if (scale == 0)
+        ereport(NOTICE, (errmsg("Using a scale of zero for ORC DECIMAL")));
+    }
+    if (HAEQ_TYPE_UDT(datatype))
+      ereport(ERROR,
+              (errcode(ERRCODE_SYNTAX_ERROR),
+               errmsg("unsupported data types %s for columns of external ORC "
+                      "table is specified.",
+                      TypeNameToString(makeTypeNameFromOid(datatype, -1))),
+               errOmitLocation(true)));
+  }
 
-	PG_RETURN_VOID() ;
+  PG_RETURN_VOID();
 }
 
 /*
diff --git a/src/backend/access/orc/orcam.c b/src/backend/access/orc/orcam.c
index df7242e..2439704 100644
--- a/src/backend/access/orc/orcam.c
+++ b/src/backend/access/orc/orcam.c
@@ -74,6 +74,7 @@ typedef struct OrcFormatData {
   char **colRawValues;
   uint64 *colValLength;
   TimestampType *colTimestamp;
+  struct varlena **colFixedLenUDT;
 } OrcFormatData;
 
 static void initOrcFormatUserData(TupleDesc tup_desc,
@@ -86,8 +87,16 @@ static void initOrcFormatUserData(TupleDesc tup_desc,
   orcFormatData->colRawValues = palloc0(sizeof(char *) * natts);
   orcFormatData->colValLength = palloc0(sizeof(uint64) * natts);
   orcFormatData->colTimestamp = palloc0(sizeof(TimestampType) * natts);
+  orcFormatData->colFixedLenUDT = palloc0(sizeof(struct varlena *) * natts);
 
   for (int i = 0; i < orcFormatData->numberOfColumns; ++i) {
+    // allocate memory for colFixedLenUDT[i] of fixed-length type in advance
+    bool isFixedLengthType = tup_desc->attrs[i]->attlen > 0 ? true : false;
+    if (isFixedLengthType) {
+      orcFormatData->colFixedLenUDT[i] = (struct valena *)palloc0(
+          tup_desc->attrs[i]->attlen + sizeof(uint32_t));
+    }
+
     orcFormatData->colNames[i] = palloc0(NAMEDATALEN);
     strcpy(orcFormatData->colNames[i], tup_desc->attrs[i]->attname.data);
 
@@ -105,8 +114,12 @@ static void initOrcFormatUserData(TupleDesc tup_desc,
 }
 
 static freeOrcFormatUserData(OrcFormatData *orcFormatData) {
-  for (int i = 0; i < orcFormatData->numberOfColumns; ++i)
+  for (int i = 0; i < orcFormatData->numberOfColumns; ++i) {
     pfree(orcFormatData->colNames[i]);
+    if (orcFormatData->colFixedLenUDT[i])
+      pfree(orcFormatData->colFixedLenUDT[i]);
+  }
+
   pfree(orcFormatData->colTimestamp);
   pfree(orcFormatData->colValLength);
   pfree(orcFormatData->colRawValues);
@@ -235,17 +248,37 @@ static void convertAndFillIntoOrcFormatData(OrcFormatData *orcFormatData,
       int *date = (int *)(&(values[i]));
       *date += POSTGRES_EPOCH_JDATE - UNIX_EPOCH_JDATE;
       orcFormatData->colRawValues[i] = (char *)(&(values[i]));
-    } else if (dataType == HAWQ_TYPE_TEXT || dataType == HAWQ_TYPE_BPCHAR ||
-               dataType == HAWQ_TYPE_VARCHAR) {
-      struct varlena *data = PG_DETOAST_DATUM(values[i]);
-      orcFormatData->colRawValues[i] = (char *)data;
-    } else if (dataType == HAWQ_TYPE_BYTE) {
-      orcFormatData->colRawValues[i] = (char *)PG_DETOAST_DATUM(values[i]);
     } else if (dataType == HAWQ_TYPE_NUMERIC) {
       Numeric num = DatumGetNumeric(values[i]);
       orcFormatData->colRawValues[i] = (char *)num;
-      if (NUMERIC_IS_NAN(num))
-        nulls[i] = true;
+      if (NUMERIC_IS_NAN(num)) nulls[i] = true;
+    } else {
+      // Check whether values[i] is fixed length udt.
+      bool isFixedLengthType = tupleDesc->attrs[i]->attlen > 0 ? true : false;
+      bool isPassByVal = tupleDesc->attrs[i]->attbyval;
+      if (isFixedLengthType) {
+        uint32_t dataLen = tupleDesc->attrs[i]->attlen;
+        uint32_t totalLen = dataLen + sizeof(uint32_t);
+
+        uint32_t tmpLen = __builtin_bswap32(totalLen);
+        char *lenArr = (char *)(&tmpLen);
+        memcpy(orcFormatData->colFixedLenUDT[i]->vl_len_, lenArr,
+               sizeof(uint32_t));
+
+        if (isPassByVal) {  // pass by val
+          char *data = (char *)(&values[i]);
+          memcpy(orcFormatData->colFixedLenUDT[i]->vl_dat, data, dataLen);
+          orcFormatData->colRawValues[i] =
+              (char *)(orcFormatData->colFixedLenUDT[i]);
+        } else {  // pass by pointer
+          char *data = (char *)(values[i]);
+          memcpy(orcFormatData->colFixedLenUDT[i]->vl_dat, data, dataLen);
+          orcFormatData->colRawValues[i] =
+              (char *)(orcFormatData->colFixedLenUDT[i]);
+        }
+      } else {
+        orcFormatData->colRawValues[i] = (char *)PG_DETOAST_DATUM(values[i]);
+      }
     }
   }
 }
@@ -402,51 +435,68 @@ void orcReadNext(OrcScanDescData *scanData, TupleTableSlot *slot) {
         continue;
 
       switch (tupleDesc->attrs[i]->atttypid) {
-      case HAWQ_TYPE_BOOL: {
-        values[i] = BoolGetDatum(*(bool *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_INT2: {
-        values[i] = Int16GetDatum(*(int16_t *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_INT4: {
-        values[i] = Int32GetDatum(*(int32_t *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_INT8:
-      case HAWQ_TYPE_TIME:
-      case HAWQ_TYPE_TIMESTAMP:
-      case HAWQ_TYPE_TIMESTAMPTZ: {
-        values[i] = Int64GetDatum(*(int64_t *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_FLOAT4: {
-        values[i] = Float4GetDatum(*(float *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_FLOAT8: {
-        values[i] = Float8GetDatum(*(double *)(orcFormatData->colRawValues[i]));
-        break;
-      }
-      case HAWQ_TYPE_VARCHAR:
-      case HAWQ_TYPE_TEXT:
-      case HAWQ_TYPE_BPCHAR:
-      case HAWQ_TYPE_BYTE:
-      case HAWQ_TYPE_NUMERIC: {
-        SET_VARSIZE((struct varlena *)(orcFormatData->colRawValues[i]),
-                    orcFormatData->colValLength[i]);
-        values[i] = PointerGetDatum(orcFormatData->colRawValues[i]);
-        break;
-      }
-      case HAWQ_TYPE_DATE: {
-        values[i] = Int32GetDatum(*(int32_t *)(orcFormatData->colRawValues[i]) -
-                                  POSTGRES_EPOCH_JDATE + UNIX_EPOCH_JDATE);
-        break;
-      }
-      default: {
-        break;
-      }
+        case HAWQ_TYPE_BOOL: {
+          values[i] = BoolGetDatum(*(bool *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_INT2: {
+          values[i] =
+              Int16GetDatum(*(int16_t *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_INT4: {
+          values[i] =
+              Int32GetDatum(*(int32_t *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_INT8:
+        case HAWQ_TYPE_TIME:
+        case HAWQ_TYPE_TIMESTAMP:
+        case HAWQ_TYPE_TIMESTAMPTZ: {
+          values[i] =
+              Int64GetDatum(*(int64_t *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_FLOAT4: {
+          values[i] =
+              Float4GetDatum(*(float *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_FLOAT8: {
+          values[i] =
+              Float8GetDatum(*(double *)(orcFormatData->colRawValues[i]));
+          break;
+        }
+        case HAWQ_TYPE_DATE: {
+          values[i] =
+              Int32GetDatum(*(int32_t *)(orcFormatData->colRawValues[i]) -
+                            POSTGRES_EPOCH_JDATE + UNIX_EPOCH_JDATE);
+          break;
+        }
+        default: {
+          // Check whether value[i] is fixed length udt.
+          bool isFixedLengthType =
+              tupleDesc->attrs[i]->attlen > 0 ? true : false;
+          bool isPassByVal = tupleDesc->attrs[i]->attbyval;
+          if (isFixedLengthType) {
+            if (isPassByVal) {  // pass by val
+              struct varlena *var =
+                  (struct varlena *)(orcFormatData->colRawValues[i]);
+              uint32 valLen = *(uint32 *)(var->vl_len_);
+              memcpy((void *)&values[i], var->vl_dat, valLen);
+            } else {  // pass by pointer
+              SET_VARSIZE((struct varlena *)(orcFormatData->colRawValues[i]),
+                          orcFormatData->colValLength[i]);
+              values[i] = PointerGetDatum(orcFormatData->colRawValues[i] +
+                                          sizeof(uint32_t));
+            }
+          } else {
+            SET_VARSIZE((struct varlena *)(orcFormatData->colRawValues[i]),
+                        orcFormatData->colValLength[i]);
+            values[i] = PointerGetDatum(orcFormatData->colRawValues[i]);
+          }
+          break;
+        }
       }
     }
     TupSetVirtualTupleNValid(slot, slot->tts_tupleDescriptor->natts);
diff --git a/src/backend/utils/hawq_type_mapping.c b/src/backend/utils/hawq_type_mapping.c
index be9eb4d..844bf5f 100644
--- a/src/backend/utils/hawq_type_mapping.c
+++ b/src/backend/utils/hawq_type_mapping.c
@@ -1,28 +1,29 @@
 /*-------------------------------------------------------------------------
-*
-* hawq_type_mapping.c
-*     Definitions for hawq type mapping function
-*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*   http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*
-*-------------------------------------------------------------------------
-*/
+ *
+ * hawq_type_mapping.c
+ *     Definitions for hawq type mapping function
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ *-------------------------------------------------------------------------
+ */
 
+#include "catalog/pg_magic_oid.h"
 #include "utils/hawq_type_mapping.h"
 #include "miscadmin.h"
 
@@ -129,10 +130,13 @@ int32_t map_hawq_type_to_common_plan(int32_t hawqTypeID) {
     case HAWQ_TYPE_POLYGON:
     case HAWQ_TYPE_CIRCLE:
     default:
-      return type_is_rowtype(hawqTypeID)
-                 ? (STRUCTEXID)
-                 : (type_is_basetype(hawqTypeID) ? IOBASETYPEID
-                                                 : INVALIDTYPEID);
+      if (HAEQ_TYPE_UDT(hawqTypeID))
+        return BINARYID;
+      else
+        return type_is_rowtype(hawqTypeID)
+                   ? (STRUCTEXID)
+                   : (type_is_basetype(hawqTypeID) ? IOBASETYPEID
+                                                   : INVALIDTYPEID);
   }
 }
 
@@ -189,7 +193,6 @@ bool checkORCUnsupportedDataType(int32_t hawqTypeID) {
     case HAWQ_TYPE_INT2:
     case HAWQ_TYPE_INT4:
     case HAWQ_TYPE_INT8:
-    case HAWQ_TYPE_TID:
     case HAWQ_TYPE_FLOAT4:
     case HAWQ_TYPE_FLOAT8:
     case HAWQ_TYPE_TEXT:
@@ -211,6 +214,6 @@ bool checkORCUnsupportedDataType(int32_t hawqTypeID) {
     case HAWQ_TYPE_UNKNOWN:
       return false;
     default:
-      return true;
+      return !HAEQ_TYPE_UDT(hawqTypeID);
   }
 }
diff --git a/src/include/utils/hawq_type_mapping.h b/src/include/utils/hawq_type_mapping.h
index 79320ea..d3e0f20 100644
--- a/src/include/utils/hawq_type_mapping.h
+++ b/src/include/utils/hawq_type_mapping.h
@@ -81,6 +81,8 @@
 
 #define HAWQ_TYPE_UNKNOWN 705
 
+#define HAEQ_TYPE_UDT(x) ( x > FirstNormalObjectId)
+
 extern int32_t map_hawq_type_to_common_plan(int32_t hawqTypeID);
 
 // if hawq type unsupported, return true