You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/10/13 18:08:27 UTC

[GitHub] [pinot] yupeng9 opened a new pull request, #9588: add validation for proto decoder

yupeng9 opened a new pull request, #9588:
URL: https://github.com/apache/pinot/pull/9588

   Add validation for protobuf decoder on the necessary properties
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1004817285


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {
+        throw new IllegalStateException("Missing property of descriptorFile for ProtoBufMessageDecoder");
+      }
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.protoClassName")) {

Review Comment:
   Seems Kartik is OOO.
   What is the exception you got without setting this field? We want to make the validation align with the implementation. If you think this setting should be mandatory, we should also change the implementation accordingly



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] codecov-commenter commented on pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #9588:
URL: https://github.com/apache/pinot/pull/9588#issuecomment-1289469886

   # [Codecov](https://codecov.io/gh/apache/pinot/pull/9588?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#9588](https://codecov.io/gh/apache/pinot/pull/9588?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5616c12) into [master](https://codecov.io/gh/apache/pinot/commit/cea282c3682001e3af18a1e97627f2a7c5ddfcf4?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (cea282c) will **decrease** coverage by `2.65%`.
   > The diff coverage is `62.50%`.
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #9588      +/-   ##
   ============================================
   - Coverage     69.99%   67.34%   -2.66%     
   + Complexity     5261     5110     -151     
   ============================================
     Files          1934     1443     -491     
     Lines        103303    75578   -27725     
     Branches      15694    12056    -3638     
   ============================================
   - Hits          72306    50897   -21409     
   + Misses        25896    21014    -4882     
   + Partials       5101     3667    -1434     
   ```
   
   | Flag | Coverage Δ | |
   |---|---|---|
   | integration1 | `?` | |
   | integration2 | `?` | |
   | unittests1 | `67.34% <62.50%> (-0.03%)` | :arrow_down: |
   | unittests2 | `?` | |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   | [Impacted Files](https://codecov.io/gh/apache/pinot/pull/9588?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...he/pinot/segment/local/utils/TableConfigUtils.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3Qtc2VnbWVudC1sb2NhbC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VnbWVudC9sb2NhbC91dGlscy9UYWJsZUNvbmZpZ1V0aWxzLmphdmE=) | `68.01% <62.50%> (-0.15%)` | :arrow_down: |
   | [...va/org/apache/pinot/core/routing/RoutingTable.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yb3V0aW5nL1JvdXRpbmdUYWJsZS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...va/org/apache/pinot/common/config/NettyConfig.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL05ldHR5Q29uZmlnLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...a/org/apache/pinot/common/metrics/MinionMeter.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25NZXRlci5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...g/apache/pinot/common/metrics/ControllerMeter.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Db250cm9sbGVyTWV0ZXIuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/BrokerQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJRdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [.../apache/pinot/common/metrics/MinionQueryPhase.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9NaW5pb25RdWVyeVBoYXNlLmphdmE=) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...ache/pinot/server/access/AccessControlFactory.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3Qvc2VydmVyL2FjY2Vzcy9BY2Nlc3NDb250cm9sRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/SegmentReloadMessage.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvU2VnbWVudFJlbG9hZE1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | [...he/pinot/common/messages/TableDeletionMessage.java](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWVzc2FnZXMvVGFibGVEZWxldGlvbk1lc3NhZ2UuamF2YQ==) | `0.00% <0.00%> (-100.00%)` | :arrow_down: |
   | ... and [856 more](https://codecov.io/gh/apache/pinot/pull/9588/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1000851669


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {
+        throw new IllegalStateException("Missing property of descriptorFile for ProtoBufMessageDecoder");
+      }
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.protoClassName")) {

Review Comment:
   Added this check sepcifically because we ran into very misleading error messages which took us some time to realize this missing field



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1003851062


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,19 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  @VisibleForTesting
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {

Review Comment:
   ideally we can reference the class, but it's not on the dependency of this project.
   I think the best solution to refactor this validation framework to make it pluggable, so we can move this check to the protobuf package.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1005092938


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {
+        throw new IllegalStateException("Missing property of descriptorFile for ProtoBufMessageDecoder");
+      }
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.protoClassName")) {

Review Comment:
   I think it's better to be stricter, but want to have @KKcorps's thought



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1003563687


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r996240649


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {

Review Comment:
   I think we can introduce another parent function as we add more validations for streamConfig



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1005092558


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {
+        throw new IllegalStateException("Missing property of descriptorFile for ProtoBufMessageDecoder");
+      }
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.protoClassName")) {

Review Comment:
   The error message is that the protobuf message is corrupted.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] ankitsultana commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
ankitsultana commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r996048386


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {

Review Comment:
   Should this be called `validateStreamConfig`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] jackjlli commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
jackjlli commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r1003811504


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,19 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  @VisibleForTesting
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {

Review Comment:
   nit: could we put these constants to a class and have some comments to explain that these are the required fields for protobuf message decoder?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] Jackie-Jiang commented on a diff in pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
Jackie-Jiang commented on code in PR #9588:
URL: https://github.com/apache/pinot/pull/9588#discussion_r999992682


##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {

Review Comment:
   (minor) Annotate with `@VisiableForTesting`



##########
pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java:
##########
@@ -458,6 +461,18 @@ public static void validateIngestionAggregation(String name) {
         String.format("aggregation function %s must be one of %s", name, SUPPORTED_INGESTION_AGGREGATIONS));
   }
 
+  static void validateDecoder(StreamConfig streamConfig) {
+    if (streamConfig.getDecoderClass().equals("org.apache.pinot.plugin.inputformat.protobuf.ProtoBufMessageDecoder")) {
+      // check the existence of the needed decoder props
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.descriptorFile")) {
+        throw new IllegalStateException("Missing property of descriptorFile for ProtoBufMessageDecoder");
+      }
+      if (!streamConfig.getDecoderProperties().containsKey("stream.kafka.decoder.prop.protoClassName")) {

Review Comment:
   It this mandatory? Per the implementation, seems it is optional. cc @KKcorps 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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


[GitHub] [pinot] yupeng9 merged pull request #9588: add validation for proto decoder

Posted by GitBox <gi...@apache.org>.
yupeng9 merged PR #9588:
URL: https://github.com/apache/pinot/pull/9588


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


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