You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafodion.apache.org by li...@apache.org on 2016/11/08 09:25:09 UTC

[2/5] incubator-trafodion git commit: second round of function support

second round of function support


Project: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/commit/79d36dc3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/tree/79d36dc3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-trafodion/diff/79d36dc3

Branch: refs/heads/master
Commit: 79d36dc32ecefca9a624ceeeb9246d49f54534b1
Parents: 4c4dfc4
Author: Liu Ming <ov...@sina.com>
Authored: Wed Nov 2 21:34:02 2016 -0400
Committer: Liu Ming <ov...@sina.com>
Committed: Wed Nov 2 21:34:02 2016 -0400

----------------------------------------------------------------------
 core/sql/common/OperTypeEnum.h     |   1 +
 core/sql/exp/ExpPackDefs.cpp       |  15 ++++
 core/sql/exp/exp_clause.cpp        |  18 ++++
 core/sql/exp/exp_clause.h          |   7 +-
 core/sql/exp/exp_function.cpp      | 141 +++++++++++++++++++++++++++++++-
 core/sql/exp/exp_function.h        |  87 ++++++++++++++++++++
 core/sql/generator/GenItemFunc.cpp |  27 ++++++
 core/sql/optimizer/SynthType.cpp   |  47 +++++++++++
 core/sql/parser/sqlparser.y        |  42 ++++++++++
 9 files changed, 383 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/common/OperTypeEnum.h
----------------------------------------------------------------------
diff --git a/core/sql/common/OperTypeEnum.h b/core/sql/common/OperTypeEnum.h
index 6e3981a..d32be18 100644
--- a/core/sql/common/OperTypeEnum.h
+++ b/core/sql/common/OperTypeEnum.h
@@ -786,6 +786,7 @@ enum OperatorTypeEnum {
                         ITM_MD5       = 2635,
                         ITM_SHA1      = 2636,
                         ITM_SHA2      = 2637,
+                        ITM_SHA       = 2638,
 
                         // Items for needed for Translating to UCS2 output strings
                         ITM_DATEFMT     = 2990,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/exp/ExpPackDefs.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/ExpPackDefs.cpp b/core/sql/exp/ExpPackDefs.cpp
index 3bfea00..c8b7c65 100644
--- a/core/sql/exp/ExpPackDefs.cpp
+++ b/core/sql/exp/ExpPackDefs.cpp
@@ -584,6 +584,21 @@ Long ExHeaderClause::pack(void * space)
   return packClause(space, sizeof(ExHeaderClause));
 }  
 
