#general


@john.knappster: @john.knappster has joined the channel
@john.knappster: Hi, I'm new to Pinot and have a possibly dumb question - with so many features and horizontal scalability, when should one *not* use Pinot as a general relational database solution?
  @mayanks: Pinot is optimized for performance, but is not a general relational database. For example, it doesn't provide transactions, or support for complex joins/nested queries.
  @john.knappster: Thanks. Do you have any links that talk about the limitations? I know that joins are not supported but presto is supposed to enable that.
  @mayanks: Yes, we close those gaps with Presto
  @mayanks: We don't have an exhaustive list of limitations but here are the major ones: ```1. Non transactional, eventually consistent 2. Does not support full SQL (used with Presto to fill that gap). 3. Not a source of truth```
  @john.knappster: I see. Well, it sounds like it should pair well (to use wine terminology) with any other transactional database. Use the transactional DB for read-modify-write and use Pinot for reads so long as eventually consistent is acceptable.
  @john.knappster: Can it be paired with a no-sql DB like Mongo DB?
  @dlavoie: If you can leverage a CDC infrastructure to get your changeset to pinot, it does work nicely together
  @mayanks: So the pattern there is to emit the changeset to a messaging system (eg Kafka) which Pinot can consume from
  @dlavoie: And since Pinot scales well for reads and write, it can be fed by different transactional sources from your system.
  @john.knappster: CDC = change data analysis?
  @dlavoie: Change Data Capture
  @john.knappster: Any recommended CDC architectures? debezium any good?
  @mayanks: Yeah I have heard folks using that
  @chinmay.cerebro: @john.knappster
  @chinmay.cerebro: Debezium is definitely the way to go !
  @chinmay.cerebro: Kenny has another blog coming up with Debezium and Pinot upserts which is even better
@jan.stegink: @jan.stegink has joined the channel
@aiyer: @aiyer has joined the channel

#random


@john.knappster: @john.knappster has joined the channel
@jan.stegink: @jan.stegink has joined the channel
@aiyer: @aiyer has joined the channel

#troubleshooting


