You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/01/06 11:48:11 UTC

[GitHub] [nifi-minifi-cpp] adamdebreceni opened a new pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

adamdebreceni opened a new pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233


   Thank you for submitting a contribution to Apache NiFi - MiNiFi C++.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced
        in the commit message?
   
   - [ ] Does your PR title start with MINIFICPP-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically main)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   ### For code changes:
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the LICENSE file?
   - [ ] If applicable, have you updated the NOTICE file?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI results for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r783842590



##########
File path: extensions/http-curl/tests/C2DebugBundleTest.cpp
##########
@@ -0,0 +1,205 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+
+#include "TestBase.h"
+
+#include "c2/C2Agent.h"
+#include "protocols/RESTProtocol.h"
+#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "io/ArchiveStream.h"
+
+using std::literals::chrono_literals::operator""s;
+
+class VerifyDebugInfo : public VerifyC2Base {
+ public:
+  explicit VerifyDebugInfo(std::function<bool()> verify): verify_(std::move(verify)) {}
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTProtocol>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    assert(utils::verifyEventHappenedInPollTime(std::chrono::seconds(300), verify_));
+  }
+
+  void configureC2() override {
+    VerifyC2Base::configureC2();
+    configuration->set("nifi.c2.agent.heartbeat.period", "100");
+  }
+
+  std::function<bool()> verify_;
+};
+
+class C2DebugBundleHandler : public ServerAwareHandler {
+  static int field_found(const char* key, const char* filename, char* /*path*/, size_t /*pathlen*/, void* user_data) {

Review comment:
       moved them




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
lordgamez commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r783793699



##########
File path: extensions/http-curl/protocols/RESTSender.cpp
##########
@@ -102,17 +104,36 @@ const C2Payload RESTSender::sendPayload(const std::string url, const Direction d
   client.setKeepAliveIdle(std::chrono::milliseconds(2000));
   client.setConnectionTimeout(std::chrono::milliseconds(2000));
   if (direction == Direction::TRANSMIT) {
-    input = std::unique_ptr<utils::ByteInputCallBack>(new utils::ByteInputCallBack());
-    callback = std::unique_ptr<utils::HTTPUploadCallback>(new utils::HTTPUploadCallback());
-    input->write(outputConfig);
-    callback->ptr = input.get();
-    callback->pos = 0;
     client.set_request_method("POST");
     if (!ssl_context_service_ && url.find("https://") == 0) {
       setSecurityContext(client, "POST", url);
     }
-    client.setUploadCallback(callback.get());
-    client.setPostSize(outputConfig.size());
+    if (payload.getOperation() == Operation::TRANSFER) {
+      // treat nested payloads as files
+      for (auto& file : payload.getNestedPayloads()) {

Review comment:
       Could this be const?

##########
File path: extensions/http-curl/tests/C2DebugBundleTest.cpp
##########
@@ -0,0 +1,205 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+
+#include "TestBase.h"
+
+#include "c2/C2Agent.h"
+#include "protocols/RESTProtocol.h"
+#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "io/ArchiveStream.h"
+
+using std::literals::chrono_literals::operator""s;
+
+class VerifyDebugInfo : public VerifyC2Base {
+ public:
+  explicit VerifyDebugInfo(std::function<bool()> verify): verify_(std::move(verify)) {}
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTProtocol>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    assert(utils::verifyEventHappenedInPollTime(std::chrono::seconds(300), verify_));
+  }
+
+  void configureC2() override {
+    VerifyC2Base::configureC2();
+    configuration->set("nifi.c2.agent.heartbeat.period", "100");
+  }
+
+  std::function<bool()> verify_;
+};
+
+class C2DebugBundleHandler : public ServerAwareHandler {
+  static int field_found(const char* key, const char* filename, char* /*path*/, size_t /*pathlen*/, void* user_data) {

Review comment:
       Public functions should be listed first in the class definition according to core guidelines [NL.16](https://github.com/isocpp/CppCoreGuidelines/blob/master/CppCoreGuidelines.md#Rl-order).

##########
File path: extensions/http-curl/tests/C2DebugBundleTest.cpp
##########
@@ -0,0 +1,205 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+
+#include "TestBase.h"
+
+#include "c2/C2Agent.h"
+#include "protocols/RESTProtocol.h"
+#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "io/ArchiveStream.h"
+
+using std::literals::chrono_literals::operator""s;
+
+class VerifyDebugInfo : public VerifyC2Base {
+ public:
+  explicit VerifyDebugInfo(std::function<bool()> verify): verify_(std::move(verify)) {}
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTProtocol>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    assert(utils::verifyEventHappenedInPollTime(std::chrono::seconds(300), verify_));
+  }
+
+  void configureC2() override {
+    VerifyC2Base::configureC2();
+    configuration->set("nifi.c2.agent.heartbeat.period", "100");
+  }
+
+  std::function<bool()> verify_;
+};
+
+class C2DebugBundleHandler : public ServerAwareHandler {
+  static int field_found(const char* key, const char* filename, char* /*path*/, size_t /*pathlen*/, void* user_data) {
+    auto& file_content = *static_cast<std::optional<std::string>*>(user_data);
+    if (!filename || std::string(filename) != "debug.tar.gz") {
+      throw std::runtime_error("Unknown form entry: " + std::string{key});
+    }
+    if (file_content) {
+      throw std::runtime_error("Debug archive has already been extracted: " + std::string{key});
+    }
+    return MG_FORM_FIELD_STORAGE_GET;
+  }
+  static int field_get(const char* /*key*/, const char* value, size_t valuelen, void* user_data) {
+    auto& file_content = *static_cast<std::optional<std::string>*>(user_data);
+    file_content = "";
+    (*file_content) += std::string(value, valuelen);
+    return MG_FORM_FIELD_HANDLE_GET;
+  }
+
+ public:
+  bool handlePost(CivetServer* /*server*/, struct mg_connection *conn) override {
+    std::optional<std::string> file_content;
+    mg_form_data_handler form_handler;
+    form_handler.field_found = field_found;
+    form_handler.field_get = field_get;
+    form_handler.user_data = &file_content;
+    mg_handle_form_request(conn, &form_handler);
+    assert(file_content);
+    {
+      std::lock_guard<std::mutex> lock(mtx_);
+      bundles_.push_back(std::move(*file_content));
+    }
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                      "text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+
+  std::vector<std::string> getBundles() {
+    std::lock_guard<std::mutex> lock(mtx_);
+    return bundles_;
+  }
+
+  std::mutex mtx_;
+  std::vector<std::string> bundles_;

Review comment:
       Could these be private?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r780974144



##########
File path: extensions/http-curl/client/HTTPClient.cpp
##########
@@ -62,6 +76,10 @@ HTTPClient::~HTTPClient() {
     curl_easy_cleanup(http_session_);
     http_session_ = nullptr;
   }
+  if (form_ != nullptr) {
+    curl_mime_free(form_);

Review comment:
       deleted copy constructor/assignment

##########
File path: extensions/http-curl/protocols/RESTSender.h
##########
@@ -52,7 +52,7 @@ class RESTSender : public RESTProtocol, public C2Protocol {
   void initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) override;
 
  protected:
-  virtual const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, const std::string outputConfig);
+  const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, std::optional<std::string> data);

Review comment:
       removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] lordgamez closed pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
lordgamez closed pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r780375784



##########
File path: extensions/http-curl/client/HTTPClient.cpp
##########
@@ -62,6 +76,10 @@ HTTPClient::~HTTPClient() {
     curl_easy_cleanup(http_session_);
     http_session_ = nullptr;
   }
+  if (form_ != nullptr) {
+    curl_mime_free(form_);

Review comment:
       Could you disable the copy of HTTPClient to make this (and the rest of the class) safe?

##########
File path: extensions/http-curl/protocols/RESTSender.h
##########
@@ -52,7 +52,7 @@ class RESTSender : public RESTProtocol, public C2Protocol {
   void initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) override;
 
  protected:
-  virtual const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, const std::string outputConfig);
+  const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, std::optional<std::string> data);

Review comment:
       Since this is no longer virtual, could you also remove `const` from the return type? Top-level `const` is ignored in the return type.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] szaszm commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
szaszm commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r780375784



##########
File path: extensions/http-curl/client/HTTPClient.cpp
##########
@@ -62,6 +76,10 @@ HTTPClient::~HTTPClient() {
     curl_easy_cleanup(http_session_);
     http_session_ = nullptr;
   }
+  if (form_ != nullptr) {
+    curl_mime_free(form_);

Review comment:
       Could you disable the copy of HTTPClient to make this (and the rest of the class) safe?

##########
File path: extensions/http-curl/protocols/RESTSender.h
##########
@@ -52,7 +52,7 @@ class RESTSender : public RESTProtocol, public C2Protocol {
   void initialize(core::controller::ControllerServiceProvider* controller, const std::shared_ptr<Configure> &configure) override;
 
  protected:
-  virtual const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, const std::string outputConfig);
+  const C2Payload sendPayload(const std::string url, const Direction direction, const C2Payload &payload, std::optional<std::string> data);

Review comment:
       Since this is no longer virtual, could you also remove `const` from the return type? Top-level `const` is ignored in the return type.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r783842716



##########
File path: extensions/http-curl/tests/C2DebugBundleTest.cpp
##########
@@ -0,0 +1,205 @@
+/**
+ *
+ * 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.
+ */
+
+#undef NDEBUG
+
+#include "TestBase.h"
+
+#include "c2/C2Agent.h"
+#include "protocols/RESTProtocol.h"
+#include "protocols/RESTSender.h"
+#include "HTTPIntegrationBase.h"
+#include "HTTPHandlers.h"
+#include "io/ArchiveStream.h"
+
+using std::literals::chrono_literals::operator""s;
+
+class VerifyDebugInfo : public VerifyC2Base {
+ public:
+  explicit VerifyDebugInfo(std::function<bool()> verify): verify_(std::move(verify)) {}
+
+  void testSetup() override {
+    LogTestController::getInstance().setTrace<minifi::c2::C2Agent>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTSender>();
+    LogTestController::getInstance().setDebug<minifi::c2::RESTProtocol>();
+    VerifyC2Base::testSetup();
+  }
+
+  void runAssertions() override {
+    assert(utils::verifyEventHappenedInPollTime(std::chrono::seconds(300), verify_));
+  }
+
+  void configureC2() override {
+    VerifyC2Base::configureC2();
+    configuration->set("nifi.c2.agent.heartbeat.period", "100");
+  }
+
+  std::function<bool()> verify_;
+};
+
+class C2DebugBundleHandler : public ServerAwareHandler {
+  static int field_found(const char* key, const char* filename, char* /*path*/, size_t /*pathlen*/, void* user_data) {
+    auto& file_content = *static_cast<std::optional<std::string>*>(user_data);
+    if (!filename || std::string(filename) != "debug.tar.gz") {
+      throw std::runtime_error("Unknown form entry: " + std::string{key});
+    }
+    if (file_content) {
+      throw std::runtime_error("Debug archive has already been extracted: " + std::string{key});
+    }
+    return MG_FORM_FIELD_STORAGE_GET;
+  }
+  static int field_get(const char* /*key*/, const char* value, size_t valuelen, void* user_data) {
+    auto& file_content = *static_cast<std::optional<std::string>*>(user_data);
+    file_content = "";
+    (*file_content) += std::string(value, valuelen);
+    return MG_FORM_FIELD_HANDLE_GET;
+  }
+
+ public:
+  bool handlePost(CivetServer* /*server*/, struct mg_connection *conn) override {
+    std::optional<std::string> file_content;
+    mg_form_data_handler form_handler;
+    form_handler.field_found = field_found;
+    form_handler.field_get = field_get;
+    form_handler.user_data = &file_content;
+    mg_handle_form_request(conn, &form_handler);
+    assert(file_content);
+    {
+      std::lock_guard<std::mutex> lock(mtx_);
+      bundles_.push_back(std::move(*file_content));
+    }
+    mg_printf(conn, "HTTP/1.1 200 OK\r\nContent-Type: "
+                      "text/plain\r\nContent-Length: 0\r\nConnection: close\r\n\r\n");
+    return true;
+  }
+
+  std::vector<std::string> getBundles() {
+    std::lock_guard<std::mutex> lock(mtx_);
+    return bundles_;
+  }
+
+  std::mutex mtx_;
+  std::vector<std::string> bundles_;

Review comment:
       made them private




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi-minifi-cpp] adamdebreceni commented on a change in pull request #1233: MINIFICPP-1684 - Provide agent logs through the c2 protocol

Posted by GitBox <gi...@apache.org>.
adamdebreceni commented on a change in pull request #1233:
URL: https://github.com/apache/nifi-minifi-cpp/pull/1233#discussion_r783842490



##########
File path: extensions/http-curl/protocols/RESTSender.cpp
##########
@@ -102,17 +104,36 @@ const C2Payload RESTSender::sendPayload(const std::string url, const Direction d
   client.setKeepAliveIdle(std::chrono::milliseconds(2000));
   client.setConnectionTimeout(std::chrono::milliseconds(2000));
   if (direction == Direction::TRANSMIT) {
-    input = std::unique_ptr<utils::ByteInputCallBack>(new utils::ByteInputCallBack());
-    callback = std::unique_ptr<utils::HTTPUploadCallback>(new utils::HTTPUploadCallback());
-    input->write(outputConfig);
-    callback->ptr = input.get();
-    callback->pos = 0;
     client.set_request_method("POST");
     if (!ssl_context_service_ && url.find("https://") == 0) {
       setSecurityContext(client, "POST", url);
     }
-    client.setUploadCallback(callback.get());
-    client.setPostSize(outputConfig.size());
+    if (payload.getOperation() == Operation::TRANSFER) {
+      // treat nested payloads as files
+      for (auto& file : payload.getNestedPayloads()) {

Review comment:
       made it const




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org