+NA_EIDPROC Long ExFunctionSha::pack(void * space)
+{
+  return packClause(space, sizeof(ExFunctionSha));
+}
+
+NA_EIDPROC Long ExFunctionSha2::pack(void * space)
+{
+  return packClause(space, sizeof(ExFunctionSha2));
+}
+
+NA_EIDPROC Long ExFunctionMd5::pack(void * space)
+{
+  return packClause(space, sizeof(ExFunctionMd5));
+}
+
 NA_EIDPROC Long ExFunctionCrc32::pack(void * space)
 {
   return packClause(space, sizeof(ExFunctionCrc32));

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/exp/exp_clause.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/exp_clause.cpp b/core/sql/exp/exp_clause.cpp
index 79c42cc..8fe135d 100644
--- a/core/sql/exp/exp_clause.cpp
+++ b/core/sql/exp/exp_clause.cpp
@@ -338,6 +338,15 @@ ex_clause::ex_clause(clause_type type,
 	case ITM_CURR_TRANSID:
 	  setClassID(FUNC_CURR_TRANSID_ID);
 	  break;
+        case ITM_SHA:
+          setClassID(FUNC_SHA_ID);
+          break;
+        case ITM_SHA2:
+          setClassID(FUNC_SHA2_ID);
+          break;
+        case ITM_MD5:
+          setClassID(FUNC_MD5_ID);
+          break; 
         case ITM_CRC32:
           setClassID(FUNC_CRC32_ID);
           break; 
@@ -987,6 +996,15 @@ NA_EIDPROC char *ex_clause::findVTblPtr(short classID)
     case ex_clause::FUNC_HBASE_VERSION:
       GetVTblPtr(vtblPtr, ExFunctionHbaseVersion);
       break;
+    case ex_clause::FUNC_SHA_ID:
+      GetVTblPtr(vtblPtr, ExFunctionSha);
+      break;
+    case ex_clause::FUNC_SHA2_ID:
+      GetVTblPtr(vtblPtr, ExFunctionSha2);
+      break;
+    case ex_clause::FUNC_MD5_ID:
+      GetVTblPtr(vtblPtr, ExFunctionMd5);
+      break;
     case ex_clause::FUNC_CRC32_ID:
       GetVTblPtr(vtblPtr, ExFunctionCrc32);
       break;

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/exp/exp_clause.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/exp_clause.h b/core/sql/exp/exp_clause.h
index 0ce3b51..defad8b 100644
--- a/core/sql/exp/exp_clause.h
+++ b/core/sql/exp/exp_clause.h
@@ -203,7 +203,12 @@ public:
     FUNC_ISIP_ID             = 118,
     FUNC_INETATON_ID         = 119,
     FUNC_INETNTOA_ID         = 120,
-    FUNC_CRC32_ID            = 121
+    FUNC_CRC32_ID            = 121,
+    FUNC_MD5_ID              = 122,
+    FUNC_SHA1_ID             = 123,
+    FUNC_SHA2_ID             = 124,
+    FUNC_SHA_ID              = 125
+
   };
 
   // max number of operands (including result) in a clause.

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/exp/exp_function.cpp
----------------------------------------------------------------------
diff --git a/core/sql/exp/exp_function.cpp b/core/sql/exp/exp_function.cpp
index f819165..de36408 100644
--- a/core/sql/exp/exp_function.cpp
+++ b/core/sql/exp/exp_function.cpp
@@ -40,6 +40,10 @@
 
 
 #include <math.h>
+#include <zlib.h>
+#include "openssl/md5.h"
+#include <openssl/sha.h>  
+
 #define MathSqrt(op, err) sqrt(op)
 
 #include <ctype.h>
@@ -203,6 +207,9 @@ ExUnPackCol::ExUnPackCol(){};
 ExFunctionRangeLookup::ExFunctionRangeLookup(){};
 ExAuditImage::ExAuditImage(){};
 ExFunctionCrc32::ExFunctionCrc32(){};
+ExFunctionMd5::ExFunctionMd5(){};
+ExFunctionSha::ExFunctionSha(){};
+ExFunctionSha2::ExFunctionSha2(){};
 ExFunctionIsIP::ExFunctionIsIP(){};
 ExFunctionInetAton::ExFunctionInetAton(){};
 ExFunctionInetNtoa::ExFunctionInetNtoa(){};
@@ -227,6 +234,27 @@ ExFunctionCrc32::ExFunctionCrc32(OperatorTypeEnum oper_type,
   
 };
 
+ExFunctionMd5::ExFunctionMd5(OperatorTypeEnum oper_type,
+			       Attributes ** attr, Space * space)
+     : ex_function_clause(oper_type, 2, attr, space)
+{
+  
+};
+
+ExFunctionSha::ExFunctionSha(OperatorTypeEnum oper_type,
+			       Attributes ** attr, Space * space)
+     : ex_function_clause(oper_type, 2, attr, space)
+{
+  
+};
+
+ExFunctionSha2::ExFunctionSha2(OperatorTypeEnum oper_type,
+			       Attributes ** attr, Space * space)
+     : ex_function_clause(oper_type, 2, attr, space)
+{
+  
+};
+
 ExFunctionIsIP::ExFunctionIsIP(OperatorTypeEnum oper_type,
 			       Attributes ** attr, Space * space)
      : ex_function_clause(oper_type, 2, attr, space)
@@ -7912,11 +7940,122 @@ ex_expr::exp_return_type ExFunctionCrc32::eval(char * op_data[],
                                                         CollHeap *heap,
                                                         ComDiagsArea **diags)
 {
+  Attributes *resultAttr   = getOperand(0);
+  Attributes *srcAttr   = getOperand(1);
+
+  Lng32 slen = srcAttr->getLength(op_data[-MAX_OPERANDS+1]);
+  Lng32 rlen = resultAttr->getLength();
+
     *(ULng32*)op_data[0] = 0; 
-//stub for now
+    ULng32 crc = crc32(0L, Z_NULL, 0);
+    crc = crc32 (crc, (const Bytef*)op_data[1], slen);
+    *(ULng32*)op_data[0] = crc; 
     return ex_expr::EXPR_OK;
 }
 
+ex_expr::exp_return_type ExFunctionSha2::eval(char * op_data[],
+                                                        CollHeap *heap,
+                                                        ComDiagsArea **diags)
+{
+
+  unsigned char sha[SHA_DIGEST_LENGTH + 1]={0};  
+
+  Attributes *resultAttr   = getOperand(0);
+  Attributes *srcAttr   = getOperand(1);
+
+  Lng32 slen = srcAttr->getLength(op_data[-MAX_OPERANDS+1]);
+  Lng32 rlen = resultAttr->getLength();
+
+  if(rlen < 40)
+  {
+    //ExRaiseSqlError(heap, diagsArea, EXE_GETBIT_ERROR);
+    return ex_expr::EXPR_ERROR;
+  }
+
+  SHA_CTX  sha_ctx;
+
+  SHA_Init(&sha_ctx);  
+  SHA_Update(&sha_ctx, op_data[1], slen);
+  SHA_Final((unsigned char*) sha,&sha_ctx); 
+  char tmp[2];
+  for(int i=0; i < SHA_DIGEST_LENGTH ; i++ )
+  {
+    tmp[0]=tmp[1]='0';
+    sprintf(tmp, "%.2x", (int)sha[i]);
+    memcpy(op_data[0]+i*2, tmp, 2);
+  }
+   
+  return ex_expr::EXPR_OK;
+}
+
+ex_expr::exp_return_type ExFunctionSha::eval(char * op_data[],
+                                                        CollHeap *heap,
+                                                        ComDiagsArea **diags)
+{
+
+  unsigned char sha[SHA_DIGEST_LENGTH + 1]={0};  
+
+  Attributes *resultAttr   = getOperand(0);
+  Attributes *srcAttr   = getOperand(1);
+
+  Lng32 slen = srcAttr->getLength(op_data[-MAX_OPERANDS+1]);
+  Lng32 rlen = resultAttr->getLength();
+
+  if(rlen < 40)
+  {
+    //ExRaiseSqlError(heap, diagsArea, EXE_GETBIT_ERROR);
+    return ex_expr::EXPR_ERROR;
+  }
+
+  SHA_CTX  sha_ctx;
+
+  SHA_Init(&sha_ctx);  
+  SHA_Update(&sha_ctx, op_data[1], slen);
+  SHA_Final((unsigned char*) sha,&sha_ctx); 
+  char tmp[2];
+  for(int i=0; i < SHA_DIGEST_LENGTH ; i++ )
+  {
+    tmp[0]=tmp[1]='0';
+    sprintf(tmp, "%.2x", (int)sha[i]);
+    memcpy(op_data[0]+i*2, tmp, 2);
+  }
+   
+  return ex_expr::EXPR_OK;
+}
+ex_expr::exp_return_type ExFunctionMd5::eval(char * op_data[],
+                                                        CollHeap *heap,
+                                                        ComDiagsArea **diags)
+{
+  unsigned char md5[17]={0};  
+
+  Attributes *resultAttr   = getOperand(0);
+  Attributes *srcAttr   = getOperand(1);
+
+  Lng32 slen = srcAttr->getLength(op_data[-MAX_OPERANDS+1]);
+  Lng32 rlen = resultAttr->getLength();
+
+  if(rlen < 32)
+  {
+    //ExRaiseSqlError(heap, diagsArea, EXE_GETBIT_ERROR);
+    return ex_expr::EXPR_ERROR;
+  }
+
+  MD5_CTX  md5_ctx;
+
+  MD5_Init(&md5_ctx);  
+  MD5_Update(&md5_ctx, op_data[1], slen);
+  MD5_Final((unsigned char*) md5,&md5_ctx); 
+  char tmp[2];
+  for(int i=0; i < 16; i++ )
+  {
+    tmp[0]=tmp[1]='0';
+    sprintf(tmp, "%.2x", (int)md5[i]);
+    memcpy(op_data[0]+i*2, tmp, 2);
+  }
+   
+  return ex_expr::EXPR_OK;
+}
+
 ex_expr::exp_return_type ExFunctionIsIP::eval(char * op_data[],
                                                         CollHeap *heap,
                                                         ComDiagsArea **diags)

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/exp/exp_function.h
----------------------------------------------------------------------
diff --git a/core/sql/exp/exp_function.h b/core/sql/exp/exp_function.h
index 22ef8c9..cfa5907 100644
--- a/core/sql/exp/exp_function.h
+++ b/core/sql/exp/exp_function.h
@@ -885,6 +885,93 @@ public:
   // ---------------------------------------------------------------------
 };
 
+class SQLEXP_LIB_FUNC ExFunctionSha: public ex_function_clause {
+public:
+  NA_EIDPROC ExFunctionSha(OperatorTypeEnum oper_type,
+				Attributes ** attr,
+				Space * space);
+  NA_EIDPROC ExFunctionSha();
+
+  NA_EIDPROC ex_expr::exp_return_type eval(char *op_data[], CollHeap*, 
+					   ComDiagsArea** = 0);  
+  NA_EIDPROC Long pack(void *);
+
+  // ---------------------------------------------------------------------
+  // Redefinition of methods inherited from NAVersionedObject.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC virtual unsigned char getClassVersionID()
+  {
+    return 1;
+  }
+
+  NA_EIDPROC virtual void populateImageVersionIDArray()
+  {
+    setImageVersionID(2,getClassVersionID());
+    ex_function_clause::populateImageVersionIDArray();
+  }
+
+  NA_EIDPROC virtual short getClassSize() { return (short)sizeof(*this); }
+  // ---------------------------------------------------------------------
+};
+
+class SQLEXP_LIB_FUNC ExFunctionSha2: public ex_function_clause {
+public:
+  NA_EIDPROC ExFunctionSha2(OperatorTypeEnum oper_type,
+				Attributes ** attr,
+				Space * space);
+  NA_EIDPROC ExFunctionSha2();
+
+  NA_EIDPROC ex_expr::exp_return_type eval(char *op_data[], CollHeap*, 
+					   ComDiagsArea** = 0);  
+  NA_EIDPROC Long pack(void *);
+
+  // ---------------------------------------------------------------------
+  // Redefinition of methods inherited from NAVersionedObject.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC virtual unsigned char getClassVersionID()
+  {
+    return 1;
+  }
+
+  NA_EIDPROC virtual void populateImageVersionIDArray()
+  {
+    setImageVersionID(2,getClassVersionID());
+    ex_function_clause::populateImageVersionIDArray();
+  }
+
+  NA_EIDPROC virtual short getClassSize() { return (short)sizeof(*this); }
+  // ---------------------------------------------------------------------
+};
+
+class SQLEXP_LIB_FUNC ExFunctionMd5: public ex_function_clause {
+public:
+  NA_EIDPROC ExFunctionMd5(OperatorTypeEnum oper_type,
+				Attributes ** attr,
+				Space * space);
+  NA_EIDPROC ExFunctionMd5();
+
+  NA_EIDPROC ex_expr::exp_return_type eval(char *op_data[], CollHeap*, 
+					   ComDiagsArea** = 0);  
+  NA_EIDPROC Long pack(void *);
+
+  // ---------------------------------------------------------------------
+  // Redefinition of methods inherited from NAVersionedObject.
+  // ---------------------------------------------------------------------
+  NA_EIDPROC virtual unsigned char getClassVersionID()
+  {
+    return 1;
+  }
+
+  NA_EIDPROC virtual void populateImageVersionIDArray()
+  {
+    setImageVersionID(2,getClassVersionID());
+    ex_function_clause::populateImageVersionIDArray();
+  }
+
+  NA_EIDPROC virtual short getClassSize() { return (short)sizeof(*this); }
+  // ---------------------------------------------------------------------
+};
+
 class SQLEXP_LIB_FUNC ExFunctionIsIP : public ex_function_clause {
 public:
   NA_EIDPROC ExFunctionIsIP(OperatorTypeEnum oper_type,

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/generator/GenItemFunc.cpp
----------------------------------------------------------------------
diff --git a/core/sql/generator/GenItemFunc.cpp b/core/sql/generator/GenItemFunc.cpp
index d025c22..f1a6ec3 100644
--- a/core/sql/generator/GenItemFunc.cpp
+++ b/core/sql/generator/GenItemFunc.cpp
@@ -218,6 +218,33 @@ short BuiltinFunction::codeGen(Generator * generator)
       
       break;
 
+   case ITM_SHA:
+      {
+         function_clause =
+           new(generator->getSpace()) ExFunctionSha(getOperatorType(),
+                                                         attr, space);
+      }
+     
+      break;
+
+   case ITM_SHA2:
+      {
+         function_clause =
+           new(generator->getSpace()) ExFunctionSha2(getOperatorType(),
+                                                         attr, space);
+      }
+     
+      break;
+
+   case ITM_MD5:
+      {
+         function_clause =
+           new(generator->getSpace()) ExFunctionMd5(getOperatorType(),
+                                                         attr, space);
+      }
+     
+      break;
+
    case ITM_CRC32:
       {
          function_clause =

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/optimizer/SynthType.cpp
----------------------------------------------------------------------
diff --git a/core/sql/optimizer/SynthType.cpp b/core/sql/optimizer/SynthType.cpp
index 1566219..de4abd4 100644
--- a/core/sql/optimizer/SynthType.cpp
+++ b/core/sql/optimizer/SynthType.cpp
@@ -1010,6 +1010,53 @@ const NAType *BuiltinFunction::synthesizeType()
 	    SQLChar(maxLength, typ1.supportsSQLnull());
       }
     break;
+    case ITM_SHA:
+    case ITM_SHA2:
+      {
+        // type cast any params
+        ValueId vid1 = child(0)->getValueId();
+        SQLChar c1(ComSqlId::MAX_QUERY_ID_LEN);
+        vid1.coerceType(c1, NA_CHARACTER_TYPE);
+        //input type must be string
+        const NAType &typ1 = child(0)->getValueId().getType();
+
+        if (typ1.getTypeQualifier() != NA_CHARACTER_TYPE)
+          {
+	    *CmpCommon::diags() << DgSqlCode(-4045) << DgString0("IS_IP");
+	    return NULL;
+          }
+
+        retType = new HEAP
+           SQLVarChar(40, FALSE);
+	if (typ1.supportsSQLnull())
+	  {
+	    retType->setNullable(TRUE);
+	  }
+      }
+    break;
+    case ITM_MD5:
+      {
+        // type cast any params
+        ValueId vid1 = child(0)->getValueId();
+        SQLChar c1(ComSqlId::MAX_QUERY_ID_LEN);
+        vid1.coerceType(c1, NA_CHARACTER_TYPE);
+        //input type must be string
+        const NAType &typ1 = child(0)->getValueId().getType();
+
+        if (typ1.getTypeQualifier() != NA_CHARACTER_TYPE)
+          {
+	    *CmpCommon::diags() << DgSqlCode(-4045) << DgString0("IS_IP");
+	    return NULL;
+          }
+
+        retType = new HEAP
+           SQLVarChar(33, FALSE);
+	if (typ1.supportsSQLnull())
+	  {
+	    retType->setNullable(TRUE);
+	  }
+      }
+    break;
     case ITM_CRC32:
       {
         const NAType &typ1 = child(0)->getValueId().getType();

http://git-wip-us.apache.org/repos/asf/incubator-trafodion/blob/79d36dc3/core/sql/parser/sqlparser.y
----------------------------------------------------------------------
diff --git a/core/sql/parser/sqlparser.y b/core/sql/parser/sqlparser.y
index 2fd5209..10261d4 100755
--- a/core/sql/parser/sqlparser.y
+++ b/core/sql/parser/sqlparser.y
@@ -1036,6 +1036,7 @@ static void enableMakeQuotedStringISO88591Mechanism()
 %token <tokval> TOK_SET
 %token <tokval> TOK_SETS
 %token <tokval> TOK_SG_TABLE
+%token <tokval> TOK_SHA
 %token <tokval> TOK_SHA1
 %token <tokval> TOK_SHA2
 %token <tokval> TOK_SHAPE
@@ -9636,6 +9637,31 @@ misc_function :
 				{
 				  $$ = $2;
 				}
+
+     | TOK_SHA  '(' value_expression ')'
+                {
+                    $$ = new (PARSERHEAP())
+                    BuiltinFunction(ITM_SHA,
+                            CmpCommon::statementHeap(),
+                            1, $3);
+                }
+
+     | TOK_SHA2 '(' value_expression ')'
+                {
+                    $$ = new (PARSERHEAP())
+                    BuiltinFunction(ITM_SHA2,
+                            CmpCommon::statementHeap(),
+                            1, $3);
+                }
+
+     | TOK_MD5 '(' value_expression ')'
+                {
+                    $$ = new (PARSERHEAP())
+                    BuiltinFunction(ITM_MD5,
+                            CmpCommon::statementHeap(),
+                            1, $3);
+                }
+
      | TOK_CRC32 '(' value_expression ')'
                 {
                     $$ = new (PARSERHEAP())
@@ -9645,6 +9671,22 @@ misc_function :
                 }
 
 
+     | TOK_SHA1 '(' value_expression ')'
+                {
+                    $$ = new (PARSERHEAP())
+                    BuiltinFunction(ITM_SHA1,
+                            CmpCommon::statementHeap(),
+                            1, $3);
+                }
+
+     | TOK_SHA2 '(' value_expression ')'
+                {
+                    $$ = new (PARSERHEAP())
+                    BuiltinFunction(ITM_SHA2,
+                            CmpCommon::statementHeap(),
+                            1, $3);
+                }
+
      | TOK_GREATEST '(' value_expression ',' value_expression ')'
                   {
                     $$ = new (PARSERHEAP())