You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by xi...@apache.org on 2020/05/26 09:02:46 UTC

[incubator-pinot] 01/01: Make Pinot Broker/Server can start by just passing a config file

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

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

commit 1a1fac812b974ed9723691b8e2f090f8dccb9017
Author: Xiang Fu <fx...@gmail.com>
AuthorDate: Tue May 26 02:01:55 2020 -0700

    Make Pinot Broker/Server can start by just passing a config file
---
 .../broker/broker/helix/HelixBrokerStarter.java    | 28 ++++++++++++++++++----
 .../broker/broker/HelixBrokerStarterTest.java      |  5 +++-
 .../apache/pinot/common/utils/CommonConstants.java |  3 +++
 .../pinot/integration/tests/ClusterTest.java       | 12 ++++++++--
 .../tests/ServerStarterIntegrationTest.java        |  6 +++--
 .../server/starter/helix/HelixServerStarter.java   | 22 +++++++++++++----
 .../tools/admin/command/StartBrokerCommand.java    |  5 +++-
 .../tools/admin/command/StartServerCommand.java    |  5 +++-
 .../pinot/tools/perf/PerfBenchmarkDriver.java      | 11 +++++++--
 .../src/main/resources/conf/pinot-broker.conf      |  5 ++++
 .../src/main/resources/conf/pinot-controller.conf  |  7 ++++++
 .../src/main/resources/conf/pinot-server.conf      |  7 ++++++
 12 files changed, 99 insertions(+), 17 deletions(-)

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 0fc14fb..b801044 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
@@ -93,21 +93,39 @@ public class HelixBrokerStarter {
   // Participant Helix manager handles Helix functionality such as state transitions and messages
   private HelixManager _participantHelixManager;
 
+  @Deprecated
   public HelixBrokerStarter(Configuration brokerConf, String clusterName, String zkServer)
       throws Exception {
     this(brokerConf, clusterName, zkServer, null);
   }
 
+  @Deprecated
   public HelixBrokerStarter(Configuration brokerConf, String clusterName, String zkServer, @Nullable String brokerHost)
       throws Exception {
+    this(applyBrokerConfigs(brokerConf, clusterName, zkServer, brokerHost));
+  }
+
+  @Deprecated
+  private static Configuration applyBrokerConfigs(Configuration brokerConf, String clusterName, String zkServers, @Nullable String brokerHost) {
+    brokerConf.setProperty(Helix.CONFIG_OF_CLUSTER_NAME, clusterName);
+    brokerConf.setProperty(Helix.CONFIG_OF_ZOOKEEPR_SERVER, zkServers);
+    if (brokerHost == null) {
+      brokerConf.clearProperty(Broker.CONFIG_OF_BROKER_HOSTNAME);
+    } else {
+      brokerConf.setProperty(Broker.CONFIG_OF_BROKER_HOSTNAME, brokerHost);
+    }
+    return brokerConf;
+  }
+
+  public HelixBrokerStarter(Configuration brokerConf) throws Exception {
     _brokerConf = brokerConf;
     setupHelixSystemProperties();
 
-    _clusterName = clusterName;
+    _clusterName = brokerConf.getString(Helix.CONFIG_OF_CLUSTER_NAME);
 
     // Remove all white-spaces from the list of zkServers (if any).
-    _zkServers = zkServer.replaceAll("\\s+", "");
-
+    _zkServers = brokerConf.getString(Helix.CONFIG_OF_ZOOKEEPR_SERVER).replaceAll("\\s+", "");
+    String brokerHost = brokerConf.getString(Broker.CONFIG_OF_BROKER_HOSTNAME);
     if (brokerHost == null) {
       brokerHost = _brokerConf.getBoolean(CommonConstants.Helix.SET_INSTANCE_ID_TO_HOSTNAME_KEY, false) ? NetUtil
           .getHostnameOrAddress() : NetUtil.getHostAddress();
@@ -354,7 +372,9 @@ public class HelixBrokerStarter {
     int port = 5001;
     brokerConf.addProperty(Helix.KEY_OF_BROKER_QUERY_PORT, port);
     brokerConf.addProperty(Broker.CONFIG_OF_BROKER_TIMEOUT_MS, 60 * 1000L);
-    return new HelixBrokerStarter(brokerConf, "quickstart", "localhost:2122");
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, "quickstart");
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, "localhost:2122");
+    return new HelixBrokerStarter(brokerConf);
   }
 
   public static void main(String[] args)
diff --git a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
index a2970ab..44f249f 100644
--- a/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
+++ b/pinot-broker/src/test/java/org/apache/pinot/broker/broker/HelixBrokerStarterTest.java
@@ -31,6 +31,7 @@ import org.apache.pinot.broker.routing.RoutingTable;
 import org.apache.pinot.broker.routing.timeboundary.TimeBoundaryInfo;
 import org.apache.pinot.common.metadata.segment.OfflineSegmentZKMetadata;
 import org.apache.pinot.common.request.BrokerRequest;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.CommonConstants.Helix;
 import org.apache.pinot.common.utils.ZkStarter;
 import org.apache.pinot.common.utils.config.TagNameUtils;
@@ -77,7 +78,9 @@ public class HelixBrokerStarterTest extends ControllerTest {
 
     Configuration brokerConf = new BaseConfiguration();
     brokerConf.addProperty(Helix.KEY_OF_BROKER_QUERY_PORT, 18099);
-    _brokerStarter = new HelixBrokerStarter(brokerConf, getHelixClusterName(), ZkStarter.DEFAULT_ZK_STR);
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, getHelixClusterName());
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, ZkStarter.DEFAULT_ZK_STR);
+    _brokerStarter = new HelixBrokerStarter(brokerConf);
     _brokerStarter.start();
 
     addFakeBrokerInstancesToAutoJoinHelixCluster(NUM_BROKERS - 1, true);
diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
index 1b5740a..aac7710 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/CommonConstants.java
@@ -111,6 +111,8 @@ public class CommonConstants {
     public static final String CONFIG_OF_SERVER_FLAPPING_TIME_WINDOW_MS = "pinot.server.flapping.timeWindowMs";
     public static final String CONFIG_OF_MINION_FLAPPING_TIME_WINDOW_MS = "pinot.minion.flapping.timeWindowMs";
     public static final String DEFAULT_FLAPPING_TIME_WINDOW_MS = "1";
+    public static final String CONFIG_OF_CLUSTER_NAME = "pinot.cluster.name";
+    public static final String CONFIG_OF_ZOOKEEPR_SERVER = "pinot.zk.server";
   }
 
   public static class Broker {
@@ -135,6 +137,7 @@ public class CommonConstants {
     public static final String CONFIG_OF_BROKER_TIMEOUT_MS = "pinot.broker.timeoutMs";
     public static final long DEFAULT_BROKER_TIMEOUT_MS = 10_000L;
     public static final String CONFIG_OF_BROKER_ID = "pinot.broker.id";
+    public static final String CONFIG_OF_BROKER_HOSTNAME = "pinot.broker.hostname";
     // Configuration to consider the broker ServiceStatus as being STARTED if the percent of resources (tables) that
     // are ONLINE for this this broker has crossed the threshold percentage of the total number of tables
     // that it is expected to serve.
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
index d4f7494..bed4365 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ClusterTest.java
@@ -43,6 +43,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.http.HttpStatus;
 import org.apache.pinot.broker.broker.helix.HelixBrokerStarter;
 import org.apache.pinot.broker.requesthandler.PinotQueryRequest;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.CommonConstants.Broker;
 import org.apache.pinot.common.utils.CommonConstants.Helix;
 import org.apache.pinot.common.utils.CommonConstants.Minion;
@@ -64,6 +65,8 @@ import org.apache.pinot.spi.utils.JsonUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.pinot.common.utils.CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNotNull;
 import static org.testng.Assert.assertTrue;
@@ -103,11 +106,14 @@ public abstract class ClusterTest extends ControllerTest {
     _brokerStarters = new ArrayList<>(numBrokers);
     for (int i = 0; i < numBrokers; i++) {
       Configuration brokerConf = new BaseConfiguration();
+      brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, getHelixClusterName());
+      brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, zkStr);
+      brokerConf.addProperty(Broker.CONFIG_OF_BROKER_HOSTNAME, LOCAL_HOST);
       brokerConf.setProperty(Broker.CONFIG_OF_BROKER_TIMEOUT_MS, 60 * 1000L);
       brokerConf.setProperty(Helix.KEY_OF_BROKER_QUERY_PORT, Integer.toString(basePort + i));
       brokerConf.setProperty(Broker.CONFIG_OF_DELAY_SHUTDOWN_TIME_MS, 0);
       overrideBrokerConf(brokerConf);
-      HelixBrokerStarter brokerStarter = new HelixBrokerStarter(brokerConf, getHelixClusterName(), zkStr, LOCAL_HOST);
+      HelixBrokerStarter brokerStarter = new HelixBrokerStarter(brokerConf);
       brokerStarter.start();
       _brokerStarters.add(brokerStarter);
     }
@@ -151,7 +157,9 @@ public abstract class ClusterTest extends ControllerTest {
             .setProperty(Server.CONFIG_OF_INSTANCE_SEGMENT_TAR_DIR, Server.DEFAULT_INSTANCE_SEGMENT_TAR_DIR + "-" + i);
         configuration.setProperty(Server.CONFIG_OF_ADMIN_API_PORT, baseAdminApiPort - i);
         configuration.setProperty(Server.CONFIG_OF_NETTY_PORT, baseNettyPort + i);
-        _serverStarters.add(new HelixServerStarter(getHelixClusterName(), zkStr, configuration));
+        configuration.addProperty(CONFIG_OF_CLUSTER_NAME, getHelixClusterName());
+        configuration.addProperty(CONFIG_OF_ZOOKEEPR_SERVER, zkStr);
+        _serverStarters.add(new HelixServerStarter(configuration));
       }
     } catch (Exception e) {
       throw new RuntimeException(e);
diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ServerStarterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ServerStarterIntegrationTest.java
index 9c175ce..6513700 100644
--- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ServerStarterIntegrationTest.java
+++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/ServerStarterIntegrationTest.java
@@ -21,6 +21,7 @@ package org.apache.pinot.integration.tests;
 import org.apache.commons.configuration.BaseConfiguration;
 import org.apache.commons.configuration.Configuration;
 import org.apache.helix.model.InstanceConfig;
+import org.apache.pinot.common.utils.CommonConstants;
 import org.apache.pinot.common.utils.NetUtil;
 import org.apache.pinot.common.utils.ZkStarter;
 import org.apache.pinot.controller.helix.ControllerTest;
@@ -54,8 +55,9 @@ public class ServerStarterIntegrationTest extends ControllerTest {
   private void verifyInstanceConfig(Configuration serverConf, String expectedInstanceId, String expectedHost,
       int expectedPort)
       throws Exception {
-    HelixServerStarter helixServerStarter =
-        new HelixServerStarter(getHelixClusterName(), ZkStarter.DEFAULT_ZK_STR, serverConf);
+    serverConf.addProperty(CONFIG_OF_CLUSTER_NAME, getHelixClusterName());
+    serverConf.addProperty(CONFIG_OF_ZOOKEEPR_SERVER, ZkStarter.DEFAULT_ZK_STR);
+    HelixServerStarter helixServerStarter = new HelixServerStarter(serverConf);
     helixServerStarter.stop();
 
     assertEquals(helixServerStarter.getInstanceId(), expectedInstanceId);
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
index c6b0083..3036c7f 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/starter/helix/HelixServerStarter.java
@@ -106,15 +106,27 @@ public class HelixServerStarter {
   private final AdminApiApplication _adminApiApplication;
   private final RealtimeLuceneIndexRefreshState _realtimeLuceneIndexRefreshState;
 
+  @Deprecated
   public HelixServerStarter(String helixClusterName, String zkAddress, Configuration serverConf)
       throws Exception {
+    this(applyServerConfig(serverConf, helixClusterName, zkAddress));
+  }
+
+  @Deprecated
+  private static Configuration applyServerConfig(Configuration serverConf, String helixClusterName, String zkAddress) {
+    serverConf.setProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, helixClusterName);
+    serverConf.setProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, zkAddress);
+    return serverConf;
+  }
+
+  public HelixServerStarter(Configuration serverConf) throws Exception {
     LOGGER.info("Starting Pinot server");
     long startTimeMs = System.currentTimeMillis();
 
-    _helixClusterName = helixClusterName;
-    _zkAddress = zkAddress;
     // Make a clone so that changes to the config won't propagate to the caller
     _serverConf = ConfigurationUtils.cloneConfiguration(serverConf);
+    _helixClusterName = _serverConf.getString(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME);
+    _zkAddress = _serverConf.getString(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER);
 
     String host = _serverConf.getString(KEY_OF_SERVER_NETTY_HOST,
         _serverConf.getBoolean(CommonConstants.Helix.SET_INSTANCE_ID_TO_HOSTNAME_KEY, false) ? NetUtil
@@ -131,7 +143,7 @@ public class HelixServerStarter {
         _helixClusterName, _instanceId);
     setupHelixSystemProperties();
     _helixManager =
-        HelixManagerFactory.getZKHelixManager(helixClusterName, _instanceId, InstanceType.PARTICIPANT, _zkAddress);
+        HelixManagerFactory.getZKHelixManager(_helixClusterName, _instanceId, InstanceType.PARTICIPANT, _zkAddress);
 
     LOGGER.info("Initializing server instance and registering state model factory");
     Utils.logVersions();
@@ -575,10 +587,12 @@ public class HelixServerStarter {
       throws Exception {
     Configuration serverConf = new BaseConfiguration();
     int port = 8003;
+    serverConf.addProperty(CONFIG_OF_CLUSTER_NAME, "quickstart");
+    serverConf.addProperty(CONFIG_OF_ZOOKEEPR_SERVER, "localhost:2191");
     serverConf.addProperty(KEY_OF_SERVER_NETTY_PORT, port);
     serverConf.addProperty(CONFIG_OF_INSTANCE_DATA_DIR, "/tmp/PinotServer/test" + port + "/index");
     serverConf.addProperty(CONFIG_OF_INSTANCE_SEGMENT_TAR_DIR, "/tmp/PinotServer/test" + port + "/segmentTar");
-    return new HelixServerStarter("quickstart", "localhost:2191", serverConf);
+    return new HelixServerStarter(serverConf);
   }
 
   public static void main(String[] args)
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartBrokerCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartBrokerCommand.java
index 1e6af76..f434419 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartBrokerCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartBrokerCommand.java
@@ -118,11 +118,14 @@ public class StartBrokerCommand extends AbstractBaseAdminCommand implements Comm
         }
 
         brokerConf = new BaseConfiguration();
+        brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, _clusterName);
+        brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, _zkAddress);
         brokerConf.addProperty(CommonConstants.Helix.KEY_OF_BROKER_QUERY_PORT, _brokerPort);
+        brokerConf.addProperty(CommonConstants.Broker.CONFIG_OF_BROKER_HOSTNAME, _brokerHost);
       }
 
       LOGGER.info("Executing command: " + toString());
-      _brokerStarter = new HelixBrokerStarter(brokerConf, _clusterName, _zkAddress, _brokerHost);
+      _brokerStarter = new HelixBrokerStarter(brokerConf);
       _brokerStarter.start();
 
       String pidFile = ".pinotAdminBroker-" + System.currentTimeMillis() + ".pid";
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartServerCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartServerCommand.java
index 964ea12..ad743ad 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartServerCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/StartServerCommand.java
@@ -148,6 +148,9 @@ public class StartServerCommand extends AbstractBaseAdminCommand implements Comm
         }
 
         configuration = new PropertiesConfiguration();
+
+        configuration.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, _clusterName);
+        configuration.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, _zkAddress);
         configuration.addProperty(CommonConstants.Helix.KEY_OF_SERVER_NETTY_HOST, _serverHost);
         configuration.addProperty(CommonConstants.Helix.KEY_OF_SERVER_NETTY_PORT, _serverPort);
         configuration.addProperty(CommonConstants.Server.CONFIG_OF_ADMIN_API_PORT, _serverAdminPort);
@@ -156,7 +159,7 @@ public class StartServerCommand extends AbstractBaseAdminCommand implements Comm
       }
 
       LOGGER.info("Executing command: " + toString());
-      new HelixServerStarter(_clusterName, _zkAddress, configuration);
+      new HelixServerStarter(configuration);
       String pidFile = ".pinotAdminServer-" + String.valueOf(System.currentTimeMillis()) + ".pid";
       savePID(System.getProperty("java.io.tmpdir") + File.separator + pidFile);
       return true;
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/perf/PerfBenchmarkDriver.java b/pinot-tools/src/main/java/org/apache/pinot/tools/perf/PerfBenchmarkDriver.java
index 68a54db..7f4df1f 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/perf/PerfBenchmarkDriver.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/perf/PerfBenchmarkDriver.java
@@ -63,6 +63,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.yaml.snakeyaml.Yaml;
 
+import static org.apache.pinot.common.utils.CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME;
+import static org.apache.pinot.common.utils.CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER;
+
 
 @SuppressWarnings("FieldCanBeLocal")
 public class PerfBenchmarkDriver {
@@ -223,8 +226,10 @@ public class PerfBenchmarkDriver {
     String brokerInstanceName = "Broker_localhost_" + CommonConstants.Helix.DEFAULT_BROKER_QUERY_PORT;
     brokerConf.setProperty(CommonConstants.Helix.Instance.INSTANCE_ID_KEY, brokerInstanceName);
     brokerConf.setProperty(CommonConstants.Broker.CONFIG_OF_BROKER_TIMEOUT_MS, BROKER_TIMEOUT_MS);
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_CLUSTER_NAME, _clusterName);
+    brokerConf.addProperty(CommonConstants.Helix.CONFIG_OF_ZOOKEEPR_SERVER, _zkAddress);
     LOGGER.info("Starting broker instance: {}", brokerInstanceName);
-    new HelixBrokerStarter(brokerConf, _clusterName, _zkAddress).start();
+    new HelixBrokerStarter(brokerConf).start();
   }
 
   private void startServer()
@@ -238,12 +243,14 @@ public class PerfBenchmarkDriver {
     serverConfiguration
         .addProperty(CommonConstants.Server.CONFIG_OF_INSTANCE_SEGMENT_TAR_DIR, _serverInstanceSegmentTarDir);
     serverConfiguration.addProperty(CommonConstants.Helix.KEY_OF_SERVER_NETTY_HOST, "localhost");
+    serverConfiguration.addProperty(CONFIG_OF_CLUSTER_NAME, _clusterName);
+    serverConfiguration.addProperty(CONFIG_OF_ZOOKEEPR_SERVER, _zkAddress);
     if (_segmentFormatVersion != null) {
       serverConfiguration.setProperty(CommonConstants.Server.CONFIG_OF_SEGMENT_FORMAT_VERSION, _segmentFormatVersion);
     }
     serverConfiguration.setProperty(CommonConstants.Server.CONFIG_OF_INSTANCE_ID, _serverInstanceName);
     LOGGER.info("Starting server instance: {}", _serverInstanceName);
-    new HelixServerStarter(_clusterName, _zkAddress, serverConfiguration);
+    new HelixServerStarter(serverConfiguration);
   }
 
   private void startHelixResourceManager()
diff --git a/pinot-tools/src/main/resources/conf/pinot-broker.conf b/pinot-tools/src/main/resources/conf/pinot-broker.conf
new file mode 100644
index 0000000..1082423
--- /dev/null
+++ b/pinot-tools/src/main/resources/conf/pinot-broker.conf
@@ -0,0 +1,5 @@
+clusterName=pinot-quickstart
+zkAddress=localhost:2181
+pinot.broker.client.queryPort=8099
+pinot.broker.routing.table.builder.class=random
+pinot.set.instance.id.to.hostname=true
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/conf/pinot-controller.conf b/pinot-tools/src/main/resources/conf/pinot-controller.conf
new file mode 100644
index 0000000..42d6f29
--- /dev/null
+++ b/pinot-tools/src/main/resources/conf/pinot-controller.conf
@@ -0,0 +1,7 @@
+controller.helix.cluster.name=pinot-quickstart
+controller.port=9000
+controller.vip.host=localhost
+controller.vip.port=9000
+controller.data.dir=pinot/controller
+controller.zk.str=localhost:2181
+pinot.set.instance.id.to.hostname=true
\ No newline at end of file
diff --git a/pinot-tools/src/main/resources/conf/pinot-server.conf b/pinot-tools/src/main/resources/conf/pinot-server.conf
new file mode 100644
index 0000000..a24cd19
--- /dev/null
+++ b/pinot-tools/src/main/resources/conf/pinot-server.conf
@@ -0,0 +1,7 @@
+clusterName=pinot-quickstart
+zkAddress=localhost:2181
+pinot.server.netty.port=8098
+pinot.server.adminapi.port=8097
+pinot.server.instance.dataDir=pinot/server/data/index
+pinot.server.instance.segmentTarDir=pinot/server/data/segmentTar
+pinot.set.instance.id.to.hostname=true
\ No newline at end of file


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