You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/10/28 09:51:07 UTC

[GitHub] [druid] smildlzj opened a new issue, #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

smildlzj opened a new issue, #13276:
URL: https://github.com/apache/druid/issues/13276

   Please provide a detailed title (e.g. "Broker crashes when using TopN query with Bound filter" instead of just "Broker crashes").
   
   ### Affected Version
   24.0.0
   
   
   ### Description
   upgrade from 0.23.0,  hadoop index task fail.
   
   _common/runtime.properties
   ```
   druid.extensions.loadList=["druid-datasketches", "druid-lookups-cached-global", "mysql-metadata-storage", "druid-hdfs-storage", "druid-multi-stage-query"]
   ```
   
   * payload
   ```
   {
     "type": "index_hadoop",
     "id": "index_yzjconnectapp@event_2022-10-28T11:00_2022-10-28T12:00_2022-10-28T16:25",
     "spec": {
       "dataSchema": {
         "dataSource": "yzjconnectapp@event",
         "timestampSpec": null,
         "dimensionsSpec": null,
         "metricsSpec": [],
         "granularitySpec": {
           "type": "uniform",
           "segmentGranularity": "HOUR",
           "queryGranularity": "HOUR",
           "rollup": false,
           "intervals": [
             "2022-10-28T03:00:00.000Z/2022-10-28T04:00:00.000Z"
           ]
         },
         "transformSpec": {
           "filter": null,
           "transforms": []
         },
         "parser": {
           "type": "hadoopyString",
           "parseSpec": {
             "format": "json",
             "dimensionsSpec": {
               "dimensions": [
                 "event",
                 "eid",
                 "oid",
                 "spaceName",
                 "spaceId",
                 "ename",
                 "appNum",
                 "mainUserFlag",
                 "userName",
                 "userRole",
                 "appType",
                 "appName",
                 "partnerSum",
                 "@network_provider",
                 "@country",
                 "@city",
                 "@province",
                 "@user_agent",
                 "@os",
                 "@os_version",
                 "@clientId",
                 "@deviceid"
               ]
             },
             "timestampSpec": {
               "format": "auto",
               "column": "@timestamp"
             }
           }
         }
       },
       "ioConfig": {
         "type": "hadoop",
         "inputSpec": {
           "type": "static",
           "paths": "hdfs://hadoop.architecture:8020/xxxx/dataly/druid/history/yzjconnectapp@event/2022-10-28_11.log"
         },
         "metadataUpdateSpec": null,
         "segmentOutputPath": null
       },
       "tuningConfig": {
         "type": "hadoop",
         "workingPath": null,
         "version": "2022-10-28T08:25:20.425Z",
         "partitionsSpec": {
           "type": "hashed",
           "numShards": null,
           "partitionDimensions": [],
           "partitionFunction": "murmur3_32_abs",
           "maxRowsPerSegment": 6000000
         },
         "shardSpecs": {},
         "indexSpec": {
           "bitmap": {
             "type": "roaring",
             "compressRunOnSerialization": true
           },
           "dimensionCompression": "lz4",
           "metricCompression": "lz4",
           "longEncoding": "longs",
           "segmentLoader": null
         },
         "indexSpecForIntermediatePersists": {
           "bitmap": {
             "type": "roaring",
             "compressRunOnSerialization": true
           },
           "dimensionCompression": "lz4",
           "metricCompression": "lz4",
           "longEncoding": "longs",
           "segmentLoader": null
         },
         "appendableIndexSpec": {
           "type": "onheap",
           "preserveExistingMetrics": false
         },
         "maxRowsInMemory": 1000000,
         "maxBytesInMemory": 0,
         "leaveIntermediate": false,
         "cleanupOnFailure": true,
         "overwriteFiles": false,
         "ignoreInvalidRows": true,
         "jobProperties": {
           "mapreduce.map.java.opts": "-Duser.timezone=UTC+8 -Dfile.encoding=UTF-8",
           "mapreduce.reduce.java.opts": "-Duser.timezone=UTC+8 -Dfile.encoding=UTF-8",
           "mapreduce.cluster.local.dir": "${hadoop.tmp.dir}/mapred/local/index_yzjconnectapp@event_2022-10-28T11:00_2022-10-28T12:00_2022-10-28T16:25"
         },
         "combineText": false,
         "useCombiner": false,
         "numBackgroundPersistThreads": 0,
         "forceExtendableShardSpecs": false,
         "useExplicitVersion": false,
         "allowedHadoopPrefix": [],
         "logParseExceptions": false,
         "maxParseExceptions": 2147483647,
         "useYarnRMJobStatusFallback": true,
         "awaitSegmentAvailabilityTimeoutMillis": 0
       },
       "uniqueId": "03b57b35798e481d9b005051a481f1c9"
     },
     "hadoopDependencyCoordinates": null,
     "classpathPrefix": null,
     "context": {
       "forceTimeChunkLock": true,
       "useLineageBasedSegmentAllocation": true
     },
     "groupId": "index_yzjconnectapp@event_2022-10-28T11:00_2022-10-28T12:00_2022-10-28T16:25",
     "dataSource": "yzjconnectapp@event",
     "resource": {
       "availabilityGroup": "index_yzjconnectapp@event_2022-10-28T11:00_2022-10-28T12:00_2022-10-28T16:25",
       "requiredCapacity": 1
     }
   }
   ```
   
   ```
   2022-10-28T16:25:34,775 ERROR [task-runner-0-priority-0] org.apache.druid.indexing.common.task.HadoopIndexTask - Got invocation target exception in run()
   java.lang.ExceptionInInitializerError: null
   	at org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopDetermineConfigInnerProcessingRunner.runTask(HadoopIndexTask.java:780) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_342]
   	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_342]
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_342]
   	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_342]
   	at org.apache.druid.indexing.common.task.HadoopIndexTask.runInternal(HadoopIndexTask.java:366) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at org.apache.druid.indexing.common.task.HadoopIndexTask.runTask(HadoopIndexTask.java:297) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at org.apache.druid.indexing.common.task.AbstractBatchIndexTask.run(AbstractBatchIndexTask.java:187) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at org.apache.druid.indexing.overlord.SingleTaskBackgroundRunner$SingleTaskBackgroundRunnerCallable.call(SingleTaskBackgroundRunner.java:477) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at org.apache.druid.indexing.overlord.SingleTaskBackgroundRunner$SingleTaskBackgroundRunnerCallable.call(SingleTaskBackgroundRunner.java:449) ~[druid-indexing-service-24.0.0.jar:24.0.0]
   	at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[?:1.8.0_342]
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_342]
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_342]
   	at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_342]
   Caused by: com.google.inject.CreationException: Unable to create injector, see the following errors:
   
   1) No implementation for java.util.Set<org.apache.druid.discovery.NodeRole> annotated with @org.apache.druid.guice.annotations.Self() was bound.
     while locating java.util.Set<org.apache.druid.discovery.NodeRole> annotated with @org.apache.druid.guice.annotations.Self()
       for the 2nd parameter of org.apache.druid.msq.guice.MSQIndexingModule.makeBouncer(MSQIndexingModule.java:193)
     at org.apache.druid.msq.guice.MSQIndexingModule.makeBouncer(MSQIndexingModule.java:193) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.msq.guice.MSQIndexingModule)
   
   2) A binding to com.google.common.base.Supplier<org.apache.druid.server.audit.SQLAuditManagerConfig> was already configured at org.apache.druid.guice.JsonConfigProvider.bind(JsonConfigProvider.java:151) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule).
     at org.apache.druid.guice.JsonConfigProvider.bind(JsonConfigProvider.java:151) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule)
   
   3) A binding to org.apache.druid.server.audit.SQLAuditManagerConfig was already configured at org.apache.druid.guice.JsonConfigProvider.bind(JsonConfigProvider.java:152) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.metadata.storage.mysql.MySQLMetadataStorageModule).
     at org.apache.druid.guice.JsonConfigProvider.bind(JsonConfigProvider.java:152) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.metadata.storage.postgresql.PostgreSQLMetadataStorageModule)
   
   4) No implementation for java.util.Set<org.apache.druid.discovery.NodeRole> annotated with @org.apache.druid.guice.annotations.Self() was bound.
     at org.apache.druid.msq.guice.MSQIndexingModule.makeBouncer(MSQIndexingModule.java:193) (via modules: com.google.inject.util.Modules$OverrideModule -> org.apache.druid.msq.guice.MSQIndexingModule)
   
   4 errors
   	at com.google.inject.internal.Errors.throwCreationExceptionIfErrorsExist(Errors.java:470) ~[guice-4.1.0.jar:?]
   	at com.google.inject.internal.InternalInjectorCreator.initializeStatically(InternalInjectorCreator.java:155) ~[guice-4.1.0.jar:?]
   	at com.google.inject.internal.InternalInjectorCreator.build(InternalInjectorCreator.java:107) ~[guice-4.1.0.jar:?]
   	at com.google.inject.Guice.createInjector(Guice.java:99) ~[guice-4.1.0.jar:?]
   	at com.google.inject.Guice.createInjector(Guice.java:73) ~[guice-4.1.0.jar:?]
   	at com.google.inject.Guice.createInjector(Guice.java:62) ~[guice-4.1.0.jar:?]
   	at org.apache.druid.initialization.ExtensionInjectorBuilder.build(ExtensionInjectorBuilder.java:49) ~[druid-server-24.0.0.jar:24.0.0]
   	at org.apache.druid.initialization.ServerInjectorBuilder.build(ServerInjectorBuilder.java:113) ~[druid-server-24.0.0.jar:24.0.0]
   	at org.apache.druid.initialization.ServerInjectorBuilder.makeServerInjector(ServerInjectorBuilder.java:71) ~[druid-server-24.0.0.jar:24.0.0]
   	at org.apache.druid.initialization.Initialization.makeInjectorWithModules(Initialization.java:63) ~[druid-server-24.0.0.jar:24.0.0]
   	at org.apache.druid.indexer.HadoopDruidIndexerConfig.<clinit>(HadoopDruidIndexerConfig.java:109) ~[druid-indexing-hadoop-24.0.0.jar:24.0.0]
   	... 14 more
   2022-10-28T16:25:34,806 INFO [task-runner-0-priority-0] org.apache.druid.indexing.worker.executor.ExecutorLifecycle - Task completed with status: {
     "id" : "index_yzjconnectapp@event_2022-10-28T11:00_2022-10-28T12:00_2022-10-28T16:25",
     "status" : "FAILED",
     "duration" : 5052,
     "errorMsg" : "java.lang.ExceptionInInitializerError\n\tat org.apache.druid.indexing.common.task.HadoopIndexTask$Hado...",
     "location" : {
       "host" : null,
       "port" : -1,
       "tlsPort" : -1
     }
   }
   ```


-- 
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@druid.apache.org.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1324623616

   will be fixed in 25.0 via https://github.com/apache/druid/pull/13371. My fix was only a partial fix. 


-- 
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@druid.apache.org

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


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


[GitHub] [druid] cryptoe commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
cryptoe commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1296814073

   I think it's related to this: https://github.com/apache/druid/pull/13138
   We are doing a 24.0.1 release which will backport this fix. So you can do 3 things:
   1. Either wait for the 24.0.1 release which will come in 2 weeks tops.
   2. Or backport his patch to your internal repo and build out the druid package.
   3. Downgrade to 0.23.0


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1305122479

   https://github.com/apache/druid/pull/13318 should fix it. 


-- 
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@druid.apache.org

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


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


[GitHub] [druid] raisonqyc commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
raisonqyc commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1301844403

   "hadoopDependencyCoordinates"  may not be set


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 closed issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 closed issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail
URL: https://github.com/apache/druid/issues/13276


-- 
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@druid.apache.org

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


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


[GitHub] [druid] cryptoe commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
cryptoe commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1302042779

   @599166320 Could you try a build from the latest master and see if the issue is still there ?


-- 
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@druid.apache.org

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


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


[GitHub] [druid] 599166320 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
599166320 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1304836801

   > @599166320 Could you try a build from the latest master and see if the issue is still there ?
   
   The latest master also has the same exception.