@john.knappster: @john.knappster has joined the channel
@phuchdh: Hi, I got this issue when submit spark-job to ingest batch file. ```21/04/20 03:03:42 ERROR org.apache.pinot.tools.admin.command.LaunchDataIngestionJobCommand: Got exception to kick off standalone data ingestion job - java.lang.RuntimeException: Caught exception during running - org.apache.pinot.plugin.ingestion.batch.spark.SparkSegmentGenerationJobRunner at org.apache.pinot.spi.ingestion.batch.IngestionJobLauncher.kickoffIngestionJob(IngestionJobLauncher.java:144) at org.apache.pinot.spi.ingestion.batch.IngestionJobLauncher.runIngestionJob(IngestionJobLauncher.java:117) at org.apache.pinot.tools.admin.command.LaunchDataIngestionJobCommand.execute(LaunchDataIngestionJobCommand.java:132) at org.apache.pinot.tools.admin.command.LaunchDataIngestionJobCommand.main(LaunchDataIngestionJobCommand.java:67) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52) at $apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:951) at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:180) at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203) at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90) at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1030) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1039) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.nio.file.FileSystemNotFoundException: Provider "gs" not installed at java.nio.file.Paths.get(Paths.java:147) at org.apache.pinot.plugin.filesystem.GcsPinotFS.copy(GcsPinotFS.java:262) at org.apache.pinot.plugin.ingestion.batch.spark.SparkSegmentGenerationJobRunner.run(SparkSegmentGenerationJobRunner.java:344) at org.apache.pinot.spi.ingestion.batch.IngestionJobLauncher.kickoffIngestionJob(IngestionJobLauncher.java:142) ... 15 more```
  @phuchdh: after some deep-dive, I think it’s the Scala filesystem only 2 providers: file, jar
  @phuchdh: base on some explain & discuss : I suggest move `java.nio.file.Path` to `org.apache.hadoop.fs.Path` Have any idea ?
  @jackie.jxt: @fx19880617 ^^
  @fx19880617: can you try to add ```google-cloud-nio``` dependency into pinot-gcs pom file?
  @fx19880617:
  @fx19880617: hmm, seems there is already one in pom: ``` <dependency> <groupId>com.google.cloud</groupId> <artifactId>google-cloud-nio</artifactId> <version>0.120.0-alpha</version> </dependency>```
  @phuchdh: yup, in pom already has dependency
  @fx19880617: I’m looking at here:
  @fx19880617: maybe we can modify this code to directly use google cloudstorage api
  @fx19880617: does this code work on your standalone application
  @fx19880617: or test
  @fx19880617: but only failed on spark?
  @phuchdh: yup.
  @phuchdh: this code work on standalone application
  @phuchdh: but fail on spark cluster on dataproc
  @fx19880617: hmm
  @fx19880617: on spark cluster, can you try to put the gcs plugin to the classpath
  @phuchdh: already tried. :smile:
  @fx19880617: I assume this is java 8?
  @fx19880617: or 11?
  @phuchdh: java 8
  @phuchdh: let me check image spark
  @fx19880617: another way is to shade pinot-ingestion-spark and pinot-gcs together into one jar
  @phuchdh: :smile: another way to fix, remove `java.nio.file.Paths` in gcs-pinot, then custom another Paths.get function ?
  @fx19880617: right
  @fx19880617: then you need to implement the relatize function there
  @fx19880617: I feel better to just use native google gcs lib
  @phuchdh: well, just another error ini sendSegmentUri pharse. ```Caused by: java.lang.IllegalStateException: PinotFS for scheme: gs has not been initialized at shaded.com.google.common.base.Preconditions.checkState(Preconditions.java:518) at org.apache.pinot.spi.filesystem.PinotFSFactory.create(PinotFSFactory.java:80) at org.apache.pinot.plugin.ingestion.batch.common.SegmentPushUtils.sendSegmentUris(SegmentPushUtils.java:158) at org.apache.pinot.plugin.ingestion.batch.spark.SparkSegmentUriPushJobRunner$1.call(SparkSegmentUriPushJobRunner.java:122) at org.apache.pinot.plugin.ingestion.batch.spark.SparkSegmentUriPushJobRunner$1.call(SparkSegmentUriPushJobRunner.java:117) ```
  @phuchdh: are you have any idea how to fix it ?
  @phuchdh:
  @fx19880617: It’s missing fs init in the code:
  @fx19880617: merged the above pr, can you try it again ?
