You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "Jackie-Jiang (via GitHub)" <gi...@apache.org> on 2023/05/30 17:39:49 UTC

[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #10528: PinotBufferFactory and a buffer implementation that uses Unsafe

Jackie-Jiang commented on code in PR #10528:
URL: https://github.com/apache/pinot/pull/10528#discussion_r1210586980


##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiNodesOfflineClusterIntegrationTest.java:
##########
@@ -138,28 +140,28 @@ public void testServerHardFailure()
 
     // Take a server and shut down its query server to mimic a hard failure
     BaseServerStarter serverStarter = _serverStarters.get(NUM_SERVERS - 1);
-    serverStarter.getServerInstance().shutDown();
-
-    // First query should hit all servers and get connection refused exception
-    testCountStarQuery(NUM_SERVERS, true);
-
-    // Second query should not hit the failed server, and should return the correct result
-    testCountStarQuery(NUM_SERVERS - 1, false);
-
-    // Restart the failed server, and it should be included in the routing again
-    serverStarter.stop();
-    serverStarter = startOneServer(NUM_SERVERS - 1);
-    _serverStarters.set(NUM_SERVERS - 1, serverStarter);
-    TestUtils.waitForCondition(aVoid -> {
-      try {
+    try {
+      serverStarter.getServerInstance().shutDown();
+
+      // First query should hit all servers and get connection refused or reset exception
+      // TODO: This is a flaky test. There is a race condition between shutDown and the query being executed.

Review Comment:
   Let's create an issue to fix this test



##########
pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MultiNodesOfflineClusterIntegrationTest.java:
##########
@@ -172,7 +174,6 @@ private void testCountStarQuery(int expectedNumServersQueried, boolean exception
       assertEquals(exceptions.size(), 2);
       JsonNode firstException = exceptions.get(0);
       assertEquals(firstException.get("errorCode").intValue(), QueryException.BROKER_REQUEST_SEND_ERROR_CODE);
-      assertTrue(firstException.get("message").textValue().contains("Connection refused"));

Review Comment:
   I feel we should't remove this check. Are you removing it in order to pass the test?



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/unsafe/UnsafePinotBufferFactory.java:
##########
@@ -0,0 +1,57 @@
+/**
+ * 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.
+ */
+package org.apache.pinot.segment.spi.memory.unsafe;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteOrder;
+import org.apache.pinot.segment.spi.memory.NonNativePinotDataBuffer;
+import org.apache.pinot.segment.spi.memory.PinotBufferFactory;
+import org.apache.pinot.segment.spi.memory.PinotDataBuffer;
+import xerial.larray.impl.OSInfo;
+
+
+public class UnsafePinotBufferFactory implements PinotBufferFactory {
+  public UnsafePinotBufferFactory() {
+    if (OSInfo.isWindows()) {

Review Comment:
   Does this mean WINDOWS with java 17 won't work?



##########
.github/workflows/pinot_tests.yml:
##########
@@ -281,4 +273,4 @@ jobs:
         env:
           MAVEN_OPTS: -Xmx2G -DfailIfNoTests=false -Dmaven.wagon.httpconnectionManager.ttlSeconds=25 -Dmaven.wagon.http.retryHandler.count=30 -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false
         run: |
-          mvn clean install -DskipTests -Ppresto-driver -am -B -pl ':presto-pinot-driver' -Djdk.version=8 -T 16 || exit 1
+          mvn clean install -Dmaven.test.skip=true -Ppresto-driver -am -B -pl ':presto-pinot-driver' -Djdk.version=8 -T 16 || exit 1

Review Comment:
   Is this change required? Or both ways work?



##########
pinot-segment-spi/src/main/java/org/apache/pinot/segment/spi/memory/PinotBufferFactory.java:
##########
@@ -0,0 +1,40 @@
+/**
+ * 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.
+ */
+package org.apache.pinot.segment.spi.memory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteOrder;
+
+
+public interface PinotBufferFactory {
+
+  PinotDataBuffer allocateDirect(long size, ByteOrder byteOrder);
+
+  default PinotDataBuffer readFile(File file, long offset, long size, ByteOrder byteOrder)
+      throws IOException {
+    // TODO: Shouldn't we allocate size - offset instead of size?

Review Comment:
   This `size` is not the file size, but the size to read, so we should allocate `size` instead of `size - offset`



-- 
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: commits-unsubscribe@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org