-- 
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@druid.apache.org

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


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


[GitHub] [druid] 599166320 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
599166320 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1318577074

   I have updated the modifications of #13318 and #13138 to 24.0.1-rc, but the same problem still occurs


-- 
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@druid.apache.org

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


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


[GitHub] [druid] 599166320 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
599166320 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1302025789

   > I think it's related to this: #13138 We are doing a 24.0.1 release which will backport this fix. So you can do 3 things:
   > 
   > 1. Either wait for the 24.0.1 release which will come in 2 weeks tops.
   > 2. Or backport his patch to your internal repo and build out the druid package.
   > 3. Downgrade to 0.23.0
   
   I also found this problem in the test cluster. I upgraded two classes according to #13138 : `HadoopIndexTask` and `HadoopTask`, and still reported an error.
   


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1305114640

   I think it's unrelated to https://github.com/apache/druid/pull/13138 and needs a fix in `MSQIndexingModule`.  For now, you can remove the `multi-stage-query-extension` from load list and the upgrade should work just fine. 
   
   


-- 
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@druid.apache.org

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


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


[GitHub] [druid] abhishekagarwal87 commented on issue #13276: upgrade to 24.0.0 from 0.23.0, Hadoop index task fail

Posted by GitBox <gi...@apache.org>.
abhishekagarwal87 commented on issue #13276:
URL: https://github.com/apache/druid/issues/13276#issuecomment-1318585002

   can you share the logs from the patched services that are still throwing the exception? 


-- 
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@druid.apache.org

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


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