@jan.stegink: @jan.stegink has joined the channel
@ravikumar.m: Dear All, I am trying a PoC which I am trying to ingest data from S3 bucket to Pinot local. 1. Data exist in S3. 2. Data in JSON format. 3. Need to push to Pinot in each 15 mins interval. 4. Output location(Pinot segments) is local file system. I have few doubts: 1. Pinot itself can do ingest, store, query or we should need any other tools to do. I find one blog, in that they are using spark and for ingestion. So we should need something to ingestion, is Pinot itself can not get data from S3. 2. In my S3 bucket the folder structure 'year/month/day' but in Pinot job spec we can specify the a folder can not cover sub folders(as per Pinot documentation) so how can I specify in my job spec yaml to cover S3 sub folders also. How can I overcome this limitation.
@aiyer: @aiyer has joined the channel
@sleepythread: ```You will also need to provide proper Hadoop dependencies jars from your Hadoop installation to your Pinot startup scripts.```
@sleepythread: With respect to this, how do we specify the jar location and what jar we need to supply ?
  @mayanks: @fx19880617 ^^
  @sleepythread: ```export HADOOP_HOME=/local/hadoop/ export HADOOP_CONF_DIR=/local/hadoop/etc/hadoop/ export JAVA_OPTS="-Xms12G -Xmx28G -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCApplicationConcurrentTime -Xloggc:gc-pinot-server.log -Dplugins.dir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/plugins/ -Dlog4j2.configurationFile=file:/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/conf/pinot-server-log4j2.xml" /home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/bin/start-server.sh -zkAddress hotpanel1.mlan:2181,hotpanel3.mlan:2181,bi24.mlan:2181 -configFileName /home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/conf/server.conf``` Server Conf ```pinot.server.instance.enable.split.commit=true pinot.server.storage.factory.class.hdfs=org.apache.pinot.plugin.filesystem.HadoopPinotFS pinot.server.storage.factory.hdfs.hadoop.conf.path=/local/hadoop/etc/hadoop/ pinot.server.segment.fetcher.protocols=file,http,hdfs pinot.server.segment.fetcher.hdfs.class=org.apache.pinot.common.utils.fetcher.PinotFSSegmentFetcher pinot.server.instance.dataDir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/data/PinotServer/index pinot.server.instance.segmentTarDir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/data/PinotServer/segmentTar``` Error which I am getting ```2021/04/20 17:19:31.724 ERROR [StartServiceManagerCommand] [Start a Pinot [SERVER]] Failed to start a Pinot [SERVER] at 3.081 since launch java.lang.RuntimeException: java.lang.RuntimeException: Could not initialize HadoopPinotFS at org.apache.pinot.spi.filesystem.PinotFSFactory.register(PinotFSFactory.java:58) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.spi.filesystem.PinotFSFactory.init(PinotFSFactory.java:74) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.server.starter.helix.SegmentFetcherAndLoader.<init>(SegmentFetcherAndLoader.java:71) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.server.starter.helix.HelixServerStarter.start(HelixServerStarter.java:336) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.service.PinotServiceManager.startServer(PinotServiceManager.java:150) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.service.PinotServiceManager.startRole(PinotServiceManager.java:95) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.admin.command.StartServiceManagerCommand$1.lambda$run$0(StartServiceManagerCommand.java:260) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.admin.command.StartServiceManagerCommand.startPinotService(StartServiceManagerCommand.java:286) [pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.admin.command.StartServiceManagerCommand.access$000(StartServiceManagerCommand.java:57) [pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.tools.admin.command.StartServiceManagerCommand$1.run(StartServiceManagerCommand.java:260) [pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] Caused by: java.lang.RuntimeException: Could not initialize HadoopPinotFS at org.apache.pinot.plugin.filesystem.HadoopPinotFS.init(HadoopPinotFS.java:71) ~[pinot-hdfs-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.spi.filesystem.PinotFSFactory.register(PinotFSFactory.java:54) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] ... 9 more Caused by: java.io.IOException: No FileSystem for scheme: hdfs at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2644) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2651) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:170) ~[pinot-parquet-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.plugin.filesystem.HadoopPinotFS.init(HadoopPinotFS.java:67) ~[pinot-hdfs-0.7.0-SNAPSHOT-shaded.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] at org.apache.pinot.spi.filesystem.PinotFSFactory.register(PinotFSFactory.java:54) ~[pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar:0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e] ... 9 more```
  @fx19880617: can you try to put to put hadoop jar into your java classpath ?
  @fx19880617: or put it into pinot plugin directory
  @sleepythread: which specific jar ?
  @sleepythread: hadoop-common ?
  @fx19880617: hadoop common
  @sleepythread: There are lots of jars
  @sleepythread: Cool. Thanks.
  @mayanks: Thanks @fx19880617.
  @mayanks: @sleepythread Could you please confirm if this solves the issue, we can add that into Faq.
  @sleepythread: The server script is quitting now. ```2021/04/20 18:04:01.026 INFO [HelixInstanceDataManagerConfig] [Start a Pinot [SERVER]] InstanceDataManagerConfig, key: reload.consumingsegment , value: false 2021/04/20 18:04:01.026 INFO [HelixInstanceDataManagerConfig] [Start a Pinot [SERVER]] InstanceDataManagerConfig, key: id , value: Server_10.10.211.27_8098 2021/04/20 18:04:01.026 INFO [HelixInstanceDataManagerConfig] [Start a Pinot [SERVER]] InstanceDataManagerConfig, key: readmode , value: mmap 2021/04/20 18:04:01.027 INFO [HelixInstanceDataManager] [Start a Pinot [SERVER]] HelixInstanceDataManagerConfig: Instance Id: Server_10.10.211.27_8098 Instance Data Dir: /home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/data/PinotServer/index Instance Segment Tar Dir: /home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/data/PinotServer/segmentTar Bootstrap Segment Dir: null Read Mode: mmap Segment format version: null 2021/04/20 18:04:01.029 INFO [SegmentBuildTimeLeaseExtender] [Start a Pinot [SERVER]] Initialized segment build time lease extender executor 2021/04/20 18:04:01.030 INFO [HelixInstanceDataManager] [Start a Pinot [SERVER]] Initialized Helix instance data manager 2021/04/20 18:04:02.382 INFO [FunctionRegistry] [Start a Pinot [SERVER]] Initialized FunctionRegistry with 119 functions: [fromepochminutesbucket, arrayunionint, codepoint, mod, sha256, year, yearofweek, upper, arraycontainsstring, arraydistinctstring, bytestohex, tojsonmapstr, trim, timezoneminute, sqrt, togeometry, normalize, fromepochdays, arraydistinctint, exp, jsonpathlong, yow, toepochhoursrounded, lower, toutf8, concat, ceil, todatetime, jsonpathstring, substr, dayofyear, contains, jsonpatharray, arrayindexofint, fromepochhoursbucket, arrayindexofstring, minus, arrayunionstring, toepochhours, toepochdaysrounded, millisecond, fromepochhours, arrayreversestring, dow, doy, min, toepochsecondsrounded, strpos, jsonpath, tosphericalgeography, fromepochsecondsbucket, max, reverse, hammingdistance, stpoint, abs, timezonehour, toepochseconds, arrayconcatint, quarter, md5, ln, toepochminutes, arraysortstring, replace, strrpos, jsonpathdouble, stastext, second, arraysortint, split, fromepochdaysbucket, lpad, day, toepochminutesrounded, fromdatetime, fromepochseconds, arrayconcatstring, base64encode, ltrim, arraysliceint, chr, sha, plus, base64decode, month, arraycontainsint, toepochminutesbucket, startswith, week, jsonformat, sha512, arrayslicestring, fromepochminutes, remove, dayofmonth, times, hour, rpad, arrayremovestring, now, divide, bigdecimaltobytes, floor, toepochsecondsbucket, toepochdaysbucket, hextobytes, rtrim, length, toepochhoursbucket, bytestobigdecimal, toepochdays, arrayreverseint, datetrunc, minute, round, dayofweek, arrayremoveint, weekofyear] in 1352ms 2021/04/20 18:04:02.383 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing query executor of class: org.apache.pinot.core.query.executor.ServerQueryExecutorV1Impl 2021/04/20 18:04:02.387 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]] Default timeout for query executor : 15000 2021/04/20 18:04:02.387 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]] Trying to build SegmentPrunerService 2021/04/20 18:04:02.389 INFO [SegmentPrunerService] [Start a Pinot [SERVER]] Adding segment pruner: DataSchemaSegmentPruner 2021/04/20 18:04:02.389 INFO [SegmentPrunerService] [Start a Pinot [SERVER]] Adding segment pruner: ColumnValueSegmentPruner 2021/04/20 18:04:02.389 INFO [SegmentPrunerService] [Start a Pinot [SERVER]] Adding segment pruner: SelectionQuerySegmentPruner 2021/04/20 18:04:02.389 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]] Trying to build QueryPlanMaker 2021/04/20 18:04:02.390 INFO [InstancePlanMakerImplV2] [Start a Pinot [SERVER]] Initializing plan maker with maxInitialResultHolderCapacity: 10000, numGroupsLimit: 100000 2021/04/20 18:04:02.390 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]] Trying to build QueryExecutorTimer 2021/04/20 18:04:02.390 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing query scheduler 2021/04/20 18:04:02.393 INFO [QuerySchedulerFactory] [Start a Pinot [SERVER]] Using FCFS query scheduler 2021/04/20 18:04:02.394 INFO [ResourceManager] [Start a Pinot [SERVER]] Initializing with 48 query runner threads and 96 worker threads 2021/04/20 18:04:02.399 INFO [QueryScheduler] [Start a Pinot [SERVER]] Query log max rate: 10000.0 2021/04/20 18:04:02.399 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing Netty query server on port: 8098 2021/04/20 18:04:02.400 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing transform functions 2021/04/20 18:04:02.410 INFO [ServerInstance] [Start a Pinot [SERVER]] Finish initializing server instance 2021/04/20 18:04:02.413 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Did not find any fs classes in the configuration 2021/04/20 18:04:02.414 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Got scheme hdfs, initializing class org.apache.pinot.plugin.filesystem.HadoopPinotFS 2021/04/20 18:04:02.414 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Initializing PinotFS for scheme hdfs, classname org.apache.pinot.plugin.filesystem.HadoopPinotFS```
  @sleepythread: Error is gone
  @sleepythread: @fx19880617 ```/usr/bin/java -Xms12G -Xmx28G -XX:+UseG1GC -XX:MaxGCPauseMillis=200 -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintGCApplicationStoppedTime -XX:+PrintGCApplicationConcurrentTime -Xloggc:gc-pinot-server.log -Dplugins.dir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/plugins/ -Dlog4j2.configurationFile=file:/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/conf/pinot-server-log4j2.xml -cp /local/hadoop/share/hadoop/common/hadoop-common-2.7.1.jar -Dplugins.dir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/plugins -classpath /local/hadoop/share/hadoop/common/hadoop-common-2.7.1.jar:/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/lib/pinot-all-0.7.0-SNAPSHOT-jar-with-dependencies.jar -Dapp.name=start-server -Dapp.pid=51665 -Dapp.repo=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/lib -Dapp.home=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin -Dbasedir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin org.apache.pinot.tools.admin.PinotServer -zkAddress hotpanel1.mlan:2181,hotpanel3.mlan:2181,bi24.mlan:2181 -configFileName /home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/conf/server.conf``` I am directly running this command but the App is existing right away
  @sleepythread: Quitting*
  @sleepythread: ```2021/04/20 18:42:41.346 INFO [InstancePlanMakerImplV2] [Start a Pinot [SERVER]] Initializing plan maker with maxInitialResultHolderCapacity: 10000, numGroupsLimit: 100000 2021/04/20 18:42:41.346 INFO [ServerQueryExecutorV1Impl] [Start a Pinot [SERVER]] Trying to build QueryExecutorTimer 2021/04/20 18:42:41.346 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing query scheduler 2021/04/20 18:42:41.348 INFO [QuerySchedulerFactory] [Start a Pinot [SERVER]] Using FCFS query scheduler 2021/04/20 18:42:41.349 INFO [ResourceManager] [Start a Pinot [SERVER]] Initializing with 48 query runner threads and 96 worker threads 2021/04/20 18:42:41.353 INFO [QueryScheduler] [Start a Pinot [SERVER]] Query log max rate: 10000.0 2021/04/20 18:42:41.354 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing Netty query server on port: 8098 2021/04/20 18:42:41.355 INFO [ServerInstance] [Start a Pinot [SERVER]] Initializing transform functions 2021/04/20 18:42:41.364 INFO [ServerInstance] [Start a Pinot [SERVER]] Finish initializing server instance 2021/04/20 18:42:41.367 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Did not find any fs classes in the configuration 2021/04/20 18:42:41.367 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Got scheme hdfs, initializing class org.apache.pinot.plugin.filesystem.HadoopPinotFS 2021/04/20 18:42:41.367 INFO [PinotFSFactory] [Start a Pinot [SERVER]] Initializing PinotFS for scheme hdfs, classname org.apache.pinot.plugin.filesystem.HadoopPinotFS```
  @sleepythread: After this
  @fx19880617: you mean the server exits after this?
  @sleepythread: Yes
  @fx19880617: hmm
  @sleepythread: No error
  @sleepythread: It just stops.
  @fx19880617: is there console output or pinot-server-logs for the reason?
  @sleepythread: The above is console output.
  @sleepythread: I will add some more logs at the place its quiting.
  @fx19880617: sure
  @fx19880617: is there any log files on the working directory?
  @sleepythread: ```2021/04/20 18:43:58.393 INFO [MBeanRegistrar] [Start a Pinot [SERVER]] MBean HelixCallback:Change=HEALTH,Key=PinotCluster.Server_10.10.211.27_8098,Type=PARTICIPANT has been registered. 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-protobuf 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-kafka-2.0 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-avro 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-distribution 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-csv 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-s3 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-segment-uploader-default 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-yammer 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-batch-ingestion-standalone 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.402 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-confluent-avro 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-thrift 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-orc 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-batch-ingestion-spark 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-gcs 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-hdfs 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-batch-ingestion-hadoop 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-adls 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-json 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-minion-builtin-tasks 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-parquet 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.403 INFO [Utils] [Start a Pinot [SERVER]] Using pinot-segment-writer-file-based 0.7.0-SNAPSHOT-b9b31e5d2ecdc954934cdc1143d6f729b9f22c0e 2021/04/20 18:43:58.408 INFO [DefaultHelixStarterServerConfig] [Start a Pinot [SERVER]] External config key: pinot.server.segment.fetcher.hdfs.class, value: org.apache.pinot.common.utils.fetcher.PinotFSSegmentFetcher 2021/04/20 18:43:58.408 INFO [DefaultHelixStarterServerConfig] [Start a Pinot [SERVER]] External config key: pinot.server.segment.fetcher.protocols, value: file,http,hdfs 2021/04/20 18:43:58.408 INFO [DefaultHelixStarterServerConfig] [Start a Pinot [SERVER]] External config key: pinot.server.instance.enable.split.commit, value: true 2021/04/20 18:43:58.408 INFO [DefaultHelixStarterServerConfig] [Start a Pinot [SERVER]] E```
  @fx19880617: from the starter cmd, there seems to be some redundent: ```-cp /local/hadoop/share/hadoop/common/hadoop-common-2.7.1.jar -Dplugins.dir=/home/akashmishra/hpgraph/apache-pinot-incubating-0.7.0-SNAPSHOT-bin/plugins ```
@neilteng233: Hi some of my pinot instances are dead for unknown reasons. Is that any way to recover that? Because queries may fail if it is directed to a dead broker.
  @neilteng233: I am using the pinot 0.7 docker
  @mayanks: You can restart the nodes to bring them back
  @mayanks: However, please also check the logs to see why they went into this state (could be OOM).
  @neilteng233: docker restart pinot- controller/broker/server does not recover the state.
  @neilteng233: BTW, where is the log?
  @neilteng233: I can see in the cluster manager UI that has a error under liveinstance config:
  @neilteng233: { "code": 404, "error": "ZKPath /PinotCluster/LIVEINSTANCES/Controller_172.18.0.7_9000 does not exist:" }
  @mayanks: Logs should be on the instances themselves (whereever the log4j was configured)
  @mayanks: The restart should show some info in the logs on why this is happening.
  @neilteng233: I tailed the log from where it starts to throw exceptions

#pinot-dev


@kmvb.tau: @kmvb.tau has joined the channel
@nikhil.sulegaon: @nikhil.sulegaon has joined the channel

#complex-type-support


@g.kishore: Sorry I could not make it to the meeting
@g.kishore: is there is a summary?
@g.kishore: @amrish.k.lal
@amrish.k.lal: @g.kishore I think we all are on the same page. Will post a summary tomorrow morning after looking into existing complex type support a bit if that is ok
@mayanks: @mayanks has joined the channel
@amrish.k.lal: I have updated the with what we discussed yesterday. In short: we are going to be be supporting _Complex Type_ for ingesting non-tabular data (which includes, but is not limited to, hierarchical JSON data). For near term support: 1) data being ingested will conform to a __, 2) we will store _Complex Type_ data as JSON string, but with a version number so that the storage format can be changed later on without breaking backward compatibility, and 3) we will provide additional query language capability using dot notation (for example SELECT state.city.zip FROM table) and array subscripts (SELECT state.city[3].zip FROM table). and such queries will be rewritten to use JSON_EXTRACT_SCALAR and JSON_MATCH functions. In the long term, we will work towards providing better storage, indexing, and querying capability for _Complex Type_ data. I think we are all on same page, but please let me know if I missed including anything.
@g.kishore: why is 1) needed if we are storing it as JSON?
@g.kishore: I can ask this in the doc
@amrish.k.lal: #1 evolved from discussion that JSON is just one type of complex type and there could be other complex types as well (XML etc for example). For now though we use JSON as prime example of _Complex Type_.
@steotia: The goal is to support nested data and we should be able to capture the nested column by composing multiple complex types together (STRUCT, LIST, MAP) during schema definition.
@g.kishore: are we saying JSON is same as STRUCT
  @steotia: the root type of JSON could either be STRUCT , MAP or LIST
  @g.kishore: How will the user know what is the Struct?
  @g.kishore: Why not have another type JSON
  @g.kishore: And we decide internally how something should be stored
