You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "harmeeksinghbedi (via GitHub)" <gi...@apache.org> on 2023/03/26 01:43:27 UTC

[GitHub] [beam] harmeeksinghbedi opened a new issue, #25981: [Bug]: Crash when running SQLTransform query

harmeeksinghbedi opened a new issue, #25981:
URL: https://github.com/apache/beam/issues/25981

   ### What happened?
   
   Running SQL query : SELECT t1.productname, t1.price, t1.inventorycount , t2.customerid , t2.customername, t2.customeraddress, t2.cdate from FACT_TABLE t1 JOIN LOOKUP_TABLE t2 on t1.productname=t2.productname WHERE t1.productname IS NOT NULL  Where LOOKUP_TABLE is using BeamSqlSeekableTable  results in a crash in "apache calcite" JaninoRelMetadataProvider class (This issue is reproducible in apache calicite 0.28 and below , and does not exisit above 0.28). 
   
   All apache beam releases are using apache calcite 0.28 and lower. The reason for this bug is as follows 
   JaninoRelMetadataProvider in branch calcite 0.28.0 in function has following code 
   
   static  <MH extends MetadataHandler<?>> MH compile(String className,
         String classBody, Class<MH> handlerClass,
         List<Object> argList) throws CompileException, IOException {
       final ICompilerFactory compilerFactory;
       ClassLoader classLoader =
           Objects.requireNonNull(JaninoRelMetadataProvider.class.getClassLoader(), "classLoader");
       try {
         compilerFactory = CompilerFactoryFactory.getDefaultCompilerFactory(classLoader);
       } catch (Exception e) {
         throw new IllegalStateException(
             "Unable to instantiate java compiler", e);
       }
   
       final ISimpleCompiler compiler = compilerFactory.newSimpleCompiler();
       compiler.setParentClassLoader(JaninoRexCompiler.class.getClassLoader());
   
       final String s = "public final class " + className
           + " implements " + **handlerClass.getCanonicalName()** + " {\n"
           + classBody
           + "\n"
           + "}";
    
   The problamatic line is handlerClass.getCanonicalName() where nested java classes such as "org.apache.calcite.rel.metadata.BuiltInMetadata$NonCumulativeCost$Handler would wrongly be written as. "org.apache.calcite.rel.metadata.BuiltInMetadata.NonCumulativeCost.Handler" leading to classloader loading issues  . 
   
   Version of Beam : 2.46.0 used for tests ,  Java beam sdk was used . Other queries also failed with similar results. 
   Apache calcite 0.29.0 addresses this issue . 
   
    
   
   ### Issue Priority
   
   Priority: 2 (default / most bugs should be filed as P2)
   
   ### Issue Components
   
   - [ ] Component: Python SDK
   - [X] Component: Java SDK
   - [ ] Component: Go SDK
   - [ ] Component: Typescript SDK
   - [ ] Component: IO connector
   - [ ] Component: Beam examples
   - [ ] Component: Beam playground
   - [ ] Component: Beam katas
   - [ ] Component: Website
   - [ ] Component: Spark Runner
   - [ ] Component: Flink Runner
   - [ ] Component: Samza Runner
   - [ ] Component: Twister2 Runner
   - [ ] Component: Hazelcast Jet Runner
   - [ ] Component: Google Cloud Dataflow Runner


-- 
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: github-unsubscribe@beam.apache.org.apache.org

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


[GitHub] [beam] harmeeksinghbedi commented on issue #25981: [Bug]: Crash when running SQLTransform query

Posted by "harmeeksinghbedi (via GitHub)" <gi...@apache.org>.
harmeeksinghbedi commented on issue #25981:
URL: https://github.com/apache/beam/issues/25981#issuecomment-1483968743

   Here is the wrong example of the class being generated by Janino. 
   
   -----> Class generated 
   **public final class GeneratedMetadata_NonCumulativeCostHandler implements org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.BuiltInMetadata.NonCumulativeCost.Handler**
   {
     public final org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner$NonCumulativeCostImpl provider0;
     public final org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMdPercentageOriginalRows$RelMdNonCumulativeCost provider1;
     public GeneratedMetadata_NonCumulativeCostHandler(
         org.apache.beam.sdk.extensions.sql.impl.CalciteQueryPlanner$NonCumulativeCostImpl provider0,
         org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMdPercentageOriginalRows$RelMdNonCumulativeCost provider1) {
       this.provider0 = provider0;
       this.provider1 = provider1;
     }
     public org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.MetadataDef getDef() {
       return provider0.getDef();
     }
   
     private final Object methodKey0 = new org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.janino.DescriptiveCacheKey("public abstract org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.BuiltInMetadata$NonCumulativeCost$Handler.getNonCumulativeCost(org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode,org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery)");
     public org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost getNonCumulativeCost(
         org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.RelNode r,
         org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.RelMetadataQuery mq) {
       while (r instanceof org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.DelegatingMetadataRel) {
         r = ((org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.DelegatingMetadataRel) r).getMetadataDelegateRel();
       }
       final java.util.List key = org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.runtime.FlatLists.of(methodKey0);
       final Object v = mq.map.get(r, key);
       if (v != null) {
         if (v == org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.NullSentinel.ACTIVE) {
           throw new org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.CyclicMetadataException();
         }  
         if (v == org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.NullSentinel.INSTANCE) {
           return null;
         }      
         return (org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost) v;
       }
       mq.map.put(r, key,org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.NullSentinel.ACTIVE);
       try {
         final org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.plan.RelOptCost x = getNonCumulativeCost_(r, mq);
         mq.map.put(r, key, org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.metadata.NullSentinel.mask(x));
         return x;
       } catch (java.lang.Exception e) {
         mq.map.row(r).clear();
         throw e;
   


-- 
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: github-unsubscribe@beam.apache.org

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


[GitHub] [beam] harmeeksinghbedi closed issue #25981: [Bug]: Crash when running SQLTransform query

Posted by "harmeeksinghbedi (via GitHub)" <gi...@apache.org>.
harmeeksinghbedi closed issue #25981: [Bug]: Crash when running SQLTransform query 
URL: https://github.com/apache/beam/issues/25981


-- 
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: github-unsubscribe@beam.apache.org

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