Uploaded image for project: 'Apache Drill'
  1. Apache Drill
  2. DRILL-1172

Unable to query Hive tables from Drill on CDH4

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 0.4.0
    • Storage - Hive
    • None
    • CDH4, CDH5, vanilla hadoop

    Description

      I am running Drill on CDH4.
      I have created a Hive Storage plugin as follows:

      {
        "type" : "hive",
        "enabled" : true,
        "configProps" : {
          "hive.metastore.uris" : "thrift://10.10.30.156:9083",
          "fs.default.name" : "hdfs://10.10.30.156:8020/",
          "hive.metastore.sasl.enabled" : "false"
        }
      }
      

      I connect to sqlline as follows:
      /opt/drill/apache-drill-1.0.0-m2-incubating-SNAPSHOT/bin/sqlline -u "jdbc:drill:schema=hive;zk=10.10.30.156:2181"

      When I run a query against a hive table it errors out as follows:

      0: jdbc:drill:schema=hive> select * from `student`;
      Query failed: org.apache.drill.exec.rpc.RpcException: Remote failure while running query.[error_id: "1a2b3324-66bf-46cd-8353-2c5a7d52c2aa"
      endpoint {
        address: "perfnode156.perf.lab"
        user_port: 31010
        control_port: 31011
        data_port: 31012
      }
      error_type: 0
      message: "Failure while setting up Foreman. < AssertionError:[ Internal error: Error while applying rule DrillPushProjIntoScan, args [rel#5:ProjectRel.NONE.ANY([]).[](child=rel#4:Subset#0.ENUMERABLE.ANY([]).[],rownum=$0,name=$1,age=$2,gpa=$3,studentnum=$4), rel#2:EnumerableTableAccessRel.ENUMERABLE.ANY([]).[](table=[hive, student])] ] < IllegalArgumentException:[ Wrong FS: hdfs://perfnode156.perf.lab:8020/user/hive/warehouse/student, expected: file:/// ]"
      ]
      Error: exception while executing query (state=,code=0)
      

      I have even tried setting the below property in the hive storage plugin but it still errors out:
      "fs.defaultFS" : "hdfs://10.10.30.156:8020/"

      The exception in the drillbit.log is

      2014-07-22 23:46:16,056 [4454e978-5f18-440e-97c8-1f01cf4475cf:foreman] ERROR o.a.drill.exec.work.foreman.Foreman - Error 07265edc-756e-4aa2-9bef-a244501a072e: Failure while setting up Foreman.
      java.lang.IllegalArgumentException: Wrong FS: hdfs://perfnode156.perf.lab:8020/user/hive/warehouse/student, expected: file:///
      	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:627) ~[hadoop-common-2.0.0-cdh4.7.0.jar:na]
      	at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:69) ~[hadoop-common-2.0.0-cdh4.7.0.jar:na]
      	at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:464) ~[hadoop-common-2.0.0-cdh4.7.0.jar:na]
      	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:380) ~[hadoop-common-2.0.0-cdh4.7.0.jar:na]
      	at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1380) ~[hadoop-common-2.0.0-cdh4.7.0.jar:na]
      	at org.apache.drill.exec.store.hive.HiveScan.getSplits(HiveScan.java:164) ~[drill-storage-hive-core-1.0.0-m2-incubating-SNAPSHOT.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.store.hive.HiveScan.<init>(HiveScan.java:121) ~[drill-storage-hive-core-1.0.0-m2-incubating-SNAPSHOT.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.store.hive.HiveStoragePlugin.getPhysicalScan(HiveStoragePlugin.java:75) ~[drill-storage-hive-core-1.0.0-m2-incubating-SNAPSHOT.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.store.hive.HiveStoragePlugin.getPhysicalScan(HiveStoragePlugin.java:39) ~[drill-storage-hive-core-1.0.0-m2-incubating-SNAPSHOT.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.store.AbstractStoragePlugin.getPhysicalScan(AbstractStoragePlugin.java:53) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.planner.logical.DrillTable.getGroupScan(DrillTable.java:54) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.planner.logical.DrillPushProjIntoScan.onMatch(DrillPushProjIntoScan.java:53) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.eigenbase.relopt.volcano.VolcanoRuleCall.onMatch(VolcanoRuleCall.java:221) ~[optiq-core-0.7-20140710.204128-10.jar:na]
      	at org.eigenbase.relopt.volcano.VolcanoPlanner.findBestExp(VolcanoPlanner.java:653) ~[optiq-core-0.7-20140710.204128-10.jar:na]
      	at net.hydromatic.optiq.prepare.PlannerImpl.transform(PlannerImpl.java:271) ~[optiq-core-0.7-20140710.204128-10.jar:na]
      	at org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.convertToDrel(DefaultSqlHandler.java:136) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler.getPlan(DefaultSqlHandler.java:116) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.planner.sql.DrillSqlWorker.getPlan(DrillSqlWorker.java:129) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.work.foreman.Foreman.runSQL(Foreman.java:400) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.work.foreman.Foreman.run(Foreman.java:216) ~[drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:242) [drill-java-exec-1.0.0-m2-incubating-SNAPSHOT-rebuffed.jar:1.0.0-m2-incubating-SNAPSHOT]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_55]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_55]
      	at java.lang.Thread.run(Thread.java:744) [na:1.7.0_55]
      

      In the incubator-drill/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveScan.java class, the error is occurring because the FileSystem object being created in the getSplits() method is pointing to file:///
      This FileSystem object is being created from a JobConf object.
      The contents of the JobConf object are:
      Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml

      The fs.default.name is set to file:/// in /opt/cloudera/parcels/CDH-4.7.0-1.cdh4.7.0.p0.40/share/doc/hadoop-2.0.0+1604/hadoop-project-dist/hadoop-common/core-default.xml.
      However changing that to hdfs://10.10.30.156:8020/ doesn't make any difference.

      Attachments

        Issue Links

          Activity

            People

              DrillCommitter DrillCommitter
              amitskatti Amit Katti
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: