Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-2609

NPE when pruning partitions by thrift method get_partitions_by_filter

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 0.7.1
    • None
    • Metastore
    • None

    Description

      It's a datanucleus bug indeed.

      try this code:

      boolean open = false;
      for (int i = 0; i < 5 && !open; ++i) {
        try {
          transport.open();
          open = true;
        } catch (TTransportException e) {
          System.out.println("failed to connect to MetaStore, re-trying...");
          try {
            Thread.sleep(1000);
          } catch (InterruptedException ignore) {}
        }
      }
      
      try {
        List<Partition> parts =
            client.get_partitions_by_filter("default", "partitioned_nation",
                "pt < '2'", (short) -1);
        for (Partition part : parts) {
          System.out.println(part.getSd().getLocation());
        }
      } catch (Exception te) {
        te.printStackTrace();
      }
      

      A NPEexception would be thrown on the thrift server side

      11/11/25 13:11:55 ERROR api.ThriftHiveMetastore$Processor: Internal error processing get_partitions_by_filter
      java.lang.NullPointerException
              at org.datanucleus.store.mapped.mapping.MappingHelper.getMappingIndices(MappingHelper.java:35)
              at org.datanucleus.store.mapped.expression.StatementText.applyParametersToStatement(StatementText.java:194)
              at org.datanucleus.store.rdbms.query.RDBMSQueryUtils.getPreparedStatementForQuery(RDBMSQueryUtils.java:233)
              at org.datanucleus.store.rdbms.query.legacy.SQLEvaluator.evaluate(SQLEvaluator.java:115)
              at org.datanucleus.store.rdbms.query.legacy.JDOQLQuery.performExecute(JDOQLQuery.java:288)
              at org.datanucleus.store.query.Query.executeQuery(Query.java:1657)
              at org.datanucleus.store.rdbms.query.legacy.JDOQLQuery.executeQuery(JDOQLQuery.java:245)
              at org.datanucleus.store.query.Query.executeWithMap(Query.java:1526)
              at org.datanucleus.jdo.JDOQuery.executeWithMap(JDOQuery.java:334)
              at org.apache.hadoop.hive.metastore.ObjectStore.listMPartitionsByFilter(ObjectStore.java:1329)
              at org.apache.hadoop.hive.metastore.ObjectStore.getPartitionsByFilter(ObjectStore.java:1241)
              at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler$40.run(HiveMetaStore.java:2369)
              at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler$40.run(HiveMetaStore.java:2366)
              at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.executeWithRetry(HiveMetaStore.java:307)
              at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.get_partitions_by_filter(HiveMetaStore.java:2366)
              at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Processor$get_partitions_by_filter.process(ThriftHiveMetastore.j
      ava:6099)
              at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Processor.process(ThriftHiveMetastore.java:4789)
              at org.apache.hadoop.hive.metastore.HiveMetaStore$TLoggingProcessor.process(HiveMetaStore.java:3167)
              at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:253)
              at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
              at java.lang.Thread.run(Thread.java:662)
      

      A null JavaTypeMapping was passed into org.datanucleus.store.mapped.mapping.MappingHelper.(int initialPosition, JavaTypeMapping mapping), that caused NPE.

      After digged into the datanucleus source, I found that the null value was born in the constructor of org.datanucleus.store.mapped.expression.SubstringExpression. see

          /**
           * Constructs the substring
           * @param str the String Expression
           * @param begin The start position
           * @param end The end position expression
           **/   
          public SubstringExpression(StringExpression str, NumericExpression begin, NumericExpression end)
          {
              super(str.getQueryExpression());
      
              st.append("SUBSTRING(").append(str).append(" FROM ")
                  .append(begin.add(new IntegerLiteral(qs, mapping, BigInteger.ONE)))
                  .append(" FOR ").append(end.sub(begin)).append(')');
          }
      

      The field mapping hasn't been instanced at that moment.

      How do you deal with such a external bug?

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              coderplay Min Zhou
              Votes:
              0 Vote for this issue
              Watchers:
              11 Start watching this issue

              Dates

                Created:
                Updated: