You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/02/19 13:30:56 UTC

[GitHub] [pulsar] gaoran10 opened a new pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

gaoran10 opened a new pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631


   ### Motivation
   
   Currently, the Pulsar SQL query bytes schema data will cause an error.
   
   *Reproduce*
   1. produce bytes schema data.
   2. query data by the Pulsar SQL.
   3. the error log could be seen.
   
   *Error log*
   ```
   com.google.common.util.concurrent.UncheckedExecutionException: java.nio.BufferUnderflowException
   	at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2051)
   	at com.google.common.cache.LocalCache.get(LocalCache.java:3951)
   	at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3974)
   	at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4935)
   	at org.apache.pulsar.sql.presto.PulsarSqlSchemaInfoProvider.getSchemaByVersion(PulsarSqlSchemaInfoProvider.java:76)
   	at org.apache.pulsar.sql.presto.PulsarRecordCursor.advanceNextPosition(PulsarRecordCursor.java:471)
   	at io.prestosql.spi.connector.RecordPageSource.getNextPage(RecordPageSource.java:90)
   	at io.prestosql.operator.TableScanOperator.getOutput(TableScanOperator.java:302)
   	at io.prestosql.operator.Driver.processInternal(Driver.java:379)
   	at io.prestosql.operator.Driver.lambda$processFor$8(Driver.java:283)
   	at io.prestosql.operator.Driver.tryWithLock(Driver.java:675)
   	at io.prestosql.operator.Driver.processFor(Driver.java:276)
   	at io.prestosql.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1075)
   	at io.prestosql.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)
   	at io.prestosql.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:484)
   	at io.prestosql.$gen.Presto_332__testversion____20210219_094906_2.run(Unknown Source)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: java.nio.BufferUnderflowException
   	at java.nio.Buffer.nextGetIndex(Buffer.java:509)
   	at java.nio.HeapByteBuffer.getLong(HeapByteBuffer.java:415)
   	at org.apache.pulsar.sql.presto.PulsarSqlSchemaInfoProvider.loadSchema(PulsarSqlSchemaInfoProvider.java:106)
   	at org.apache.pulsar.sql.presto.PulsarSqlSchemaInfoProvider.access$000(PulsarSqlSchemaInfoProvider.java:49)
   	at org.apache.pulsar.sql.presto.PulsarSqlSchemaInfoProvider$1.load(PulsarSqlSchemaInfoProvider.java:61)
   	at org.apache.pulsar.sql.presto.PulsarSqlSchemaInfoProvider$1.load(PulsarSqlSchemaInfoProvider.java:58)
   	at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529)
   	at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278)
   	at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155)
   	at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045)
   	... 18 more
   
   ```
   
   ### Modifications
   
   Add check for bytes schema, if the schema is bytes schema use the schema info of the bytes schema directly.
   
   ### Verifying this change
   
   Add a new integration test.
   
   ### Does this pull request potentially affect one of the following parts:
   
   *If `yes` was chosen, please highlight the changes*
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API: (no)
     - The schema: (no)
     - The default values of configurations: (no)
     - The wire protocol: (no)
     - The rest endpoints: (no)
     - The admin cli options: (no)
     - Anything that affects deployment: (no)
   
   


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] zymap commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-787811999


   This issue is introduced by #8422. So it not a bug fix for the 2.7.0. Removed the `release/2.7.1` label.


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r580848095



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -441,9 +442,11 @@ public boolean advanceNextPosition() {
         //start time for deseralizing record
         metricsTracker.start_RECORD_DESERIALIZE_TIME();
 
-        SchemaInfo schemaInfo;
+        SchemaInfo schemaInfo = getBytesSchemaInfo(pulsarSplit.getSchemaType(), pulsarSplit.getSchemaName());

Review comment:
       This method may cause the exception `java.nio.BufferUnderflowException`.
   
   ```
       private SchemaInfo loadSchema(BytesSchemaVersion bytesSchemaVersion) throws PulsarAdminException {
           ClassLoader originalContextLoader = Thread.currentThread().getContextClassLoader();
           try {
               Thread.currentThread().setContextClassLoader(InjectionManagerFactory.class.getClassLoader());
               return pulsarAdmin.schemas()
                       .getSchemaInfo(topicName.toString(), ByteBuffer.wrap(bytesSchemaVersion.get()).getLong());
           } finally {
               Thread.currentThread().setContextClassLoader(originalContextLoader);
           }
       }
   
   ```
   
   The schema version of `Schema.BYTES` is an empty bytes array. So `ByteBuffer.wrap(bytesSchemaVersion.get()).getLong()` will cause the exception `java.nio.BufferUnderflowException`.




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] codelipenghui commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r579880589



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -441,9 +442,11 @@ public boolean advanceNextPosition() {
         //start time for deseralizing record
         metricsTracker.start_RECORD_DESERIALIZE_TIME();
 
-        SchemaInfo schemaInfo;
+        SchemaInfo schemaInfo = getBytesSchemaInfo(pulsarSplit.getSchemaType(), pulsarSplit.getSchemaName());

Review comment:
       I noticed the current `getSchemaVersion` in the `RawMessageImpl` return null 
   
   ```
   @Override
       public byte[] getSchemaVersion() {
           if (msgMetadata != null && msgMetadata.getMetadata().hasSchemaVersion()) {
               return msgMetadata.getMetadata().getSchemaVersion();
           } else {
               return null;
           }
       }
   ```
   
   It should not throw `java.nio.BufferUnderflowException`. Could you please confirm here?




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] codelipenghui commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r581480479



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -441,9 +442,11 @@ public boolean advanceNextPosition() {
         //start time for deseralizing record
         metricsTracker.start_RECORD_DESERIALIZE_TIME();
 
-        SchemaInfo schemaInfo;
+        SchemaInfo schemaInfo = getBytesSchemaInfo(pulsarSplit.getSchemaType(), pulsarSplit.getSchemaName());

Review comment:
       I see.




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-785670580


   /pulsarbot run-failure-checks


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r580848095



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -441,9 +442,11 @@ public boolean advanceNextPosition() {
         //start time for deseralizing record
         metricsTracker.start_RECORD_DESERIALIZE_TIME();
 
-        SchemaInfo schemaInfo;
+        SchemaInfo schemaInfo = getBytesSchemaInfo(pulsarSplit.getSchemaType(), pulsarSplit.getSchemaName());

Review comment:
       This method may cause the exception `java.nio.BufferUnderflowException`.
   
   ```
   private SchemaInfo loadSchema(BytesSchemaVersion bytesSchemaVersion) throws PulsarAdminException {
       ClassLoader originalContextLoader = Thread.currentThread().getContextClassLoader();
       try {
           Thread.currentThread().setContextClassLoader(InjectionManagerFactory.class.getClassLoader());
           return pulsarAdmin.schemas()
                   .getSchemaInfo(topicName.toString(), ByteBuffer.wrap(bytesSchemaVersion.get()).getLong());
       } finally {
           Thread.currentThread().setContextClassLoader(originalContextLoader);
       }
   }
   ```
   
   The schema version of `Schema.BYTES` is an empty bytes array. So `ByteBuffer.wrap(bytesSchemaVersion.get()).getLong()` will cause the exception `java.nio.BufferUnderflowException`.




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-782871355


   /pulsarbot run-failure-checks


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] zymap merged pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
zymap merged pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631


   


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] zymap commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
zymap commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-784201426


   /pulsarbot run-failure-checks
   


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] sijie commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r579370010



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -560,6 +563,18 @@ public boolean advanceNextPosition() {
         return true;
     }
 
