Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-28756

RegionSizeCalculator ignored the size of memstore, which leads Spark miss data

    XMLWordPrintableJSON

Details

    Description

      RegionSizeCalculator only considers the size of StoreFile and ignores the size of MemStore. For a new region that has only been written to MemStore and has not been flushed, will consider its size to be 0.

      When we use TableInputFormat to read HBase table data in Spark.

      spark.sparkContext.newAPIHadoopRDD(
          conf,
          classOf[TableInputFormat],
          classOf[ImmutableBytesWritable],
          classOf[Result])
      }

      Spark defaults to ignoring empty InputSplits, which is determined by the configurationĀ  "spark.hadoopRDD.ignoreEmptySplits".

      private[spark] val HADOOP_RDD_IGNORE_EMPTY_SPLITS =
        ConfigBuilder("spark.hadoopRDD.ignoreEmptySplits")
          .internal()
          .doc("When true, HadoopRDD/NewHadoopRDD will not create partitions for empty input splits.")
          .version("2.3.0")
          .booleanConf
          .createWithDefault(true) 

      The above reasons lead to Spark missing data. So we should consider both the size of the StoreFile and the MemStore in the RegionSizeCalculator.

      Attachments

        Issue Links

          Activity

            People

              Ddupg Sun Xin
              Ddupg Sun Xin
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: