Skip to content

Commit e66f892

Browse files
committed
fix(security): Upgrade pinot to version 1.40 to address CVE-2025-12383(GHSA-7p63-w6x9-6gr7)
1 parent 66bb3ec commit e66f892

File tree

4 files changed

+21
-20
lines changed

4 files changed

+21
-20
lines changed

pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -66,7 +66,7 @@
6666
<dep.alluxio.version>313</dep.alluxio.version>
6767
<dep.slf4j.version>2.0.16</dep.slf4j.version>
6868
<dep.kafka.version>3.9.1</dep.kafka.version>
69-
<dep.pinot.version>1.3.0</dep.pinot.version>
69+
<dep.pinot.version>1.4.0</dep.pinot.version>
7070
<dep.druid.version>30.0.1</dep.druid.version>
7171
<dep.jaxb.version>2.3.1</dep.jaxb.version>
7272
<dep.jaxb.runtime.version>4.0.5</dep.jaxb.runtime.version>

presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/PinotStreamingQueryClient.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -15,8 +15,8 @@
1515

1616
import org.apache.pinot.common.config.GrpcConfig;
1717
import org.apache.pinot.common.proto.Server;
18-
import org.apache.pinot.common.utils.grpc.GrpcQueryClient;
19-
import org.apache.pinot.common.utils.grpc.GrpcRequestBuilder;
18+
import org.apache.pinot.common.utils.grpc.ServerGrpcQueryClient;
19+
import org.apache.pinot.common.utils.grpc.ServerGrpcRequestBuilder;
2020

2121
import java.util.HashMap;
2222
import java.util.Iterator;
@@ -30,25 +30,25 @@
3030
*/
3131
public class PinotStreamingQueryClient
3232
{
33-
private final Map<String, GrpcQueryClient> grpcQueryClientMap = new HashMap<>();
33+
private final Map<String, ServerGrpcQueryClient> grpcQueryClientMap = new HashMap<>();
3434
private final GrpcConfig config;
3535

3636
public PinotStreamingQueryClient(GrpcConfig config)
3737
{
3838
this.config = config;
3939
}
4040

41-
public Iterator<Server.ServerResponse> submit(String host, int port, GrpcRequestBuilder requestBuilder)
41+
public Iterator<Server.ServerResponse> submit(String host, int port, ServerGrpcRequestBuilder requestBuilder)
4242
{
43-
GrpcQueryClient client = getOrCreateGrpcQueryClient(host, port);
43+
ServerGrpcQueryClient client = getOrCreateGrpcQueryClient(host, port);
4444
return client.submit(requestBuilder.build());
4545
}
4646

47-
private GrpcQueryClient getOrCreateGrpcQueryClient(String host, int port)
47+
private ServerGrpcQueryClient getOrCreateGrpcQueryClient(String host, int port)
4848
{
4949
String key = String.format("%s_%d", host, port);
5050
if (!grpcQueryClientMap.containsKey(key)) {
51-
grpcQueryClientMap.put(key, new GrpcQueryClient(host, port, config));
51+
grpcQueryClientMap.put(key, new ServerGrpcQueryClient(host, port, config));
5252
}
5353
return grpcQueryClientMap.get(key);
5454
}

presto-pinot-toolkit/src/main/java/com/facebook/presto/pinot/query/PinotProxyGrpcRequestBuilder.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616
import com.facebook.presto.pinot.PinotErrorCode;
1717
import com.facebook.presto.pinot.PinotException;
1818
import org.apache.pinot.common.proto.Server;
19-
import org.apache.pinot.common.utils.grpc.GrpcRequestBuilder;
19+
import org.apache.pinot.common.utils.grpc.ServerGrpcRequestBuilder;
2020
import org.apache.pinot.spi.utils.CommonConstants;
2121

2222
import java.util.HashMap;
@@ -25,7 +25,7 @@
2525
import java.util.Optional;
2626

2727
public class PinotProxyGrpcRequestBuilder
28-
extends GrpcRequestBuilder
28+
extends ServerGrpcRequestBuilder
2929
{
3030
private static final String KEY_OF_PROXY_GRPC_FORWARD_HOST = "FORWARD_HOST";
3131
private static final String KEY_OF_PROXY_GRPC_FORWARD_PORT = "FORWARD_PORT";

presto-pinot-toolkit/src/test/java/com/facebook/presto/pinot/TestPinotSegmentPageSource.java

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -34,11 +34,12 @@
3434
import org.apache.pinot.common.datatable.DataTable;
3535
import org.apache.pinot.common.proto.Server;
3636
import org.apache.pinot.common.utils.DataSchema;
37-
import org.apache.pinot.common.utils.grpc.GrpcRequestBuilder;
37+
import org.apache.pinot.common.utils.grpc.ServerGrpcRequestBuilder;
3838
import org.apache.pinot.core.common.datatable.DataTableBuilder;
3939
import org.apache.pinot.core.common.datatable.DataTableBuilderV4;
4040
import org.apache.pinot.spi.data.DimensionFieldSpec;
4141
import org.apache.pinot.spi.data.FieldSpec;
42+
import org.apache.pinot.spi.utils.ByteArray;
4243
import org.apache.pinot.spi.utils.CommonConstants;
4344
import org.testng.annotations.Test;
4445

@@ -172,9 +173,6 @@ protected static DataTable createDataTableWithAllTypes()
172173
case STRING:
173174
dataTableBuilder.setColumn(colId, generateRandomStringWithLength(RANDOM.nextInt(20)));
174175
break;
175-
case OBJECT:
176-
dataTableBuilder.setColumn(colId, (Object) RANDOM.nextDouble());
177-
break;
178176
case BOOLEAN_ARRAY:
179177
int length = RANDOM.nextInt(20);
180178
int[] booleanArray = new int[length];
@@ -233,7 +231,7 @@ protected static DataTable createDataTableWithAllTypes()
233231
case BYTES:
234232
try {
235233
dataTableBuilder.setColumn(colId,
236-
Hex.decodeHex("0DE0B6B3A7640000".toCharArray())); // Hex of BigDecimal.ONE
234+
new ByteArray(Hex.decodeHex("0DE0B6B3A7640000".toCharArray()))); // Hex of BigDecimal.ONE
237235
}
238236
catch (DecoderException e) {
239237
throw new RuntimeException(e);
@@ -390,7 +388,6 @@ public void testPinotProxyGrpcRequest()
390388
Assert.assertEquals(grpcRequest.getSql(), "SELECT * FROM myTable");
391389
Assert.assertEquals(grpcRequest.getSegmentsCount(), 1);
392390
Assert.assertEquals(grpcRequest.getSegments(0), "segment1");
393-
Assert.assertEquals(grpcRequest.getMetadataCount(), 9);
394391
Assert.assertEquals(grpcRequest.getMetadataOrThrow("k1"), "v1");
395392
Assert.assertEquals(grpcRequest.getMetadataOrThrow("k2"), "v2");
396393
Assert.assertEquals(grpcRequest.getMetadataOrThrow("FORWARD_HOST"), "localhost");
@@ -401,6 +398,7 @@ public void testPinotProxyGrpcRequest()
401398
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.ENABLE_STREAMING), "true");
402399
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.PAYLOAD_TYPE), "sql");
403400

