You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2022/05/12 11:46:41 UTC

[GitHub] [geode-native] albertogpz opened a new pull request, #970: GEODE-10300: C++ native client: Allow locator responses greater than …

albertogpz opened a new pull request, #970:
URL: https://github.com/apache/geode-native/pull/970

   …3000 bytes
   
   If a response message from the locator to the C++ native client
   is longer than 3000 bytes the C++ native client will only
   read the first 3000 bytes.


-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r917042197


##########
cppcache/src/GetAllServersResponse.cpp:
##########
@@ -21,10 +21,10 @@ namespace geode {
 namespace client {
 
 void GetAllServersResponse::toData(DataOutput& output) const {
-  int32_t numServers = static_cast<int32_t>(m_servers.size());
+  int32_t numServers = static_cast<int32_t>(servers_.size());

Review Comment:
   `auto`



##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -502,21 +502,21 @@ class APACHE_GEODE_EXPORT DataInput {
 
   inline char readPdxChar() { return static_cast<char>(readInt16()); }
 
-  inline void _checkBufferSize(size_t size, int32_t line) {
-    if ((m_bufLength - (m_buf - m_bufHead)) < size) {
+  virtual void _checkBufferSize(size_t size, int32_t line) {

Review Comment:
   This is a breaking ABI change. This should only occur at a minor release. Are we sure this needs to change?



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;

Review Comment:
   `constexpr`



##########
cppcache/test/StreamDataInputTest.cpp:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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 <gmock/gmock-actions.h>
+#include <gmock/gmock-matchers.h>
+
+#include <gtest/gtest.h>
+
+#include "CacheImpl.hpp"
+#include "Connector.hpp"
+#include "GetAllServersResponse.hpp"
+#include "ServerLocation.hpp"
+#include "StreamDataInput.hpp"
+#include "geode/DataOutput.hpp"
+#include "mock/ConnectorMock.hpp"
+
+namespace {
+
+using apache::geode::client::CacheImpl;
+using apache::geode::client::Connector;
+using apache::geode::client::ConnectorMock;
+using apache::geode::client::DataOutput;
+using apache::geode::client::GetAllServersResponse;
+using apache::geode::client::Serializable;
+using apache::geode::client::ServerLocation;
+using apache::geode::client::StreamDataInput;
+using apache::geode::client::TimeoutException;
+
+using ::testing::_;
+using ::testing::DoAll;
+using ::testing::Eq;
+using ::testing::Return;
+using ::testing::SetArrayArgument;
+using ::testing::SizeIs;
+
+const size_t kReadBuffSize = 3000;

Review Comment:
   `constexpr`



##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -452,19 +452,19 @@ class APACHE_GEODE_EXPORT DataInput {
   DataInput& operator=(DataInput&&) = default;
 
  protected:
+  const uint8_t* buf_;

Review Comment:
   Love the update to the current naming convention but please also update to full words too.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout) {}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[kBufferSize];
+  while (getBytesRemaining() < size) {
+    const auto start = std::chrono::system_clock::now();
+
+    const auto receivedLength = connector_->receive_nothrowiftimeout(
+        buff, kBufferSize,
+        std::chrono::duration_cast<std::chrono::milliseconds>(
+            remainingTimeBeforeTimeout_));

Review Comment:
   If the most granular time is actually in milliseconds does it make sense to even have `remainingTimeBeforeTimeout_` be in microseconds?



##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -452,19 +452,19 @@ class APACHE_GEODE_EXPORT DataInput {
   DataInput& operator=(DataInput&&) = default;
 
  protected:

Review Comment:
   Making all these protected is likely an ABI breaking change.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872292220


##########
cppcache/src/CacheXmlCreation.cpp:
##########
@@ -50,12 +50,12 @@ void CacheXmlCreation::create(Cache* cache) {
 }
 
 void CacheXmlCreation::setPdxIgnoreUnreadField(bool ignore) {
-  // m_cache->m_cacheImpl->setPdxIgnoreUnreadFields(ignore);
+  // cache_->m_cacheImpl->setPdxIgnoreUnreadFields(ignore);

Review Comment:
   Yep, CLion did it for me and I did not check it.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871422678


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));
+  }
+}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[BUFF_SIZE];
+  while ((m_bufLength - (m_buf - m_bufHead)) < size) {
+    const auto start = std::chrono::system_clock::now();
+
+    const auto receivedLength = m_connector->receive_nothrowiftimeout(
+        buff, BUFF_SIZE,
+        std::chrono::duration_cast<std::chrono::milliseconds>(
+            m_remainingTimeBeforeTimeout));
+
+    const auto timeSpent = std::chrono::system_clock::now() - start;
+
+    m_remainingTimeBeforeTimeout -=
+        std::chrono::duration_cast<decltype(m_remainingTimeBeforeTimeout)>(
+            timeSpent);
+
+    LOGDEBUG(
+        "received %d bytes from %s: %s, time spent: "
+        "%ld microsecs, time remaining before timeout: %ld microsecs",
+        receivedLength, m_connector->getRemoteEndpoint().c_str(),
+        Utils::convertBytesToString(reinterpret_cast<uint8_t*>(buff),
+                                    receivedLength)
+            .c_str(),
+        std::chrono::duration_cast<std::chrono::microseconds>(timeSpent)
+            .count(),
+        std::chrono::duration_cast<std::chrono::microseconds>(
+            m_remainingTimeBeforeTimeout)
+            .count());
+
+    if (m_remainingTimeBeforeTimeout <= std::chrono::microseconds ::zero()) {
+      throw(TimeoutException(std::string("Timeout when receiving from ")
+                                 .append(m_connector->getRemoteEndpoint())));
+    }
+
+    size_t newLength = m_bufLength + receivedLength;

Review Comment:
   `auto`



##########
cppcache/src/StreamDataInput.hpp:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_STREAMDATAINPUT_H_
+#define GEODE_STREAMDATAINPUT_H_
+
+#include <chrono>
+
+#include "Connector.hpp"
+#include "geode/DataInput.hpp"
+
+/**

Review Comment:
   Delete these this comment since it provides no useful in formation.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));

Review Comment:
   Let's use `std::unique_ptr` for automatic ownership and lifecycle management.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;

Review Comment:
   Google convention is `kBufferSize`.



##########
cppcache/src/StreamDataInput.hpp:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_STREAMDATAINPUT_H_
+#define GEODE_STREAMDATAINPUT_H_
+
+#include <chrono>
+
+#include "Connector.hpp"
+#include "geode/DataInput.hpp"
+
+/**
+ * @file
+ */
+
+namespace apache {
+namespace geode {
+namespace client {
+
+#include "geode/DataInput.hpp"

Review Comment:
   duplicate include.



##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -502,7 +502,7 @@ class APACHE_GEODE_EXPORT DataInput {
 
   inline char readPdxChar() { return static_cast<char>(readInt16()); }
 
-  inline void _checkBufferSize(size_t size, int32_t line) {
+  virtual inline void _checkBufferSize(size_t size, int32_t line) {

Review Comment:
   Having a mix of inline and virtual attributes produces some really interesting unknown behavior. 



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz merged pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz merged PR #970:
URL: https://github.com/apache/geode-native/pull/970


-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r948127352


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout) {}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[kBufferSize];
+  while (getBytesRemaining() < size) {
+    const auto start = std::chrono::system_clock::now();
+
+    const auto receivedLength = connector_->receive_nothrowiftimeout(
+        buff, kBufferSize,
+        std::chrono::duration_cast<std::chrono::milliseconds>(
+            remainingTimeBeforeTimeout_));

Review Comment:
   The duration cast can go away now that all time is milliseconds.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r948230496


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -60,9 +58,7 @@ void StreamDataInput::readDataIfNotAvailable(size_t size) {
             .c_str(),
         std::chrono::duration_cast<std::chrono::milliseconds>(timeSpent)
             .count(),
-        std::chrono::duration_cast<std::chrono::milliseconds>(
-            remainingTimeBeforeTimeout_)
-            .count());
+        remainingTimeBeforeTimeout_.count());
 
     if (remainingTimeBeforeTimeout_ <= std::chrono::milliseconds ::zero()) {

Review Comment:
   Surprised the formatter didn't catch this but what's with the space between milliseconds and zero?



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871464649


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -502,7 +502,7 @@ class APACHE_GEODE_EXPORT DataInput {
 
   inline char readPdxChar() { return static_cast<char>(readInt16()); }
 
-  inline void _checkBufferSize(size_t size, int32_t line) {
+  virtual inline void _checkBufferSize(size_t size, int32_t line) {

Review Comment:
   Haha! You would think.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on PR #970:
URL: https://github.com/apache/geode-native/pull/970#issuecomment-1125033318

   Pease correct your commit message now or when you merge to conform to our commit message standard.


-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] pivotal-jbarrett commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
pivotal-jbarrett commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871514907


##########
cppcache/src/StreamDataInput.hpp:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_STREAMDATAINPUT_H_
+#define GEODE_STREAMDATAINPUT_H_
+
+#include <chrono>
+
+#include "Connector.hpp"
+#include "geode/DataInput.hpp"
+
+/**

Review Comment:
   I just mean these `@file` comments we find all over the source.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r948236611


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -60,9 +58,7 @@ void StreamDataInput::readDataIfNotAvailable(size_t size) {
             .c_str(),
         std::chrono::duration_cast<std::chrono::milliseconds>(timeSpent)
             .count(),
-        std::chrono::duration_cast<std::chrono::milliseconds>(
-            remainingTimeBeforeTimeout_)
-            .count());
+        remainingTimeBeforeTimeout_.count());
 
     if (remainingTimeBeforeTimeout_ <= std::chrono::milliseconds ::zero()) {

Review Comment:
   Weird...



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r917628356


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -502,21 +502,21 @@ class APACHE_GEODE_EXPORT DataInput {
 
   inline char readPdxChar() { return static_cast<char>(readInt16()); }
 
-  inline void _checkBufferSize(size_t size, int32_t line) {
-    if ((m_bufLength - (m_buf - m_bufHead)) < size) {
+  virtual void _checkBufferSize(size_t size, int32_t line) {

Review Comment:
   I am afraid it is necessary for the solution. But given that this will go in a minor release I do not think it is an issue.
   Is there a log file in place where I could add this change as proposed in: https://lists.apache.org/thread/0v2r5trv926oj8qrgfp00467y3rycsvw
   



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871476433


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));

Review Comment:
   I do not think that's possible.
   The `DataInput` constructor has a `const uint8_t*` argument which is later assigned to the `m_bufHead` member variable.
   In the case of `StreamDataInput`, that pointer is not passed in the constructor to fill the member variable. Instead, it is allocated dynamically and it can even be reallocated.
   
   I could add a new `unique_ptr` member variable to `StreamDataInput` wrapping `m_bufHead` so that it manages the memory of `m_bufHead`. But then, I would not be able to use realloc in case I needed a larger size buffer. I would have to free m_bufHead and create a new one with a bigger size and copy into it the contents of the original one.
   I do not think it is worth it.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r917628778


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -452,19 +452,19 @@ class APACHE_GEODE_EXPORT DataInput {
   DataInput& operator=(DataInput&&) = default;
 
  protected:

Review Comment:
   Not sure. Anyhow, this fix should go in a minor release so should not be an issue.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r917731818


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout) {}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[kBufferSize];
+  while (getBytesRemaining() < size) {
+    const auto start = std::chrono::system_clock::now();
+
+    const auto receivedLength = connector_->receive_nothrowiftimeout(
+        buff, kBufferSize,
+        std::chrono::duration_cast<std::chrono::milliseconds>(
+            remainingTimeBeforeTimeout_));

Review Comment:
   Probably not. I will change it to milliseconds.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] gaussianrecurrence commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872129402


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -452,19 +452,19 @@ class APACHE_GEODE_EXPORT DataInput {
   DataInput& operator=(DataInput&&) = default;
 
  protected:
+  const uint8_t* m_buf;
+  const uint8_t* m_bufHead;

Review Comment:
   Naming notation for member attributes in Geode Native follows lowerCamelCase with _ as a postfix, to indicate that it's a member variable. As example 'm_buf' would rather be 'buf_'. Usually whenever we make a change that touches part of the code with the old notation (m_...) the idea is to change it to the right notation



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] gaussianrecurrence commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872129402


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -452,19 +452,19 @@ class APACHE_GEODE_EXPORT DataInput {
   DataInput& operator=(DataInput&&) = default;
 
  protected:
+  const uint8_t* m_buf;
+  const uint8_t* m_bufHead;

Review Comment:
   Naming notation for member attributes in Geode Native follows lowerCamelCase with '_' as a postfix, to indicate that it's a member variable. As example 'm_buf' would rather be 'buf_'. Usually whenever we make a change that touches part of the code with the old notation (m_...) the idea is to change it to the right notation



##########
cppcache/src/GetAllServersResponse.hpp:
##########
@@ -42,6 +42,11 @@ class GetAllServersResponse : public internal::DataSerializableFixedId_t<
     return std::make_shared<GetAllServersResponse>();
   }
   GetAllServersResponse() : Serializable() {}
+  explicit GetAllServersResponse(
+      std::vector<std::shared_ptr<ServerLocation> > servers)
+      : Serializable() {
+    m_servers = servers;

Review Comment:
   Same comment regarding member attributes naming conventions:
   
   - Please use lowerCamelCase_ notation.
   - All the member attributes should be declared at the end of the class rather than at the beginning.
   
   Also, I've seen member attributes scope is not explicitly set in this class, by default this means it would be private, but I usually like to explicitly state it. So don't forget to state the scope after chaning the location of the member attributes :)



##########
cppcache/src/GetAllServersResponse.hpp:
##########
@@ -42,6 +42,11 @@ class GetAllServersResponse : public internal::DataSerializableFixedId_t<
     return std::make_shared<GetAllServersResponse>();
   }
   GetAllServersResponse() : Serializable() {}
+  explicit GetAllServersResponse(
+      std::vector<std::shared_ptr<ServerLocation> > servers)
+      : Serializable() {
+    m_servers = servers;

Review Comment:
   This is only used for testing, however if you pass an object copy to this constructor and then make a copy-assign, you'd be copying this twice. Please use std::move :)



##########
cppcache/src/GetAllServersResponse.hpp:
##########
@@ -42,6 +42,11 @@ class GetAllServersResponse : public internal::DataSerializableFixedId_t<
     return std::make_shared<GetAllServersResponse>();
   }
   GetAllServersResponse() : Serializable() {}
+  explicit GetAllServersResponse(
+      std::vector<std::shared_ptr<ServerLocation> > servers)
+      : Serializable() {
+    m_servers = servers;

Review Comment:
   Usually it's preferred, when possible to use constructor inline initialization for member attributes



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;

Review Comment:
   Same comment regarding member attributes naming conventions:
   
   - Please use lowerCamelCase_ notation.
   - Also, when possible, use constructor inline initialization for member attributes.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));
+  }
+}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[kBufferSize];
+  while ((m_bufLength - (m_buf - m_bufHead)) < size) {

Review Comment:
   getBytesRemaining gives you exactly the value of (m_bufLength - (m_buf - m_bufHead)), so considering it provides you with a richer semantical meaning, and that it would probably be inlined by the compiler, I'd use that function instead.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));

Review Comment:
   One feasible solution would be to have an std::vector<uint8_t> as buffer for StreamDataInput, this way you don't need to perform memory managment, that is prone to mem leaks.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871443598


##########
cppcache/include/geode/DataInput.hpp:
##########
@@ -502,7 +502,7 @@ class APACHE_GEODE_EXPORT DataInput {
 
   inline char readPdxChar() { return static_cast<char>(readInt16()); }
 
-  inline void _checkBufferSize(size_t size, int32_t line) {
+  virtual inline void _checkBufferSize(size_t size, int32_t line) {

Review Comment:
   True! I guess the compiler should tell.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872219994


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t kBufferSize = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));
+  }
+}
+
+void StreamDataInput::readDataIfNotAvailable(size_t size) {
+  char buff[kBufferSize];
+  while ((m_bufLength - (m_buf - m_bufHead)) < size) {

Review Comment:
   Indeed! :-)



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] gaussianrecurrence commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872273823


##########
cppcache/src/CacheXmlCreation.cpp:
##########
@@ -50,12 +50,12 @@ void CacheXmlCreation::create(Cache* cache) {
 }
 
 void CacheXmlCreation::setPdxIgnoreUnreadField(bool ignore) {
-  // m_cache->m_cacheImpl->setPdxIgnoreUnreadFields(ignore);
+  // cache_->m_cacheImpl->setPdxIgnoreUnreadFields(ignore);

Review Comment:
   I think you might have change this by mistake?



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -31,65 +31,57 @@ const size_t kBufferSize = 3000;
 StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
                                  std::unique_ptr<Connector> connector,
                                  const CacheImpl* cache, Pool* pool)
-    : DataInput(nullptr, 0, cache, pool) {
-  m_remainingTimeBeforeTimeout = timeout;
-  m_connector = std::move(connector);
-  m_buf = nullptr;
-  m_bufHead = m_buf;
-  m_bufLength = 0;
-}
-
-StreamDataInput::~StreamDataInput() {
-  if (m_bufHead != nullptr) {
-    free(const_cast<uint8_t*>(m_bufHead));
-  }
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout),
+      streamBuf_(0) {
+  buf_ = nullptr;
+  bufHead_ = buf_;
+  bufLength_ = 0;
 }
 
 void StreamDataInput::readDataIfNotAvailable(size_t size) {
   char buff[kBufferSize];
-  while ((m_bufLength - (m_buf - m_bufHead)) < size) {
+  while (getBytesRemaining() < size) {
     const auto start = std::chrono::system_clock::now();
 
-    const auto receivedLength = m_connector->receive_nothrowiftimeout(
+    const auto receivedLength = connector_->receive_nothrowiftimeout(
         buff, kBufferSize,
         std::chrono::duration_cast<std::chrono::milliseconds>(
-            m_remainingTimeBeforeTimeout));
+            remainingTimeBeforeTimeout_));
 
     const auto timeSpent = std::chrono::system_clock::now() - start;
 
-    m_remainingTimeBeforeTimeout -=
-        std::chrono::duration_cast<decltype(m_remainingTimeBeforeTimeout)>(
+    remainingTimeBeforeTimeout_ -=
+        std::chrono::duration_cast<decltype(remainingTimeBeforeTimeout_)>(
             timeSpent);
 
     LOGDEBUG(
         "received %d bytes from %s: %s, time spent: "
         "%ld microsecs, time remaining before timeout: %ld microsecs",
-        receivedLength, m_connector->getRemoteEndpoint().c_str(),
+        receivedLength, connector_->getRemoteEndpoint().c_str(),
         Utils::convertBytesToString(reinterpret_cast<uint8_t*>(buff),
                                     receivedLength)
             .c_str(),
         std::chrono::duration_cast<std::chrono::microseconds>(timeSpent)
             .count(),
         std::chrono::duration_cast<std::chrono::microseconds>(
-            m_remainingTimeBeforeTimeout)
+            remainingTimeBeforeTimeout_)
             .count());
 
-    if (m_remainingTimeBeforeTimeout <= std::chrono::microseconds ::zero()) {
+    if (remainingTimeBeforeTimeout_ <= std::chrono::microseconds ::zero()) {
       throw(TimeoutException(std::string("Timeout when receiving from ")
-                                 .append(m_connector->getRemoteEndpoint())));
+                                 .append(connector_->getRemoteEndpoint())));
     }
 
-    auto newLength = m_bufLength + receivedLength;
-    auto currentPosition = m_buf - m_bufHead;
-    if ((m_bufHead) == nullptr) {
-      m_bufHead = static_cast<uint8_t*>(malloc(sizeof(uint8_t) * newLength));
-    } else {
-      m_bufHead = static_cast<uint8_t*>(
-          realloc(const_cast<uint8_t*>(m_bufHead), newLength));
-    }
-    memcpy(const_cast<uint8_t*>(m_bufHead + m_bufLength), buff, receivedLength);
-    m_buf = m_bufHead + currentPosition;
-    m_bufLength += receivedLength;
+    auto currentPosition = getBytesRead();
+    streamBuf_.resize(bufLength_ + receivedLength);
+    streamBuf_.insert(streamBuf_.begin() + bufLength_, buff,

Review Comment:
   Actually, insert, extends the buffer size, so, even this is working, you'd be allocating more memory than necessary.
   Also, in the case of a buff copy it's always best to use std::memcpy as it's optimized for this purpose by compilers.



##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -31,65 +31,57 @@ const size_t kBufferSize = 3000;
 StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
                                  std::unique_ptr<Connector> connector,
                                  const CacheImpl* cache, Pool* pool)
-    : DataInput(nullptr, 0, cache, pool) {
-  m_remainingTimeBeforeTimeout = timeout;
-  m_connector = std::move(connector);
-  m_buf = nullptr;
-  m_bufHead = m_buf;
-  m_bufLength = 0;
-}
-
-StreamDataInput::~StreamDataInput() {
-  if (m_bufHead != nullptr) {
-    free(const_cast<uint8_t*>(m_bufHead));
-  }
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout),
+      streamBuf_(0) {
+  buf_ = nullptr;

Review Comment:
   There is no need to manually initialize this, as the super constructor already takes care of this



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] gaussianrecurrence commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
gaussianrecurrence commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872148851


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -0,0 +1,98 @@
+/*
+ * 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 "StreamDataInput.hpp"
+
+#include <geode/DataInput.hpp>
+
+#include "Utils.hpp"
+#include "util/Log.hpp"
+
+namespace apache {
+namespace geode {
+namespace client {
+
+const size_t BUFF_SIZE = 3000;
+
+StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
+                                 std::unique_ptr<Connector> connector,
+                                 const CacheImpl* cache, Pool* pool)
+    : DataInput(nullptr, 0, cache, pool) {
+  m_remainingTimeBeforeTimeout = timeout;
+  m_connector = std::move(connector);
+  m_buf = nullptr;
+  m_bufHead = m_buf;
+  m_bufLength = 0;
+}
+
+StreamDataInput::~StreamDataInput() {
+  if (m_bufHead != nullptr) {
+    free(const_cast<uint8_t*>(m_bufHead));

Review Comment:
   One feasible solution would be to have an std::vector<int8_t> as buffer for StreamDataInput, this way you don't need to perform memory managment, that is prone to mem leaks.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r872356339


##########
cppcache/src/StreamDataInput.cpp:
##########
@@ -31,65 +31,57 @@ const size_t kBufferSize = 3000;
 StreamDataInput::StreamDataInput(std::chrono::milliseconds timeout,
                                  std::unique_ptr<Connector> connector,
                                  const CacheImpl* cache, Pool* pool)
-    : DataInput(nullptr, 0, cache, pool) {
-  m_remainingTimeBeforeTimeout = timeout;
-  m_connector = std::move(connector);
-  m_buf = nullptr;
-  m_bufHead = m_buf;
-  m_bufLength = 0;
-}
-
-StreamDataInput::~StreamDataInput() {
-  if (m_bufHead != nullptr) {
-    free(const_cast<uint8_t*>(m_bufHead));
-  }
+    : DataInput(nullptr, 0, cache, pool),
+      connector_(std::move(connector)),
+      remainingTimeBeforeTimeout_(timeout),
+      streamBuf_(0) {
+  buf_ = nullptr;
+  bufHead_ = buf_;
+  bufLength_ = 0;
 }
 
 void StreamDataInput::readDataIfNotAvailable(size_t size) {
   char buff[kBufferSize];
-  while ((m_bufLength - (m_buf - m_bufHead)) < size) {
+  while (getBytesRemaining() < size) {
     const auto start = std::chrono::system_clock::now();
 
-    const auto receivedLength = m_connector->receive_nothrowiftimeout(
+    const auto receivedLength = connector_->receive_nothrowiftimeout(
         buff, kBufferSize,
         std::chrono::duration_cast<std::chrono::milliseconds>(
-            m_remainingTimeBeforeTimeout));
+            remainingTimeBeforeTimeout_));
 
     const auto timeSpent = std::chrono::system_clock::now() - start;
 
-    m_remainingTimeBeforeTimeout -=
-        std::chrono::duration_cast<decltype(m_remainingTimeBeforeTimeout)>(
+    remainingTimeBeforeTimeout_ -=
+        std::chrono::duration_cast<decltype(remainingTimeBeforeTimeout_)>(
             timeSpent);
 
     LOGDEBUG(
         "received %d bytes from %s: %s, time spent: "
         "%ld microsecs, time remaining before timeout: %ld microsecs",
-        receivedLength, m_connector->getRemoteEndpoint().c_str(),
+        receivedLength, connector_->getRemoteEndpoint().c_str(),
         Utils::convertBytesToString(reinterpret_cast<uint8_t*>(buff),
                                     receivedLength)
             .c_str(),
         std::chrono::duration_cast<std::chrono::microseconds>(timeSpent)
             .count(),
         std::chrono::duration_cast<std::chrono::microseconds>(
-            m_remainingTimeBeforeTimeout)
+            remainingTimeBeforeTimeout_)
             .count());
 
-    if (m_remainingTimeBeforeTimeout <= std::chrono::microseconds ::zero()) {
+    if (remainingTimeBeforeTimeout_ <= std::chrono::microseconds ::zero()) {
       throw(TimeoutException(std::string("Timeout when receiving from ")
-                                 .append(m_connector->getRemoteEndpoint())));
+                                 .append(connector_->getRemoteEndpoint())));
     }
 
-    auto newLength = m_bufLength + receivedLength;
-    auto currentPosition = m_buf - m_bufHead;
-    if ((m_bufHead) == nullptr) {
-      m_bufHead = static_cast<uint8_t*>(malloc(sizeof(uint8_t) * newLength));
-    } else {
-      m_bufHead = static_cast<uint8_t*>(
-          realloc(const_cast<uint8_t*>(m_bufHead), newLength));
-    }
-    memcpy(const_cast<uint8_t*>(m_bufHead + m_bufLength), buff, receivedLength);
-    m_buf = m_bufHead + currentPosition;
-    m_bufLength += receivedLength;
+    auto currentPosition = getBytesRead();
+    streamBuf_.resize(bufLength_ + receivedLength);
+    streamBuf_.insert(streamBuf_.begin() + bufLength_, buff,

Review Comment:
   You are absolutely right. Thanks



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r871477559


##########
cppcache/src/StreamDataInput.hpp:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_STREAMDATAINPUT_H_
+#define GEODE_STREAMDATAINPUT_H_
+
+#include <chrono>
+
+#include "Connector.hpp"
+#include "geode/DataInput.hpp"
+
+/**

Review Comment:
   You mean all the comments? I could just say that it provides the same functionality as its parent class but it does not read data from a static buffer passed. Instead it reads the data on demand from a socket.



-- 
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: notifications-unsubscribe@geode.apache.org

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


[GitHub] [geode-native] albertogpz commented on a diff in pull request #970: GEODE-10300: C++ native client: Allow locator responses greater than …

Posted by GitBox <gi...@apache.org>.
albertogpz commented on code in PR #970:
URL: https://github.com/apache/geode-native/pull/970#discussion_r876196927


##########
cppcache/src/StreamDataInput.hpp:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#ifndef GEODE_STREAMDATAINPUT_H_
+#define GEODE_STREAMDATAINPUT_H_
+
+#include <chrono>
+
+#include "Connector.hpp"
+#include "geode/DataInput.hpp"
+
+/**
+ * @file
+ */
+
+namespace apache {
+namespace geode {
+namespace client {
+
+#include "geode/DataInput.hpp"

Review Comment:
   @pivotal-jbarrett Anything else to change?



-- 
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: notifications-unsubscribe@geode.apache.org

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