@g.kishore: should we introduce a new data type JSON?
@amrish.k.lal: I think we are saying that JSON can be ingested into a column whose type is _Complex Type._ In the near term, any JSON ingested would need to conform to a schema; however, we would definitely like to keep open the possibility of supporting JSON (which by definition doesn't need to conform to a specific schema).
  @g.kishore: I don’t agree with this
  @amrish.k.lal: what would you suggest?
  @g.kishore: We should still have the flexibility that any JSON can be ingested
  @g.kishore: Lot of folks are already using this feature
  @amrish.k.lal: I agree that there are certainly benefits to being able to ingest any JSON. @jackie.jxt ^^ @steotia ^^. Maybe we can also set up some defaults that would not require a schema?
  @steotia: This will allow us to ingest _any_ JSON
  @steotia: without breaking the existing feature
  @steotia: There is no need for decorating the nested column with JSON type explicitly
  @steotia: JSON is a kind of nested structure. The discussion we had yesterday was to be able to support nested structures (where the source form could be json, avro etc) through the same logical complex types we expose
  @amrish.k.lal: So we specify _Complex Type_ for a column and have a flag to mark type checking as optional and use some defaults in that case?
  @g.kishore: Quick call?
  @steotia: Right a person column where the data is in JSON format could be stored with root type as STRUCT and complete type as ```struct (name : string, age: INT : salary : INT, addresses : LIST (STRUCT ( apt: int, street : string, city : string, zip : INT )))```
  @jackie.jxt: The discussion here is should we support semi structured input (no fixed schema)
  @g.kishore: My vote is yes and we are already supporting that
  @jackie.jxt: That is true, but without a schema, it will be quite hard to do further optimizations
  @jackie.jxt: I can start a zoom now
  @jackie.jxt: @amrish.k.lal @steotia ^^
  @g.kishore: Let’s separate the two
  @amrish.k.lal: I am available @steotia?
  @steotia: sure
  @steotia: link ?
  @jackie.jxt:
@yupeng: for #1, i think it can still work with the complex-type ingestion proposal that i had
@yupeng: by introducing another rule of serializing the struct (and other types) to json
@yupeng: also, for #3, are the query language enhancements syntax sugar (rewrite) only? or do you consider some operator enhancement as well?
@amrish.k.lal: For now #3 is just syntax sugar. However, if in future we add an index that would allow us to directly evaluate the path expressions, then we can do that change under the covers without asking users to migrate their queries, etc. It also helps us to avoid escaping in JSON_MATCH function.
@yupeng: Sg
@amrish.k.lal: Modifying summary and based on discussion with @g.kishore: We are going to be be supporting _JSON_ column type for ingesting JSON data. For near term support: 1) data being ingested will be JSON data and does not need to conform to a schema, 2) we will store _JSON_ column type data as _STRING_, but with a version number so that the storage format can be changed later on without breaking backward compatibility, and 3) we will provide additional query language capability using dot notation (for example SELECT state.city.zip FROM table) and array subscripts (SELECT state.city[3].zip FROM table). and such queries will be rewritten to use JSON_EXTRACT_SCALAR and JSON_MATCH functions. In the long term, we will work towards providing better storage, indexing, and querying capability for  _JSON_ data. We don't want to overload the _Complex Type_, to support all different data types such as _JSON_, _XML_, etc but would treat these as different column types even though a lot of underlying structures and algorithms may be shared.
@amrish.k.lal: @yupeng FYI ^^
@yupeng: okay, does this mean ingestion with avro schema is outside the scope of this work?
@jackie.jxt: You can still store the structure as JSON string and use the JSON data type
@yupeng: can you be more specific?
@yupeng: what shall user do?
@jackie.jxt: In the ingestion config, after the complex types are extracted, we can use the `jsonFormat()` scalar function to convert the complex type into JSON and store it as JSON data type
@jackie.jxt: After adding the native complex type support (long term), we can then directly ingest the complex type data without converting them to json
@jackie.jxt: We don't want to mix the feature of json support and complex type support. LinkedIn will focus on json support first
@yupeng: without the complex data model, how do you extract complex types?
@jackie.jxt: From your design, I don't think we need to define the complex data model within the schema right?
@yupeng: mine has inbuilt rules
@yupeng: so the rule output is always primitive types
@yupeng: however with `jsonFormat` you assume a column of complex type
@yupeng: as func input
@jackie.jxt: The rule should define the columns to extract and the columns to flatten
@yupeng: that's fine
@yupeng: as long as we stick with rules
@jackie.jxt: Yes, from the ingestion side we stick with rules for now
@yupeng: okay, that works
@yupeng: then we shall reconsider some config names in
@yupeng: currently i always flatten the struct
@yupeng: we may consider a new rule to json serialize the struct
@jackie.jxt: You don't need to serialize the struct in the new added module. It can be `Map` or `List`
@jackie.jxt: In the rule, config the columns to extract and columns to flatten. If the extracted column is primitive type, leave it as is; if the extracted column is complex type, convert it to `Map` or `List`
@yupeng: okay, i think the default behavior is different from yours and mine
@yupeng: in mine, primitive types takes precedence over json type
@yupeng: so i always try to flatten it until i cannot, e.g. array
@yupeng: so perf is preferred
@jackie.jxt: While, I think the only difference here is to make the flatten columns configurable
@jackie.jxt: We can start with always flattening all complex types
@jackie.jxt: Then everything will be extracted into primitive type, and you don't need the json index feature
@yupeng: right, that's the behavior in the proposal
@yupeng: i think we are on the same page
--------------------------------------------------------------------- To unsubscribe, e-mail: dev-unsubscr...@pinot.apache.org For additional commands, e-mail: dev-h...@pinot.apache.org

Reply via email to