Uploaded image for project: 'IMPALA'
  1. IMPALA
  2. IMPALA-7044

int32 overflow in HdfsTableSink::CreateNewTmpFile()

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • Impala 2.9.0, Impala 2.10.0, Impala 2.11.0, Impala 3.0, Impala 2.12.0, Impala 2.13.0
    • Impala 3.0, Impala 2.13.0
    • Backend
    • ghx-label-6

    Description

      When writing Parquet files we compute a minimum block size based on the number of columns in the target table in hdfs-parquet-table-writer.cc:916:

      3 * DEFAULT_DATA_PAGE_SIZE * columns_.size();
      

      For tables with a large number of columns (> ~10k), this value will get larger than 2GB. When we pass it to hdfsOpenFile() in HdfsTableSink::CreateNewTmpFile() it gets cast to a signed int32 and can overflow.

      This leads to error messages like the following:

      I0516 16:13:52.755090 24257 status.cc:125] Failed to open HDFS file for writing: hdfs://localhost:20500/test-warehouse/lv.db/a/_impala_insert_staging/3c417cb973b710ab_803e898000000000/.3c417cb973b710ab-80
      3e898000000000_411033576_dir/3c417cb973b710ab-803e898000000000_271567064_data.0.parq
      Error(255): Unknown error 255
      Root cause: RemoteException: Specified block size is less than configured minimum value (dfs.namenode.fs-limits.min-block-size): -1935671296 < 1024
              at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2417)
              at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2339)
              at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:764)
              at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:451)
              at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:523)
              at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:991)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:869)
              at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:815)
              at java.security.AccessController.doPrivileged(Native Method)
              at javax.security.auth.Subject.doAs(Subject.java:422)
              at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1962)
              at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2675)
      
          @          0x187b8b3  impala::Status::Status()
          @          0x1fade89  impala::HdfsTableSink::CreateNewTmpFile()
          @          0x1faeee7  impala::HdfsTableSink::InitOutputPartition()
          @          0x1fb1389  impala::HdfsTableSink::GetOutputPartition()
          @          0x1faf34a  impala::HdfsTableSink::Send()
          @          0x1c91bcd  impala::FragmentInstanceState::ExecInternal()
          @          0x1c8efa5  impala::FragmentInstanceState::Exec()
          @          0x1c9e53f  impala::QueryState::ExecFInstance()
          @          0x1c9cdb2  _ZZN6impala10QueryState15StartFInstancesEvENKUlvE_clEv
          @          0x1c9f25d  _ZN5boost6detail8function26void_function_obj_invoker0IZN6impala10QueryState15StartFInstancesEvEUlvE_vE6invokeERNS1_15function_bufferE
          @          0x1bd6cd4  boost::function0<>::operator()()
          @          0x1ec18f9  impala::Thread::SuperviseThread()
          @          0x1ec9a95  boost::_bi::list5<>::operator()<>()
          @          0x1ec99b9  boost::_bi::bind_t<>::operator()()
          @          0x1ec997c  boost::detail::thread_data<>::run()
          @          0x31a527a  thread_proxy
          @     0x7f30246a8184  start_thread
          @     0x7f30243d503d  clone
      

      The signature of hdfsOpenFile() is as follows:

      hdfsFile hdfsOpenFile(hdfsFS fs, const char* path, int flags, int bufferSize, short replication, tSize blocksize);
      

      tSize is typedef'd to int32_t.

      The comment of hdfsOpenFile() is explicit about this:

      @param blocksize Size of block - pass 0 if you want to use the
      default configured values.  Note that if you want a block size bigger
      than 2 GB, you must use the hdfsStreamBuilder API rather than this
      deprecated function.
      

      If using hdfsStreamBuilder is not an option, we should be able to cap the blocksize to 2GB (or smaller values). It might result in a suboptimal storage layout, but will preserve correctness.

      An alternative would be to cap the maximum number of columns. In either case we should be explicit about the signed overflow, as it results in undefined behavior.

      I'm attaching a SQL file which creates a table with 11k columns and inserts a row.

      Attachments

        1. ct.sql
          270 kB
          Lars Volker

        Activity

          People

            lv Lars Volker
            lv Lars Volker
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: