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/04/13 06:51:23 UTC

[GitHub] [pulsar] eolivelli opened a new issue #10210: Schema: "IllegalReferenceCountException: refCnt: 0" while getting schema with AutoConsumeSchema

eolivelli opened a new issue #10210:
URL: https://github.com/apache/pulsar/issues/10210


   **Describe the bug**
   I am observing this error while working on new tests cases about AutoConsumeSchema.
   I am still investigating in order to find a simple way to reproduce the problem
   
   ```
   persistent://public/default/sink-created-test-kv-sink-input-kvavro-yyhymzgt] failed to get schema : io.netty.util.IllegalReferenceCountException: refCnt: 0
   java.util.concurrent.CompletionException: io.netty.util.IllegalReferenceCountException: refCnt: 0
   	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:273) ~[?:1.8.0_282]
   	at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1005) ~[?:1.8.0_282]
   	at java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2137) ~[?:1.8.0_282]
   	at org.apache.pulsar.client.impl.ClientCnx.sendGetSchema(ClientCnx.java:879) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.BinaryProtoLookupService.lambda$getSchema$8(BinaryProtoLookupService.java:227) ~[java-instance.jar:?]
   	at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670) ~[?:1.8.0_282]
   	at java.util.concurrent.CompletableFuture.uniAcceptStage(CompletableFuture.java:683) ~[?:1.8.0_282]
   	at java.util.concurrent.CompletableFuture.thenAccept(CompletableFuture.java:2010) ~[?:1.8.0_282]
   	at org.apache.pulsar.client.impl.BinaryProtoLookupService.getSchema(BinaryProtoLookupService.java:223) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider.loadSchema(MultiVersionSchemaInfoProvider.java:97) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider.access$000(MultiVersionSchemaInfoProvider.java:41) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider$1.load(MultiVersionSchemaInfoProvider.java:54) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider$1.load(MultiVersionSchemaInfoProvider.java:51) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache.get(LocalCache.java:3951) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3974) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4935) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionSchemaInfoProvider.getSchemaByVersion(MultiVersionSchemaInfoProvider.java:75) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.KeyValueSchema$2.getSchemaByVersion(KeyValueSchema.java:221) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.reader.AbstractMultiVersionReader.getSchemaInfoByVersion(AbstractMultiVersionReader.java:115) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.generic.MultiVersionGenericAvroReader.loadReader(MultiVersionGenericAvroReader.java:43) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.reader.AbstractMultiVersionReader$1.load(AbstractMultiVersionReader.java:52) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.reader.AbstractMultiVersionReader$1.load(AbstractMultiVersionReader.java:49) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3529) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2278) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2155) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2045) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache.get(LocalCache.java:3951) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3974) ~[java-instance.jar:?]
   	at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4935) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.reader.AbstractMultiVersionReader.read(AbstractMultiVersionReader.java:86) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.AbstractStructSchema.decode(AbstractStructSchema.java:60) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.KeyValueSchema.decode(KeyValueSchema.java:160) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.KeyValueSchema.lambda$decode$0(KeyValueSchema.java:151) ~[java-instance.jar:?]
   	at org.apache.pulsar.common.schema.KeyValue.decode(KeyValue.java:146) [java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.KeyValueSchema.decode(KeyValueSchema.java:151) [java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.KeyValueSchema.decode(KeyValueSchema.java:37) [java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.AutoConsumeSchema.decode(AutoConsumeSchema.java:103) [java-instance.jar:?]
   	at org.apache.pulsar.client.impl.schema.AutoConsumeSchema.decode(AutoConsumeSchema.java:41) [java-instance.jar:?]
   	at org.apache.pulsar.client.impl.MessageImpl.getValue(MessageImpl.java:356) [java-instance.jar:?]
   	at org.apache.pulsar.functions.source.PulsarRecord.getValue(PulsarRecord.java:81) [org.apache.pulsar-pulsar-functions-instance-2.8.0-SNAPSHOT.jar:2.8.0-SNAPSHOT]
   	at org.apache.pulsar.functions.instance.JavaInstanceRunnable.readInput(JavaInstanceRunnable.java:375) [org.apache.pulsar-pulsar-functions-instance-2.8.0-SNAPSHOT.jar:?]
   	at org.apache.pulsar.functions.instance.JavaInstanceRunnable.run(JavaInstanceRunnable.java:239) [org.apache.pulsar-pulsar-functions-instance-2.8.0-SNAPSHOT.jar:?]
   	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_282]
   Caused by: io.netty.util.IllegalReferenceCountException: refCnt: 0
   	at io.netty.buffer.AbstractByteBuf.ensureAccessible(AbstractByteBuf.java:1454) ~[java-instance.jar:?]
   	at io.netty.buffer.AbstractByteBuf.checkIndex(AbstractByteBuf.java:1383) ~[java-instance.jar:?]
   	at io.netty.buffer.UnsafeByteBufUtil.getBytes(UnsafeByteBufUtil.java:481) ~[java-instance.jar:?]
   	at io.netty.buffer.PooledUnsafeDirectByteBuf.getBytes(PooledUnsafeDirectByteBuf.java:130) ~[java-instance.jar:?]
   	at io.netty.buffer.PooledSlicedByteBuf.getBytes(PooledSlicedByteBuf.java:235) ~[java-instance.jar:?]
   	at io.netty.buffer.ByteBufUtil.decodeString(ByteBufUtil.java:1147) ~[java-instance.jar:?]
   	at io.netty.buffer.AbstractByteBuf.toString(AbstractByteBuf.java:1246) ~[java-instance.jar:?]
   	at org.apache.pulsar.common.api.proto.LightProtoCodec.readString(LightProtoCodec.java:250) ~[java-instance.jar:?]
   	at org.apache.pulsar.common.api.proto.Schema.getName(Schema.java:102) ~[java-instance.jar:?]
   	at org.apache.pulsar.common.protocol.schema.SchemaInfoUtil.newSchemaInfo(SchemaInfoUtil.java:48) ~[java-instance.jar:?]
   	at org.apache.pulsar.client.impl.ClientCnx.lambda$sendGetSchema$12(ClientCnx.java:891) ~[java-instance.jar:?]
   	at java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:995) ~[?:1.8.0_282]
   	... 45 more
   ```
   