401+
Assert.assertTrue(grpcRequest.getMetadataCount() >= 9, "Expected at least 9 metadata entries, but got " + grpcRequest.getMetadataCount());
404402
grpcRequest = new PinotProxyGrpcRequestBuilder()
405403
.setSegments(ImmutableList.of("segment1"))
406404
.setEnableStreaming(true)
@@ -412,20 +410,23 @@ public void testPinotProxyGrpcRequest()
412410
Assert.assertEquals(grpcRequest.getSql(), "SELECT * FROM myTable");
413411
Assert.assertEquals(grpcRequest.getSegmentsCount(), 1);
414412
Assert.assertEquals(grpcRequest.getSegments(0), "segment1");
415-
Assert.assertEquals(grpcRequest.getMetadataCount(), 7);
416413
Assert.assertEquals(grpcRequest.getMetadataOrThrow("k1"), "v1");
417414
Assert.assertEquals(grpcRequest.getMetadataOrThrow("k2"), "v2");
418415
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID), "121");
419416
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.BROKER_ID), "presto-coordinator-grpc");
420417
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.ENABLE_TRACE), "false");
421418
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.ENABLE_STREAMING), "true");
422419
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.PAYLOAD_TYPE), "sql");
420+
421+
// Verifying minimum metadata count (7 fields: k1, k2, requestId, brokerId, enableTrace, enableStreaming, payloadType)
422+
Assert.assertTrue(grpcRequest.getMetadataCount() >= 7,
423+
"Expected at least 7 metadata entries, but got " + grpcRequest.getMetadataCount());
423424
}
424425

425426
@Test
426427
public void testPinotGrpcRequest()
427428
{
428-
final Server.ServerRequest grpcRequest = new GrpcRequestBuilder()
429+
final Server.ServerRequest grpcRequest = new ServerGrpcRequestBuilder()
429430
.setSegments(ImmutableList.of("segment1"))
430431
.setEnableStreaming(true)
431432
.setRequestId(121)
@@ -435,7 +436,7 @@ public void testPinotGrpcRequest()
435436
Assert.assertEquals(grpcRequest.getSql(), "SELECT * FROM myTable");
436437
Assert.assertEquals(grpcRequest.getSegmentsCount(), 1);
437438
Assert.assertEquals(grpcRequest.getSegments(0), "segment1");
438-
Assert.assertEquals(grpcRequest.getMetadataCount(), 5);
439+
Assert.assertEquals(grpcRequest.getMetadataCount(), 6);
439440
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID), "121");
440441
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.BROKER_ID), "presto-coordinator-grpc");
441442
Assert.assertEquals(grpcRequest.getMetadataOrThrow(CommonConstants.Query.Request.MetadataKeys.ENABLE_TRACE), "false");
@@ -455,7 +456,7 @@ private static final class TestingPinotStreamingQueryClient
455456
}
456457

457458
@Override
458-
public Iterator<Server.ServerResponse> submit(String host, int port, GrpcRequestBuilder requestBuilder)
459+
public Iterator<Server.ServerResponse> submit(String host, int port, ServerGrpcRequestBuilder requestBuilder)
459460
{
460461
return new Iterator<Server.ServerResponse>()
461462
{

0 commit comments

Comments
 (0)