You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ap...@apache.org on 2021/01/05 22:01:37 UTC

[incubator-pinot] branch pinot-broker-https-discussion created (now 1460b0a)

This is an automated email from the ASF dual-hosted git repository.

apucher pushed a change to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git.


      at 1460b0a  quickstart broker tls properties

This branch includes the following new commits:

     new 40096f2  cert scripts
     new 69e2d81  remove junit imports
     new 1304b16  use unique output file names in csv mode
     new 5787625  generate localhost cert
     new 9e53803  hard-code quickstart properties
     new ba6eac2  client-broker TLS
     new 1460b0a  quickstart broker tls properties

The 7 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[incubator-pinot] 07/07: quickstart broker tls properties

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 1460b0acb0357a031458deb11d6bc62f5c884f7c
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Tue Jan 5 13:55:16 2021 -0800

    quickstart broker tls properties
---
 .../org/apache/pinot/tools/admin/command/QuickstartRunner.java   | 9 ++++++++-
 1 file changed, 8 insertions(+), 1 deletion(-)

diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
index 0ec8ce7..9225533 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
@@ -171,6 +171,13 @@ public class QuickstartRunner {
       Properties prop = new Properties();
       prop.put("pinot.broker.client.queryPort", String.valueOf(DEFAULT_BROKER_PORT + i));
 
+      prop.put("pinot.broker.client.protocol", "https");
+      prop.put("pinot.broker.client.tls.keystore.path", "/Users/alex/projects/incubator-pinot/truststore/generated.keystore.jks");
+      prop.put("pinot.broker.client.tls.keystore.password", "changeit");
+      prop.put("pinot.broker.client.tls.truststore.path", "/Users/alex/projects/incubator-pinot/truststore/generated.truststore.jks");
+      prop.put("pinot.broker.client.tls.truststore.password", "changeit");
+      prop.put("pinot.broker.client.tls.requires_client_auth", "false");
+
       try (OutputStream os = new FileOutputStream(configFileName)) {
         prop.store(os, "");
       }
@@ -310,7 +317,7 @@ public class QuickstartRunner {
       throws Exception {
     int brokerPort = _brokerPorts.get(RANDOM.nextInt(_brokerPorts.size()));
     return JsonUtils.stringToJsonNode(new PostQueryCommand().setBrokerPort(String.valueOf(brokerPort))
-        .setQueryType(CommonConstants.Broker.Request.SQL).setQuery(query).run());
+        .setBrokerProtocol("https").setQueryType(CommonConstants.Broker.Request.SQL).setQuery(query).run());
   }
 
   public static void registerDefaultPinotFS() {


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


[incubator-pinot] 05/07: hard-code quickstart properties

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 9e538030e48eb4a8fd824b6148b594cd5fef6280
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Tue Jan 5 11:01:28 2021 -0800

    hard-code quickstart properties
---
 .../tools/admin/command/QuickstartRunner.java      | 67 +++++++++++++++++++++-
 1 file changed, 64 insertions(+), 3 deletions(-)

diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
index 4598d8a..0ec8ce7 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/QuickstartRunner.java
@@ -22,17 +22,21 @@ import com.fasterxml.jackson.databind.JsonNode;
 import com.google.common.collect.ImmutableMap;
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.io.Reader;
 import java.net.InetAddress;
 import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import org.apache.commons.io.FileUtils;
 import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.common.utils.NetUtil;
 import org.apache.pinot.spi.config.table.TableType;
 import org.apache.pinot.spi.config.tenant.TenantRole;
 import org.apache.pinot.spi.env.PinotConfiguration;
@@ -47,6 +51,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 
+import static org.apache.pinot.tools.utils.PinotConfigUtils.getAvailablePort;
+
 
 public class QuickstartRunner {
   private static final Logger LOGGER = LoggerFactory.getLogger(QuickstartRunner.class.getName());
@@ -117,10 +123,41 @@ public class QuickstartRunner {
   private void startControllers()
       throws Exception {
     for (int i = 0; i < _numControllers; i++) {
+      String configFileName = String.format("./controller_%d.properties", i);
+
+      Properties prop = new Properties();
+      prop.put("cluster.tenant.isolation.enable", "true");
+      prop.put("controller.mode", "DUAL");
+      prop.put("controller.data.dir", new File(_tempDir, DEFAULT_CONTROLLER_DIR + i).getAbsolutePath());
+      prop.put("controller.retention.frequencyInSeconds", "21600");
+      prop.put("controller.port", String.valueOf(DEFAULT_CONTROLLER_PORT + i));
+      prop.put("controller.host", NetUtil.getHostAddress());
+      prop.put("controller.offline.segment.interval.checker.frequencyInSeconds", "3600");
+      prop.put("controller.realtime.segment.validation.frequencyInSeconds", "3600");
+      prop.put("controller.vip.host", NetUtil.getHostAddress());
+      prop.put("controller.broker.resource.validation.frequencyInSeconds", "3600");
+      prop.put("controller.helix.cluster.name", "QuickStartCluster");
+      prop.put("controller.zk.str", ZK_ADDRESS);
+
+      prop.put("controller.access.protocols", "https");
+      prop.put("controller.access.protocols.https.host", "0.0.0.0");
+      prop.put("controller.access.protocols.https.port", String.valueOf(9443 + i));
+      prop.put("controller.access.protocols.https.vip", "false");
+      prop.put("controller.access.protocols.https.tls.keystore.path", "/Users/alex/projects/incubator-pinot/truststore/generated.keystore.jks");
+      prop.put("controller.access.protocols.https.tls.keystore.password", "changeit");
+      prop.put("controller.access.protocols.https.tls.truststore.path", "/Users/alex/projects/incubator-pinot/truststore/generated.truststore.jks");
+      prop.put("controller.access.protocols.https.tls.truststore.password", "changeit");
+      prop.put("controller.access.protocols.https.tls.requires_client_auth", "false");
+
+      try (OutputStream os = new FileOutputStream(configFileName)) {
+        prop.store(os, "");
+      }
+
       StartControllerCommand controllerStarter = new StartControllerCommand();
       controllerStarter.setControllerPort(String.valueOf(DEFAULT_CONTROLLER_PORT + i)).setZkAddress(ZK_ADDRESS)
           .setClusterName(CLUSTER_NAME).setTenantIsolation(_enableTenantIsolation)
-          .setDataDir(new File(_tempDir, DEFAULT_CONTROLLER_DIR + i).getAbsolutePath());
+          .setDataDir(new File(_tempDir, DEFAULT_CONTROLLER_DIR + i).getAbsolutePath())
+          .setConfigFileName(configFileName);
       controllerStarter.execute();
       _controllerPorts.add(DEFAULT_CONTROLLER_PORT + i);
     }
@@ -129,8 +166,18 @@ public class QuickstartRunner {
   private void startBrokers()
       throws Exception {
     for (int i = 0; i < _numBrokers; i++) {
+      String configFileName = String.format("./broker_%d.properties", i);
+
+      Properties prop = new Properties();
+      prop.put("pinot.broker.client.queryPort", String.valueOf(DEFAULT_BROKER_PORT + i));
+
+      try (OutputStream os = new FileOutputStream(configFileName)) {
+        prop.store(os, "");
+      }
+
       StartBrokerCommand brokerStarter = new StartBrokerCommand();
-      brokerStarter.setPort(DEFAULT_BROKER_PORT + i).setZkAddress(ZK_ADDRESS).setClusterName(CLUSTER_NAME);
+      brokerStarter.setPort(DEFAULT_BROKER_PORT + i).setZkAddress(ZK_ADDRESS).setClusterName(CLUSTER_NAME)
+          .setConfigFileName(configFileName);
       brokerStarter.execute();
       _brokerPorts.add(DEFAULT_BROKER_PORT + i);
     }
@@ -139,11 +186,25 @@ public class QuickstartRunner {
   private void startServers()
       throws Exception {
     for (int i = 0; i < _numServers; i++) {
+      String configFileName = String.format("./server_%d.properties", i);
+
+      Properties prop = new Properties();
+      prop.put("pinot.server.netty.host", NetUtil.getHostAddress());
+      prop.put("pinot.server.instance.dataDir", new File(_tempDir, DEFAULT_SERVER_DATA_DIR + i).getAbsolutePath());
+      prop.put("pinot.server.instance.segmentTarDir", new File(_tempDir, DEFAULT_SERVER_SEGMENT_DIR + i).getAbsolutePath());
+      prop.put("pinot.server.netty.port", String.valueOf(DEFAULT_SERVER_NETTY_PORT + i));
+      prop.put("pinot.server.adminapi.port", String.valueOf(DEFAULT_SERVER_ADMIN_API_PORT + i));
+
+      try (OutputStream os = new FileOutputStream(configFileName)) {
+        prop.store(os, "");
+      }
+
       StartServerCommand serverStarter = new StartServerCommand();
       serverStarter.setPort(DEFAULT_SERVER_NETTY_PORT + i).setAdminPort(DEFAULT_SERVER_ADMIN_API_PORT + i)
           .setZkAddress(ZK_ADDRESS).setClusterName(CLUSTER_NAME)
           .setDataDir(new File(_tempDir, DEFAULT_SERVER_DATA_DIR + i).getAbsolutePath())
-          .setSegmentDir(new File(_tempDir, DEFAULT_SERVER_SEGMENT_DIR + i).getAbsolutePath());
+          .setSegmentDir(new File(_tempDir, DEFAULT_SERVER_SEGMENT_DIR + i).getAbsolutePath())
+          .setConfigFileName(configFileName);
       serverStarter.execute();
     }
   }


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


[incubator-pinot] 02/07: remove junit imports

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 69e2d81fc93eed2b1fc429cff6f2a336135cf0fe
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Mon Jan 4 15:05:08 2021 -0800

    remove junit imports
---
 .../org/apache/pinot/broker/pruner/SegmentZKMetadataPrunerTest.java   | 2 +-
 .../org/apache/pinot/controller/api/SegmentCompletionUtilsTest.java   | 2 +-
 .../pinot/controller/helix/core/util/SegmentDeletionManagerTest.java  | 3 +--
 .../src/test/java/org/apache/pinot/core/plan/CombinePlanNodeTest.java | 2 +-
 .../apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java  | 4 ++--
 5 files changed, 6 insertions(+), 7 deletions(-)

diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/pruner/SegmentZKMetadataPrunerTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/pruner/SegmentZKMetadataPrunerTest.java
index e345033..29decea 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/pruner/SegmentZKMetadataPrunerTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/pruner/SegmentZKMetadataPrunerTest.java
@@ -21,13 +21,13 @@ package org.apache.pinot.broker.pruner;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
-import junit.framework.Assert;
 import org.apache.pinot.common.metadata.segment.ColumnPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentPartitionMetadata;
 import org.apache.pinot.common.metadata.segment.SegmentZKMetadata;
 import org.apache.pinot.common.request.BrokerRequest;
 import org.apache.pinot.pql.parsers.Pql2Compiler;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/api/SegmentCompletionUtilsTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/api/SegmentCompletionUtilsTest.java
index 6e651cc..11cb8ba 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/api/SegmentCompletionUtilsTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/api/SegmentCompletionUtilsTest.java
@@ -18,8 +18,8 @@
  */
 package org.apache.pinot.controller.api;
 
-import junit.framework.Assert;
 import org.apache.pinot.controller.util.SegmentCompletionUtils;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
diff --git a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/util/SegmentDeletionManagerTest.java b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/util/SegmentDeletionManagerTest.java
index 5fa2517..e85a9b7 100644
--- a/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/util/SegmentDeletionManagerTest.java
+++ b/pinot-controller/src/test/java/org/apache/pinot/controller/helix/core/util/SegmentDeletionManagerTest.java
@@ -46,12 +46,11 @@ import org.apache.pinot.spi.filesystem.PinotFSFactory;
 import org.joda.time.DateTime;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 import com.google.common.io.Files;
 
-import junit.framework.Assert;
-
 
 public class SegmentDeletionManagerTest {
   final static String tableName = "table";
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/plan/CombinePlanNodeTest.java b/pinot-core/src/test/java/org/apache/pinot/core/plan/CombinePlanNodeTest.java
index 2cf4758..06cd6b3 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/plan/CombinePlanNodeTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/plan/CombinePlanNodeTest.java
@@ -26,11 +26,11 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
-import junit.framework.Assert;
 import org.apache.pinot.common.utils.CommonConstants.Server;
 import org.apache.pinot.core.plan.maker.InstancePlanMakerImplV2;
 import org.apache.pinot.core.query.request.context.QueryContext;
 import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUtils;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java
index 85cebb1..cd2a159 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/query/pruner/ColumnValueSegmentPrunerTest.java
@@ -28,8 +28,8 @@ import org.apache.pinot.core.query.request.context.utils.QueryContextConverterUt
 import org.apache.pinot.spi.data.FieldSpec.DataType;
 import org.testng.annotations.Test;
 
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertTrue;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 


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


[incubator-pinot] 04/07: generate localhost cert

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 5787625712b4cfd75372b38228cd79b8c89292b9
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Tue Jan 5 11:00:57 2021 -0800

    generate localhost cert
---
 create-certs.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/create-certs.sh b/create-certs.sh
index 2161e22..1cc8a86 100755
--- a/create-certs.sh
+++ b/create-certs.sh
@@ -18,7 +18,7 @@
 #
 
 DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
-DOMAIN="192.168.64.82.xip.io"
+DOMAIN="localhost"
 
 KEY_DIR="$DIR/truststore"
 
@@ -32,10 +32,10 @@ openssl req -x509 -newkey rsa:4096 -days 365 -nodes -keyout "$KEY_DIR/ca-key.pem
 #openssl x509 -in "$KEY_DIR/ca-cert.pem" -noout -text
 
 echo "2. Generate web server's private key and certificate signing request (CSR)"
-openssl req -newkey rsa:4096 -nodes -keyout "$KEY_DIR/key.pem" -out "$KEY_DIR/req.pem" -subj "/C=US/ST=Someplace/L=Somewhere/O=Apache Pinot/OU=Education/CN=*.$DOMAIN/emailAddress=admin@example.com"
+openssl req -newkey rsa:4096 -nodes -keyout "$KEY_DIR/key.pem" -out "$KEY_DIR/req.pem" -subj "/C=US/ST=Someplace/L=Somewhere/O=Apache Pinot/OU=Education/CN=$DOMAIN/emailAddress=admin@example.com"
 
 echo "3. Use CA's private key to sign web server's CSR and get back the signed certificate"
-echo "subjectAltName=DNS:*.$DOMAIN,IP:0.0.0.0" > "$KEY_DIR/ext.cnf"
+echo "subjectAltName=DNS:$DOMAIN,IP:0.0.0.0" > "$KEY_DIR/ext.cnf"
 openssl x509 -req -in "$KEY_DIR/req.pem" -days 60 -CA "$KEY_DIR/ca-cert.pem" -CAkey "$KEY_DIR/ca-key.pem" -CAcreateserial -out "$KEY_DIR/cert.pem" -extfile "$KEY_DIR/ext.cnf"
 
 #echo "Server's signed certificate"


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


[incubator-pinot] 03/07: use unique output file names in csv mode

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 1304b16d32c9a3ba972e4c9a05cb758485db4f25
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Mon Jan 4 17:32:05 2021 -0800

    use unique output file names in csv mode
---
 .../main/java/org/apache/pinot/tools/data/generator/DataGenerator.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
index 2cdc1cc..bd7a489 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/data/generator/DataGenerator.java
@@ -110,7 +110,7 @@ public class DataGenerator {
       throws IOException {
     final int numPerFiles = (int) (totalDocs / numFiles);
     for (int i = 0; i < numFiles; i++) {
-      try (FileWriter writer = new FileWriter(outDir + "/output.csv")) {
+      try (FileWriter writer = new FileWriter(String.format("%s/output_%d.csv", outDir, i))) {
         writer.append(StringUtils.join(genSpec.getColumns(), ",")).append('\n');
         for (int j = 0; j < numPerFiles; j++) {
           Object[] values = new Object[genSpec.getColumns().size()];


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


[incubator-pinot] 06/07: client-broker TLS

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit ba6eac20ea9350ee1cd829270695d10f16315cc2
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Tue Jan 5 13:54:52 2021 -0800

    client-broker TLS
---
 .../broker/broker/BrokerAdminApiApplication.java   | 51 ++++++++++++++++++++--
 .../broker/broker/helix/HelixBrokerStarter.java    |  2 +-
 .../api/resources/PinotQueryResource.java          | 14 +++---
 .../tools/admin/command/PostQueryCommand.java      | 15 +++++--
 4 files changed, 68 insertions(+), 14 deletions(-)

diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerAdminApiApplication.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerAdminApiApplication.java
index ba262fd..f6c25ac 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerAdminApiApplication.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/BrokerAdminApiApplication.java
@@ -27,9 +27,12 @@ import org.apache.pinot.broker.requesthandler.BrokerRequestHandler;
 import org.apache.pinot.broker.routing.RoutingManager;
 import org.apache.pinot.common.metrics.BrokerMetrics;
 import org.apache.pinot.common.utils.CommonConstants;
+import org.apache.pinot.spi.env.PinotConfiguration;
 import org.glassfish.grizzly.http.server.CLStaticHttpHandler;
 import org.glassfish.grizzly.http.server.HttpHandler;
 import org.glassfish.grizzly.http.server.HttpServer;
+import org.glassfish.grizzly.ssl.SSLContextConfigurator;
+import org.glassfish.grizzly.ssl.SSLEngineConfigurator;
 import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.grizzly2.httpserver.GrizzlyHttpServerFactory;
 import org.glassfish.jersey.jackson.JacksonFeature;
@@ -39,6 +42,17 @@ import org.glassfish.jersey.server.ResourceConfig;
 public class BrokerAdminApiApplication extends ResourceConfig {
   private static final String RESOURCE_PACKAGE = "org.apache.pinot.broker.api.resources";
 
+  // TODO find a permanent home for broker configuration keys
+  private static final String PINOT_BROKER_CLIENT_PROTOCOL = "pinot.broker.client.protocol";
+  private static final String PINOT_BROKER_CLIENT_TLS_KEYSTORE_PATH = "pinot.broker.client.tls.keystore.path";
+  private static final String PINOT_BROKER_CLIENT_TLS_KEYSTORE_PASSWORD = "pinot.broker.client.tls.keystore.password";
+  private static final String PINOT_BROKER_CLIENT_TLS_TRUSTSTORE_PATH = "pinot.broker.client.tls.truststore.path";
+  private static final String PINOT_BROKER_CLIENT_TLS_TRUSTSTORE_PASSWORD = "pinot.broker.client.tls.truststore.password";
+  private static final String PINOT_BROKER_CLIENT_TLS_REQUIRES_CLIENT_AUTH = "pinot.broker.client.tls.requires_client_auth";
+
+  private static final String PROTOCOL_HTTPS = "https";
+  private static final String PROTOCOL_HTTP = "http";
+
   private URI _baseUri;
   private HttpServer _httpServer;
 
@@ -58,13 +72,42 @@ public class BrokerAdminApiApplication extends ResourceConfig {
     registerClasses(io.swagger.jaxrs.listing.SwaggerSerializers.class);
   }
 
-  public void start(int httpPort) {
-    Preconditions.checkArgument(httpPort > 0);
-    _baseUri = URI.create("http://0.0.0.0:" + httpPort + "/");
-    _httpServer = GrizzlyHttpServerFactory.createHttpServer(_baseUri, this);
+  public void start(PinotConfiguration brokerConf) {
+    int brokerQueryPort = brokerConf.getProperty(
+        CommonConstants.Helix.KEY_OF_BROKER_QUERY_PORT, CommonConstants.Helix.DEFAULT_BROKER_QUERY_PORT);
+
+    Preconditions.checkArgument(brokerQueryPort > 0);
+    _baseUri = URI.create(String.format("%s://0.0.0.0:%d/",
+        brokerConf.getProperty(PINOT_BROKER_CLIENT_PROTOCOL, PROTOCOL_HTTP), brokerQueryPort));
+
+    _httpServer = buildHttpsServer(brokerConf);
     setupSwagger();
   }
 
+  private HttpServer buildHttpsServer(PinotConfiguration brokerConf) {
+    boolean isSecure = PROTOCOL_HTTPS.equals(brokerConf.getProperty(PINOT_BROKER_CLIENT_PROTOCOL, PROTOCOL_HTTP));
+
+    if (isSecure) {
+      return GrizzlyHttpServerFactory.createHttpServer(_baseUri, this, true, buildSSLConfig(brokerConf));
+    }
+
+    return GrizzlyHttpServerFactory.createHttpServer(_baseUri, this);
+  }
+
+  private SSLEngineConfigurator buildSSLConfig(PinotConfiguration brokerConf) {
+    SSLContextConfigurator sslContextConfigurator = new SSLContextConfigurator();
+
+    sslContextConfigurator.setKeyStoreFile(brokerConf.getProperty(PINOT_BROKER_CLIENT_TLS_KEYSTORE_PATH));
+    sslContextConfigurator.setKeyStorePass(brokerConf.getProperty(PINOT_BROKER_CLIENT_TLS_KEYSTORE_PASSWORD));
+    sslContextConfigurator.setTrustStoreFile(brokerConf.getProperty(PINOT_BROKER_CLIENT_TLS_TRUSTSTORE_PATH));
+    sslContextConfigurator.setTrustStorePass(brokerConf.getProperty(PINOT_BROKER_CLIENT_TLS_TRUSTSTORE_PASSWORD));
+
+    boolean requiresClientAuth = brokerConf.getProperty(PINOT_BROKER_CLIENT_TLS_REQUIRES_CLIENT_AUTH, false);
+
+    return new SSLEngineConfigurator(sslContextConfigurator).setClientMode(false)
+        .setWantClientAuth(requiresClientAuth).setEnabledProtocols(new String[] { "TLSv1.2" });
+  }
+
   private void setupSwagger() {
     BeanConfig beanConfig = new BeanConfig();
     beanConfig.setTitle("Pinot Broker API");
diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
index 730076e..746bf7a 100644
--- a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
+++ b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/HelixBrokerStarter.java
@@ -245,7 +245,7 @@ public class HelixBrokerStarter implements ServiceStartable {
     int brokerQueryPort = _brokerConf.getProperty(Helix.KEY_OF_BROKER_QUERY_PORT, Helix.DEFAULT_BROKER_QUERY_PORT);
     LOGGER.info("Starting broker admin application on port: {}", brokerQueryPort);
     _brokerAdminApplication = new BrokerAdminApiApplication(_routingManager, _brokerRequestHandler, _brokerMetrics);
-    _brokerAdminApplication.start(brokerQueryPort);
+    _brokerAdminApplication.start(_brokerConf);
 
     LOGGER.info("Initializing cluster change mediator");
     for (ClusterChangeHandler externalViewChangeHandler : _externalViewChangeHandlers) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
index 004f863..fa5d258 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotQueryResource.java
@@ -196,10 +196,12 @@ public class PinotQueryResource {
       LOGGER.error("Instance {} not found", instanceId);
       return QueryException.INTERNAL_ERROR.toString();
     }
+
+    // TODO extract protocol from Helix
+    String protocol = "http";
     String hostNameWithPrefix = instanceConfig.getHostName();
-    String url =
-        getQueryURL(hostNameWithPrefix.substring(hostNameWithPrefix.indexOf("_") + 1), instanceConfig.getPort(),
-            querySyntax);
+    String url = getQueryURL(protocol, hostNameWithPrefix.substring(hostNameWithPrefix.indexOf("_") + 1),
+        instanceConfig.getPort(), querySyntax);
     ObjectNode requestJson = getRequestJson(query, traceEnabled, queryOptions, querySyntax);
     return sendRequestRaw(url, query, requestJson);
   }
@@ -226,12 +228,12 @@ public class PinotQueryResource {
     return requestJson;
   }
 
-  private String getQueryURL(String hostName, String port, String querySyntax) {
+  private String getQueryURL(String protocol, String hostName, String port, String querySyntax) {
     switch (querySyntax) {
       case CommonConstants.Broker.Request.SQL:
-        return String.format("http://%s:%s/query/sql", hostName, port);
+        return String.format("%s://%s:%s/query/sql", protocol, hostName, port);
       case CommonConstants.Broker.Request.PQL:
-        return String.format("http://%s:%s/query", hostName, port);
+        return String.format("%s://%s:%s/query", protocol, hostName, port);
       default:
         throw new UnsupportedOperationException("Unsupported query syntax - " + querySyntax);
     }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/PostQueryCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/PostQueryCommand.java
index e08aa3e..511d7e7 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/PostQueryCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/PostQueryCommand.java
@@ -32,12 +32,15 @@ import org.slf4j.LoggerFactory;
 public class PostQueryCommand extends AbstractBaseAdminCommand implements Command {
   private static final Logger LOGGER = LoggerFactory.getLogger(PostQueryCommand.class.getName());
 
-  @Option(name = "-brokerHost", required = false, metaVar = "<String>", usage = "host name for controller.")
+  @Option(name = "-brokerHost", required = false, metaVar = "<String>", usage = "host name for broker.")
   private String _brokerHost;
 
   @Option(name = "-brokerPort", required = false, metaVar = "<int>", usage = "http port for broker.")
   private String _brokerPort = Integer.toString(CommonConstants.Helix.DEFAULT_BROKER_QUERY_PORT);
 
+  @Option(name = "-brokerProtocol", required = false, metaVar = "<String>", usage = "protocol for broker.")
+  private String _brokerProtocol = "http";
+
   @Option(name = "-queryType", required = false, metaVar = "<string>", usage = "Query use sql or pql.")
   private String _queryType = Request.PQL;
 
@@ -59,7 +62,8 @@ public class PostQueryCommand extends AbstractBaseAdminCommand implements Comman
 
   @Override
   public String toString() {
-    return ("PostQuery -brokerHost " + _brokerHost + " -brokerPort " + _brokerPort + " -queryType " + _queryType + " -query " + _query);
+    return ("PostQuery -brokerProtocol " + _brokerProtocol + " -brokerHost " + _brokerHost + " -brokerPort " +
+        _brokerPort + " -queryType " + _queryType + " -query " + _query);
   }
 
   @Override
@@ -82,6 +86,11 @@ public class PostQueryCommand extends AbstractBaseAdminCommand implements Comman
     return this;
   }
 
+  public PostQueryCommand setBrokerProtocol(String protocol) {
+    _brokerProtocol = protocol;
+    return this;
+  }
+
   public PostQueryCommand setQueryType(String queryType) {
     _queryType = queryType;
     return this;
@@ -100,7 +109,7 @@ public class PostQueryCommand extends AbstractBaseAdminCommand implements Comman
     LOGGER.info("Executing command: " + toString());
 
     String request;
-    String urlString = "http://" + _brokerHost + ":" + _brokerPort + "/query";
+    String urlString = _brokerProtocol + "://" + _brokerHost + ":" + _brokerPort + "/query";
     if (_queryType.toLowerCase().equals(Request.SQL)) {
       urlString += "/sql";
       request = JsonUtils.objectToString(Collections.singletonMap(Request.SQL, _query));


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


[incubator-pinot] 01/07: cert scripts

Posted by ap...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

apucher pushed a commit to branch pinot-broker-https-discussion
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git

commit 40096f28f4dfa97b490bb6f231a807d04c935822
Author: Alexander Pucher <al...@alexpucher.com>
AuthorDate: Mon Jan 4 12:06:50 2021 -0800

    cert scripts
---
 create-certs.sh    | 46 ++++++++++++++++++++++++++++++++++++++++
 push-truststore.sh | 62 ++++++++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 108 insertions(+)

diff --git a/create-certs.sh b/create-certs.sh
new file mode 100755
index 0000000..2161e22
--- /dev/null
+++ b/create-certs.sh
@@ -0,0 +1,46 @@
+#
+# 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.
+#
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+DOMAIN="192.168.64.82.xip.io"
+
+KEY_DIR="$DIR/truststore"
+
+mkdir -p $KEY_DIR
+rm $KEY_DIR/*.pem
+
+echo "1. Generate CA's private key and self-signed certificate"
+openssl req -x509 -newkey rsa:4096 -days 365 -nodes -keyout "$KEY_DIR/ca-key.pem" -out "$KEY_DIR/ca-cert.pem" -subj "/C=US/ST=Someplace/L=Somewhere/O=Apache Pinot/OU=Education/CN=*.example.org/emailAddress=admin@example.org"
+
+#echo "CA's self-signed certificate"
+#openssl x509 -in "$KEY_DIR/ca-cert.pem" -noout -text
+
+echo "2. Generate web server's private key and certificate signing request (CSR)"
+openssl req -newkey rsa:4096 -nodes -keyout "$KEY_DIR/key.pem" -out "$KEY_DIR/req.pem" -subj "/C=US/ST=Someplace/L=Somewhere/O=Apache Pinot/OU=Education/CN=*.$DOMAIN/emailAddress=admin@example.com"
+
+echo "3. Use CA's private key to sign web server's CSR and get back the signed certificate"
+echo "subjectAltName=DNS:*.$DOMAIN,IP:0.0.0.0" > "$KEY_DIR/ext.cnf"
+openssl x509 -req -in "$KEY_DIR/req.pem" -days 60 -CA "$KEY_DIR/ca-cert.pem" -CAkey "$KEY_DIR/ca-key.pem" -CAcreateserial -out "$KEY_DIR/cert.pem" -extfile "$KEY_DIR/ext.cnf"
+
+#echo "Server's signed certificate"
+#openssl x509 -in "$KEY_DIR/cert.pem" -noout -text
+
+echo "Verifying certificate"
+openssl verify -CAfile "$KEY_DIR/ca-cert.pem" "$KEY_DIR/cert.pem"
+
diff --git a/push-truststore.sh b/push-truststore.sh
new file mode 100755
index 0000000..5a4ed61
--- /dev/null
+++ b/push-truststore.sh
@@ -0,0 +1,62 @@
+#
+# 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.
+#
+
+DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )"
+JAVA_HOME="/Library/Java/JavaVirtualMachines/jdk-13.0.2.jdk/Contents/Home"
+
+CONFIG_DIR="$DIR/truststore"
+KEY_TOOL=$JAVA_HOME/bin/keytool
+KEYSTORE_PASSWORD="changeit"
+
+TRUST_STORE=$CONFIG_DIR/generated.truststore.jks
+KEY_STORE=$CONFIG_DIR/generated.keystore.jks
+P12_STORE=$CONFIG_DIR/generated.key.p12
+
+echo "removing any old generated files"
+rm -f $TRUST_STORE $KEY_STORE $P12_STORE
+echo "writing trust store"
+
+$KEY_TOOL \
+  -noprompt \
+  -import \
+  -storepass $KEYSTORE_PASSWORD \
+  -keystore $TRUST_STORE \
+  -storetype PKCS12 \
+  -file $CONFIG_DIR/ca-cert.pem
+echo "converting key/cert into PKCS12"
+
+openssl pkcs12 \
+  -export \
+  -in $CONFIG_DIR/cert.pem \
+  -inkey $CONFIG_DIR/key.pem \
+  -out $P12_STORE \
+  -password pass:$KEYSTORE_PASSWORD \
+  -name localhost
+echo "writing key store"
+
+$KEY_TOOL -importkeystore \
+  -deststorepass $KEYSTORE_PASSWORD \
+  -destkeypass $KEYSTORE_PASSWORD \
+  -destkeystore $KEY_STORE \
+  -deststoretype PKCS12 \
+  -srckeystore $P12_STORE \
+  -srcstoretype PKCS12 \
+  -srcstorepass $KEYSTORE_PASSWORD \
+  -srckeypass $KEYSTORE_PASSWORD \
+  -alias localhost


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