-- 
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] merlimat commented on issue #10210: Schema: "IllegalReferenceCountException: refCnt: 0" while getting schema with AutoConsumeSchema

Posted by GitBox <gi...@apache.org>.
merlimat commented on issue #10210:
URL: https://github.com/apache/pulsar/issues/10210#issuecomment-819086592


   I think the problem is that https://github.com/apache/pulsar/blob/c9c116767235491aca6f27ef7a07705666bbd647/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L879 is getting called potentially in a different thread, when the `request` buffer is going to be already released.
   
   In particular, due to lazy initialization of fields, calling `commandGetSchemaResponse.getSchema()` will trigger to read from the original buffer, and it causes the above error when the buffer is already gone.
   
   


-- 
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] eolivelli commented on issue #10210: Schema: "IllegalReferenceCountException: refCnt: 0" while getting schema with AutoConsumeSchema

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #10210:
URL: https://github.com/apache/pulsar/issues/10210#issuecomment-818510943


   @merlimat @codelipenghui @jerrypeng do you have cycles to take a look ?
   I hope I am wrong in my explanation


-- 
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] eolivelli commented on issue #10210: Schema: "IllegalReferenceCountException: refCnt: 0" while getting schema with AutoConsumeSchema

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #10210:
URL: https://github.com/apache/pulsar/issues/10210#issuecomment-818509839


   My understanding is that:
   - in ClientCnx#sendGetSchema we call `sendGetRawSchema`() and it returns a CompletableFuture 
   (https://github.com/apache/pulsar/blob/c9c116767235491aca6f27ef7a07705666bbd647/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L879)
   - in PulsarDecoder we receive the response from the server  and call ClientCnx#handleGetSchemaResponse ( https://github.com/apache/pulsar/blob/c12765a0530bcefca8e840d2f8f43383d7478778/pulsar-common/src/main/java/org/apache/pulsar/common/protocol/PulsarDecoder.java#L350 )
   - in PulsarDecoder we execute the "finally" block that `release` the ByteBuf that holds the message 
   - in ClientCnx#sendGetSchema  we enter the lambda function attached with "thenCompose" to the initial CF (https://github.com/apache/pulsar/blob/c9c116767235491aca6f27ef7a07705666bbd647/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ClientCnx.java#L891) but it is too late
   
   I saw this problem happening on CI, so the thread ClientCnx may have been so slow that it was possible for the "finally" block to be executed   


-- 
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] eolivelli closed issue #10210: Schema: "IllegalReferenceCountException: refCnt: 0" while getting schema with AutoConsumeSchema

Posted by GitBox <gi...@apache.org>.
eolivelli closed issue #10210:
URL: https://github.com/apache/pulsar/issues/10210


   


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