+    private SchemaInfo getBytesSchemaInfo(SchemaType schemaType, String schemaName) {
+        if (!schemaType.equals(SchemaType.BYTES) && !schemaType.equals(SchemaType.NONE)) {
+            return null;
+        }
+        if (schemaName.equals(Schema.BYTES.getSchemaInfo().getName())) {

Review comment:
       Can you compare SchemaType instead of comparing schemaName?




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-784900704


   /pulsarbot run-failure-checks


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r580735533



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -560,6 +563,18 @@ public boolean advanceNextPosition() {
         return true;
     }
 
+    private SchemaInfo getBytesSchemaInfo(SchemaType schemaType, String schemaName) {
+        if (!schemaType.equals(SchemaType.BYTES) && !schemaType.equals(SchemaType.NONE)) {
+            return null;
+        }
+        if (schemaName.equals(Schema.BYTES.getSchemaInfo().getName())) {

Review comment:
       Because of the schema types of the `Schema.BYTES ` and `Schema.BYTEBUFFER` all are `SchemaType.BYTES`, so I use the schema name to make the comparison.

##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -560,6 +563,18 @@ public boolean advanceNextPosition() {
         return true;
     }
 
+    private SchemaInfo getBytesSchemaInfo(SchemaType schemaType, String schemaName) {
+        if (!schemaType.equals(SchemaType.BYTES) && !schemaType.equals(SchemaType.NONE)) {
+            return null;
+        }
+        if (schemaName.equals(Schema.BYTES.getSchemaInfo().getName())) {

Review comment:
       The schema types of the `Schema.BYTES ` and `Schema.BYTEBUFFER` all are `SchemaType.BYTES`, so I use the schema name to make the comparison.




----------------------------------------------------------------
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.

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



[GitHub] [pulsar] gaoran10 commented on pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
gaoran10 commented on pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#issuecomment-782885559


   /pulsarbot run-failure-checks


----------------------------------------------------------------
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.

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



[GitHub] [pulsar] sijie commented on a change in pull request #9631: [Pulsar SQL] Fix Pulsar SQL query bytes schema data error

Posted by GitBox <gi...@apache.org>.
sijie commented on a change in pull request #9631:
URL: https://github.com/apache/pulsar/pull/9631#discussion_r580738041



##########
File path: pulsar-sql/presto-pulsar/src/main/java/org/apache/pulsar/sql/presto/PulsarRecordCursor.java
##########
@@ -560,6 +563,18 @@ public boolean advanceNextPosition() {
         return true;
     }
 
+    private SchemaInfo getBytesSchemaInfo(SchemaType schemaType, String schemaName) {
+        if (!schemaType.equals(SchemaType.BYTES) && !schemaType.equals(SchemaType.NONE)) {
+            return null;
+        }
+        if (schemaName.equals(Schema.BYTES.getSchemaInfo().getName())) {

Review comment:
       Okay. Make sense.




----------------------------------------------------------------
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.

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