You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hawq.apache.org by weinan003 <gi...@git.apache.org> on 2018/03/16 06:41:27 UTC

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

GitHub user weinan003 opened a pull request:

    https://github.com/apache/incubator-hawq/pull/1347

    HAWQ-1591 Common tuple batch structure for VecExe

    Define *TupleBatchData* struct to hold a batch of tuples through vectorized execution process.
     

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/weinan003/incubator-hawq 1591

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-hawq/pull/1347.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1347
    
----
commit eac251c617bea3b48cb35f5604fbd7c9c4858fa2
Author: Weinan Wang <we...@...>
Date:   2018-03-16T06:31:45Z

    HAWQ-1591 Common tuple batch structure for VecExe

----


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by weinan003 <gi...@git.apache.org>.
Github user weinan003 closed the pull request at:

    https://github.com/apache/incubator-hawq/pull/1347


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175019315
  
    --- Diff: contrib/vexecutor/tuplebatch.h ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.
    + */
    +#ifndef __TUPLEBATCH_H__
    +#define __TUPLEBATCH_H__
    +
    +#include "vexecutor.h"
    +#include "vcheck.h"
    +
    +
    +typedef struct TupleBatchData
    +{
    +    int     _tb_len;
    +    int     batchsize;  //indicate maximum number of batch
    +    int     ncols;      //the number of target table column
    +    int     nrows;      //the number of tuples scaned out
    +    int     iter;       //used for data
    +    bool*   skip;       //used for qualification
    +    vheader** datagroup;
    +}TupleBatchData,*TupleBatch;
    +
    +/* TupleBatch ctor */
    --- End diff --
    
    1. Would you give a user case of tuplebatch struct or the function call logic of its functions ?
    2. How the tuple batch is integrated with current TupleTableSlot ?


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by interma <gi...@git.apache.org>.
Github user interma commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175022310
  
    --- Diff: contrib/vexecutor/tuplebatch.h ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.
    + */
    +#ifndef __TUPLEBATCH_H__
    +#define __TUPLEBATCH_H__
    +
    +#include "vexecutor.h"
    +#include "vcheck.h"
    +
    +
    +typedef struct TupleBatchData
    +{
    +    int     _tb_len;
    --- End diff --
    
    _tb_len no use?


---

[GitHub] incubator-hawq issue #1347: HAWQ-1591 Common tuple batch structure for VecEx...

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on the issue:

    https://github.com/apache/incubator-hawq/pull/1347
  
    LGTM


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175084597
  
    --- Diff: contrib/vexecutor/tuplebatch.c ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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 "postgres.h"
    +#include "tuplebatch.h"
    +
    +static size_t vtypeSize(vheader *vh);
    +static size_t tbSerializationSize(TupleBatch tb);
    +
    +#define TBGETVALUEPTR(tb,type,colid,rowid) (&(((type *)tb->datagroup[colid])->data[rowid]))
    +#define TBGETISNULLPTR(tb,colid,rowid) (&((tb->datagroup[colid])->isnull[rowid]))
    +
    +TupleBatch tbGenerate(int colnum,int batchsize)
    +{
    +    Assert(colnum > 0 && batchsize > 0);
    +    TupleBatch tb = palloc0(sizeof(TupleBatchData));
    +    if(!tb)
    +    {
    +        elog(FATAL,"TupleBatch Allocation failed");
    +        return NULL;
    +    }
    +
    +    tb->ncols = colnum;
    +    tb->batchsize = batchsize;
    +
    +    tb->skip = palloc0(sizeof(bool) * tb->batchsize);
    +    tb->datagroup = palloc0(sizeof(struct vtypeheader*) * tb->ncols);
    +
    --- End diff --
    
    It's better to init all the struct variable members (nrow, iter) in tbGenerate or call tbReset before return.


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175018979
  
    --- Diff: contrib/vexecutor/tuplebatch.c ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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 "postgres.h"
    +#include "tuplebatch.h"
    +
    +static size_t vtypeSize(vheader *vh);
    +static size_t tbSerializationSize(TupleBatch tb);
    +
    +#define TBGETVALUEPTR(tb,type,colid,rowid) (&(((type *)tb->datagroup[colid])->data[rowid]))
    +#define TBGETISNULLPTR(tb,colid,rowid) (&((tb->datagroup[colid])->isnull[rowid]))
    +
    +TupleBatch tbGenerate(int colnum,int batchsize)
    +{
    +    Assert(colnum > 0 && batchsize > 0);
    +    TupleBatch tb = palloc0(sizeof(TupleBatchData));
    +    if(!tb)
    +    {
    +        elog(FATAL,"TupleBatch Allocation failed");
    +        return NULL;
    +    }
    +
    +    tb->ncols = colnum;
    +    tb->batchsize = batchsize;
    +
    +    tb->skip = palloc0(sizeof(bool) * tb->batchsize);
    +    tb->datagroup = palloc0(sizeof(struct vtypeheader*) * tb->ncols);
    +
    +    return tb;
    +}
    +
    +void tbDestory(TupleBatch* tb){
    +    free((*tb)->skip);
    +    for(int i = 0 ;i < (*tb)->ncols; ++i)
    +    {
    +        if((*tb)->datagroup[i])
    +            tbfreeColumn((*tb)->datagroup,i);
    +    }
    +
    +    free((*tb)->datagroup);
    +
    +    free((*tb));
    +    *tb = NULL;
    +}
    +
    +void tbReset(TupleBatch tb)
    +{
    +    tb->iter = 0;
    +    tb->nrows = 0;
    +    memset(tb->skip,0, sizeof(bool) * tb->batchsize);
    +}
    +
    +void tbCreateColumn(TupleBatch tb,int colid,Oid type)
    +{
    +    if(tb->ncols <= colid)
    +        return;
    +    int bs = tb->batchsize;
    +
    +    GetVFunc(type)->vtbuild((bs));
    +}
    +
    +void tbfreeColumn(vheader** vh,int colid)
    +{
    +    GetVFunc(vh[colid]->elemtype)->vtfree(&vh[colid]);
    +}
    +
    +static size_t vtypeSize(vheader *vh)
    +{
    +    return GetVFunc(vh->elemtype)->vtsize(vh);
    +}
    +
    +static size_t
    +tbSerializationSize(TupleBatch tb)
    +{
    +    //buffer size stick in the head of the buffer
    +    size_t len = sizeof(size_t);
    +
    +    //get TupleBatch structure size
    +    len += offsetof(TupleBatchData ,skip);
    +
    +    //get skip tag size
    +    len += sizeof( bool ) * tb->nrows;
    +
    +    //get all un-null columns data size
    +    for(int i = 0;i < tb->ncols; i++ )
    +    {
    +        if(tb->datagroup[i])
    +        {
    +            len += sizeof(int);
    +            len += vtypeSize(tb->datagroup[i]);
    +        }
    +    }
    +    return len;
    +}
    +
    +unsigned char *
    +tbSerialization(TupleBatch tb )
    +{
    +    size_t len = 0;
    +    size_t tmplen = 0;
    +    //calculate total size for TupleBatch
    +    size_t size = tbSerializationSize(tb);
    +
    +    unsigned char *buffer = palloc(size);
    +
    +    //copy TupleBatch header
    +    memcpy(buffer,&size,sizeof(size_t));
    +
    +    tmplen = offsetof(TupleBatchData ,skip);
    +    memcpy(buffer+len,tb,tmplen);
    +    len += tmplen;
    +
    +    tmplen = sizeof(bool) * tb->nrows;
    +    memcpy(buffer+len,tb->skip,tmplen);
    +    len += tmplen;
    +
    +
    +    for(int i = 0;i < tb->ncols; i++ )
    +    {
    +        if(tb->datagroup[i])
    +        {
    +            memcpy(buffer+len,&i,sizeof(int));
    +            len += sizeof(int);
    +
    +            tmplen = GetVFunc(tb->datagroup[i]->elemtype)->serialization(tb->datagroup[i],buffer + len);
    +            len += tmplen;
    +        }
    +    }
    +
    +    return buffer;
    +}
    +
    +TupleBatch tbDeserialization(unsigned char *buffer)
    +{
    +    size_t buflen;
    +    memcpy(&buflen,buffer,sizeof(size_t));
    +
    +    if(buflen < sizeof(TupleBatchData))
    +        return NULL;
    +
    +    size_t len = 0;
    +    size_t tmplen = 0;
    +    TupleBatch tb = palloc0(sizeof(TupleBatchData));
    +
    +    //deserial tb main data
    +    tmplen = offsetof(TupleBatchData,skip);
    +    memcpy(tb,buffer+len,tmplen);
    +    len += tmplen;
    +
    +    //deserial member value -- skip
    +    if(tb->nrows != 0)
    +    {
    +        tb->skip = palloc(sizeof(bool) * tb->nrows);
    +        memcpy(tb->skip,buffer+len,tmplen);
    +        len += tmplen;
    +    }
    +
    +    //deserial member value -- datagroup
    +    if(tb->ncols != 0)
    +    {
    +        tb->datagroup = palloc0(sizeof(vheader*) * tb->ncols);
    +        int colid;
    +        while (len < buflen)
    +        {
    +            memcpy(&colid,buffer + len,sizeof(int));
    +            len += sizeof(int);
    +            tb->datagroup[colid] = (vheader* ) GetVFunc(((vheader *) (buffer + len))->elemtype)->deserialization(buffer + len,&tmplen);
    --- End diff --
    
    Do the serialization and deserialization always succeed ?


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by weinan003 <gi...@git.apache.org>.
Github user weinan003 commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175034657
  
    --- Diff: contrib/vexecutor/tuplebatch.h ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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.
    + */
    +#ifndef __TUPLEBATCH_H__
    +#define __TUPLEBATCH_H__
    +
    +#include "vexecutor.h"
    +#include "vcheck.h"
    +
    +
    +typedef struct TupleBatchData
    +{
    +    int     _tb_len;
    --- End diff --
    
    pg some lib function will require an int type variable as the total size of the struct recorder


---

[GitHub] incubator-hawq pull request #1347: HAWQ-1591 Common tuple batch structure fo...

Posted by wengyanqing <gi...@git.apache.org>.
Github user wengyanqing commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/1347#discussion_r175015247
  
    --- Diff: contrib/vexecutor/tuplebatch.c ---
    @@ -0,0 +1,188 @@
    +/*
    + * 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 "postgres.h"
    +#include "tuplebatch.h"
    +
    +static size_t vtypeSize(vheader *vh);
    +static size_t tbSerializationSize(TupleBatch tb);
    +
    +#define TBGETVALUEPTR(tb,type,colid,rowid) (&(((type *)tb->datagroup[colid])->data[rowid]))
    +#define TBGETISNULLPTR(tb,colid,rowid) (&((tb->datagroup[colid])->isnull[rowid]))
    +
    +TupleBatch tbGenerate(int colnum,int batchsize)
    +{
    +    Assert(colnum > 0 && batchsize > 0);
    +    TupleBatch tb = palloc0(sizeof(TupleBatchData));
    +    if(!tb)
    +    {
    +        elog(FATAL,"TupleBatch Allocation failed");
    +        return NULL;
    +    }
    +
    +    tb->ncols = colnum;
    +    tb->batchsize = batchsize;
    +
    +    tb->skip = palloc0(sizeof(bool) * tb->batchsize);
    +    tb->datagroup = palloc0(sizeof(struct vtypeheader*) * tb->ncols);
    +
    +    return tb;
    +}
    +
    +void tbDestory(TupleBatch* tb){
    --- End diff --
    
    tbDestory -> tbDestroy ?


---