You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hawq.apache.org by amyrazz44 <gi...@git.apache.org> on 2017/01/17 09:49:54 UTC

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

GitHub user amyrazz44 opened a pull request:

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

    HAWQ-1193. Add createEncryption, getEZForPath, listEncryptionZones RP\u2026

    \u2026C for libhdfs3.

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

    $ git pull https://github.com/amyrazz44/incubator-hawq TDEPR

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

    https://github.com/apache/incubator-hawq/pull/1093.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 #1093
    
----
commit ec29a4c4f02063b496f9078cfdccd96107d7f72a
Author: ivan <iw...@pivotal.io>
Date:   2016-12-20T08:51:18Z

    HAWQ-1193. Add createEncryption, getEZForPath, listEncryptionZones RPC for libhdfs3.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96558929
  
    --- Diff: depends/libhdfs3/src/client/FileEncryptionInfo.h ---
    @@ -0,0 +1,93 @@
    +/********************************************************************
    + * 2014 -
    + * open source under Apache License Version 2.0
    + ********************************************************************/
    +/**
    + * 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 _HDFS_LIBHDFS3_CLIENT_FILEENCRYPTIONINFO_H_
    +#define _HDFS_LIBHDFS3_CLIENT_FILEENCRYPTIONINFO_H_
    +
    +#include <string>
    +
    +namespace Hdfs {
    +
    +class FileEncryptionInfo {
    +public:
    +    FileEncryptionInfo() : 
    +		cryptoProtocolVersion(0), suite(0){
    +    }
    +
    +    int getSuite() const {
    +        return suite;
    +    }
    +
    +    void setSuite(int suite) {
    +        this->suite = suite;
    +    }
    +
    +    int getCryptoProtocolVersion() const {
    +        return cryptoProtocolVersion;
    +    }
    +
    +    void setCryptoProtocolVersion(int cryptoProtocolVersion) {
    +        this->cryptoProtocolVersion = cryptoProtocolVersion;
    +    }
    +
    +    const char * getKey() const{
    +        return key.c_str();
    --- End diff --
    
    Dose key material contains \0 ? 
    If so, maybe return a std::string is better? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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

    https://github.com/apache/incubator-hawq/pull/1093#discussion_r96561226
  
    --- Diff: depends/libhdfs3/src/client/EncryptionZoneInfo.h ---
    @@ -0,0 +1,84 @@
    +/********************************************************************
    --- End diff --
    
    suggest a new file do not include line:1 ~ line:4 
    standard apache source file should start from line:5


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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

    https://github.com/apache/incubator-hawq/pull/1093#discussion_r96379546
  
    --- Diff: depends/libhdfs3/test/function/TestCInterface.cpp ---
    @@ -198,6 +201,45 @@ TEST(TestCInterfaceConnect, TestConnect_Success) {
         ASSERT_EQ(hdfsDisconnect(fs), 0);
     }
     
    +TEST(TestCInterfaceTDE, DISABLED_TestCreateEnRPC_Success) {
    +    hdfsFS fs = NULL;
    +    hdfsEncryptionZoneInfo * enInfo = NULL;
    +    char * uri = NULL;
    +    setenv("LIBHDFS3_CONF", "function-test.xml", 1);
    +    struct hdfsBuilder * bld = hdfsNewBuilder();
    +    assert(bld != NULL);
    +    hdfsBuilderSetNameNode(bld, "default");
    +    fs = hdfsBuilderConnect(bld);
    +    ASSERT_TRUE(fs != NULL);
    +	system("hadoop fs -rmr /TDE");
    --- End diff --
    
    indent doesn't match 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96559108
  
    --- Diff: depends/libhdfs3/src/client/FileStatus.h ---
    @@ -143,6 +144,21 @@ class FileStatus {
             return !symlink.empty();
         }
     
    +    /**
    +    * Get encryption information for a file.
    +    */
    +    FileEncryptionInfo* getFileEncryption(){
    +        return &fileEncryption;
    +    }
    +
    +    /**
    +    * Is an encryption file?
    +    * @return true is this is an encryption file
    +    */
    +    bool isFileEncrypted() const {
    +        return strlen(fileEncryption.getKey()) > 0 || strlen(fileEncryption.getKeyName()) > 0;
    --- End diff --
    
    `||` -> `&&` is better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq issue #1093: HAWQ-1193. Add createEncryption, getEZForPath, l...

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

    https://github.com/apache/incubator-hawq/pull/1093
  
    +1 LGTM, please add some comments to explain the logic if necessary. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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

    https://github.com/apache/incubator-hawq/pull/1093#discussion_r96570395
  
    --- Diff: depends/libhdfs3/src/client/FileEncryptionInfo.h ---
    @@ -0,0 +1,93 @@
    +/********************************************************************
    + * 2014 -
    + * open source under Apache License Version 2.0
    + ********************************************************************/
    +/**
    + * 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 _HDFS_LIBHDFS3_CLIENT_FILEENCRYPTIONINFO_H_
    +#define _HDFS_LIBHDFS3_CLIENT_FILEENCRYPTIONINFO_H_
    +
    +#include <string>
    +
    +namespace Hdfs {
    +
    +class FileEncryptionInfo {
    +public:
    +    FileEncryptionInfo() : 
    +		cryptoProtocolVersion(0), suite(0){
    +    }
    +
    +    int getSuite() const {
    +        return suite;
    +    }
    +
    +    void setSuite(int suite) {
    +        this->suite = suite;
    +    }
    +
    +    int getCryptoProtocolVersion() const {
    +        return cryptoProtocolVersion;
    +    }
    +
    +    void setCryptoProtocolVersion(int cryptoProtocolVersion) {
    +        this->cryptoProtocolVersion = cryptoProtocolVersion;
    +    }
    +
    +    const char * getKey() const{
    +        return key.c_str();
    --- End diff --
    
    According to libhdfs3 coding format, all the related class choose char * instead of sting as the data type. In the meanwhile, ConstructHdfsEncryptionFileInfo() will call Strdup() to memcpy char * in Hdfs.cpp. So, will use char * data type. Thank you, interma. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96560410
  
    --- Diff: depends/libhdfs3/src/client/Hdfs.cpp ---
    @@ -989,8 +989,31 @@ int hdfsSetReplication(hdfsFS fs, const char * path, int16_t replication) {
         return -1;
     }
     
    +static void ConstructHdfsEncryptionZoneInfo(hdfsEncryptionZoneInfo * infoEn,
    +                                  std::vector<Hdfs::EncryptionZoneInfo> & enStatus) {
    +    size_t size = enStatus.size();
    +
    +    for (size_t i = 0; i < size; ++i) {
    +        infoEn[i].mSuite = enStatus[i].getSuite();
    +        infoEn[i].mCryptoProtocolVersion = enStatus[i].getCryptoProtocolVersion();
    +        infoEn[i].mId = enStatus[i].getId();
    +        infoEn[i].mPath = Strdup(enStatus[i].getPath());
    +        infoEn[i].mKeyName = Strdup(enStatus[i].getKeyName());
    +    }
    +}
    +
    +static void ConstructHdfsEncryptionFileInfo(hdfsEncryptionFileInfo * infoEn,
    +                                  Hdfs::FileEncryptionInfo* enStatus) {
    +    infoEn->mSuite = enStatus->getSuite();
    +    infoEn->mCryptoProtocolVersion = enStatus->getCryptoProtocolVersion();
    +    infoEn->mKey = Strdup(enStatus->getKey());
    +    infoEn->mKeyName = Strdup(enStatus->getKeyName());
    +    infoEn->mIv = Strdup(enStatus->getIv());
    --- End diff --
    
    Should to confirm whether IV material contains \0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96561207
  
    --- Diff: depends/libhdfs3/src/client/Hdfs.cpp ---
    @@ -1450,6 +1491,72 @@ void hdfsFreeFileBlockLocations(BlockLocation * locations, int numOfBlock) {
         delete [] locations;
     }
     
    +int hdfsCreateEncryptionZone(hdfsFS fs, const char * path, const char * keyName) {
    +    PARAMETER_ASSERT(fs && path && strlen(path) > 0 && keyName && strlen(keyName) > 0, -1, EINVAL);
    +
    +    try {
    +        return fs->getFilesystem().createEncryptionZone(path, keyName) ? 0 : -1;
    +    } catch (const std::bad_alloc & e) {
    +        SetErrorMessage("Out of memory");
    +        errno = ENOMEM;
    +    } catch (...) {
    +        SetLastException(Hdfs::current_exception());
    +        handleException(Hdfs::current_exception());
    +    }
    +
    +    return -1;
    +}
    +
    +hdfsEncryptionZoneInfo * hdfsGetEZForPath(hdfsFS fs, const char * path) {
    +    PARAMETER_ASSERT(fs && path && strlen(path) > 0, NULL, EINVAL);
    +    hdfsEncryptionZoneInfo * retval = NULL;
    +
    +    try {
    +        retval = new hdfsEncryptionZoneInfo[1];
    +        memset(retval, 0, sizeof(hdfsEncryptionZoneInfo));
    +        std::vector<Hdfs::EncryptionZoneInfo> enStatus(1);
    +        enStatus[0] = fs->getFilesystem().getEZForPath(path);
    +        ConstructHdfsEncryptionZoneInfo(retval, enStatus);
    +        return retval;
    +    } catch (const std::bad_alloc & e) {
    +        SetErrorMessage("Out of memory");
    +        hdfsFreeEncryptionZoneInfo(retval, 1);
    +        errno = ENOMEM;
    +    } catch (...) {
    +        SetLastException(Hdfs::current_exception());
    +        hdfsFreeEncryptionZoneInfo(retval, 1);
    +        handleException(Hdfs::current_exception());
    +    }
    +
    +    return NULL;
    +}
    +
    +
    +hdfsEncryptionZoneInfo * hdfsListEncryptionZones(hdfsFS fs, int * numEntries) {
    +    PARAMETER_ASSERT(fs, NULL, EINVAL);
    +    hdfsEncryptionZoneInfo * retval = NULL;
    +    int size = 0;
    +
    +    try {
    +        std::vector<Hdfs::EncryptionZoneInfo> enStatus =
    +            fs->getFilesystem().listAllEncryptionZoneItems();
    +        size = enStatus.size();
    +        retval = new hdfsEncryptionZoneInfo[size];
    +        memset(retval, 0, sizeof(hdfsEncryptionZoneInfo) * size);
    +        ConstructHdfsEncryptionZoneInfo(&retval[0], enStatus);
    +        LOG(Hdfs::Internal::WARNING, "The size of hdfsEncryptionZoneInfo is %d", size);
    --- End diff --
    
    log level: Warning => DEBUG is better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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

    https://github.com/apache/incubator-hawq/pull/1093#discussion_r96379295
  
    --- Diff: depends/libhdfs3/src/proto/encryption.proto ---
    @@ -0,0 +1,67 @@
    +/**
    + * 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.
    + */
    +
    +/**
    + * These .proto interfaces are private and stable.
    + * Please see http://wiki.apache.org/hadoop/Compatibility
    + * for what changes are allowed for a *stable* .proto interface.
    + */
    +
    +// This file contains protocol buffers that are used throughout HDFS -- i.e.
    --- End diff --
    
    Suggest use /**/ instead of // 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96584985
  
    --- Diff: depends/libhdfs3/src/client/hdfs.h ---
    @@ -482,6 +482,30 @@ int hdfsCreateDirectory(hdfsFS fs, const char * path);
     int hdfsSetReplication(hdfsFS fs, const char * path, int16_t replication);
     
     /**
    + * hdfsEncryptionZoneInfo- Information about an encryption zone.
    + */
    +typedef struct {
    +    int mSuite; /* the suite of encryption zone */
    +    int mCryptoProtocolVersion; /* the version of crypto protocol */
    +    int64_t mId; /* the id of encryption zone */
    +    char * mPath; /* the path of encryption zone */
    +    char * mKeyName; /* the key name of encryption zone */
    +} hdfsEncryptionZoneInfo;
    +
    +
    +/**
    + * hdfsEncryptionFileInfo - Information about an encryption file/directory.
    + */
    +typedef struct {
    +    int mSuite; /* the suite of encryption file/directory */
    +    int mCryptoProtocolVersion; /* the version of crypto protocol */
    +    char * mKey; /* the key of encryption file/directory */
    +    char * mKeyName; /* the key name of encryption file/directory */
    +    char * mIv; /* the iv of encryption file/directory */
    +    char * mEzKeyVersionName; /* the version encryption file/directory */
    --- End diff --
    
    The 4 `char *` field, change to array(e.g. `char mKey[16];` ) or new a buffer(e.g. call `Strdup()` ) is better?
    
    Don't change is also ok, and should use them carefully in future code (avoid the space of their point is invalid).



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq issue #1093: HAWQ-1193. Add createEncryption, getEZForPath, l...

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

    https://github.com/apache/incubator-hawq/pull/1093
  
    @wangzw @xunzhang @interma  Please help review this pull request, thank you so much.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #1093: HAWQ-1193. Add createEncryption, getEZFor...

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/1093#discussion_r96560375
  
    --- Diff: depends/libhdfs3/src/client/Hdfs.cpp ---
    @@ -989,8 +989,31 @@ int hdfsSetReplication(hdfsFS fs, const char * path, int16_t replication) {
         return -1;
     }
     
    +static void ConstructHdfsEncryptionZoneInfo(hdfsEncryptionZoneInfo * infoEn,
    +                                  std::vector<Hdfs::EncryptionZoneInfo> & enStatus) {
    +    size_t size = enStatus.size();
    +
    +    for (size_t i = 0; i < size; ++i) {
    +        infoEn[i].mSuite = enStatus[i].getSuite();
    +        infoEn[i].mCryptoProtocolVersion = enStatus[i].getCryptoProtocolVersion();
    +        infoEn[i].mId = enStatus[i].getId();
    +        infoEn[i].mPath = Strdup(enStatus[i].getPath());
    +        infoEn[i].mKeyName = Strdup(enStatus[i].getKeyName());
    +    }
    +}
    +
    +static void ConstructHdfsEncryptionFileInfo(hdfsEncryptionFileInfo * infoEn,
    +                                  Hdfs::FileEncryptionInfo* enStatus) {
    +    infoEn->mSuite = enStatus->getSuite();
    +    infoEn->mCryptoProtocolVersion = enStatus->getCryptoProtocolVersion();
    +    infoEn->mKey = Strdup(enStatus->getKey());
    --- End diff --
    
    Still should to confirm whether key material contains \0.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---