@Override public String toString() { return String.format("BasicStats: %d, %d %s", getNumRows(), getDataSize(), getState()); } }
public BasicStats build(Partish p) { BasicStats ret = new BasicStats(p); for (IStatsEnhancer enhancer : enhancers) { ret.apply(enhancer); } return ret; }
public BasicStats(Partish p) { partish = p; rowCount = parseLong(StatsSetupConst.ROW_COUNT); rawDataSize = parseLong(StatsSetupConst.RAW_DATA_SIZE); totalSize = parseLong(StatsSetupConst.TOTAL_SIZE); currentNumRows = rowCount; currentDataSize = rawDataSize; if (currentNumRows > 0) { state = State.COMPLETE; } else { state = State.NONE; } }
@Override public void apply(BasicStats stats) { if (stats.getNumRows() == 0 || stats.getNumRows() == -1) { stats.setNumRows(1); } } }
long ds = basicStats.getDataSize(); long nr = basicStats.getNumRows(); List<ColStatistics> colStats = Lists.newArrayList(); partStats.add(basicStats); BasicStats bbs = BasicStats.buildFrom(partStats); rowCounts.add(basicStats.getNumRows()); long nr = bbs.getNumRows(); long ds = bbs.getDataSize(); stats.setBasicStatsState(bbs.getState()); if (nr > 0) { if (State.PARTIAL.morePreciseThan(bbs.getState())) { stats.setBasicStatsState(State.PARTIAL);
@Override public void apply(BasicStats stats) { // FIXME: there were different logic for part/table; merge these logics later if (stats.partish.getPartition() == null) { if (stats.getNumRows() < 0 && avgRowSize > 0) { stats.setNumRows(stats.getDataSize() / avgRowSize); } } else { if (avgRowSize > 0) { long rc = stats.getNumRows(); long s = stats.getDataSize(); if (rc <= 0 && s > 0) { rc = s / avgRowSize; stats.setNumRows(rc); } if (s <= 0 && rc > 0) { s = StatsUtils.safeMult(rc, avgRowSize); stats.setDataSize(s); } } } if (stats.getNumRows() > 0) { // FIXME: this promotion process should be removed later if (State.PARTIAL.morePreciseThan(stats.state)) { stats.state = State.PARTIAL; } } } }
BasicStats bStats = new BasicStats(pi); long nr = bStats.getNumRows(); BasicStats aggregateStat = BasicStats.buildFrom(results); aggregateStat.apply(new BasicStats.SetMinRowNumber01()); return aggregateStat.getNumRows();
@Test public void mergeWithEmpty() { HiveConf conf = new HiveConf(); int avgRowSize = 100; int r0 = 13; int r1 = 15; int deserFactor = (int) conf.getFloatVar(ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR); Partish p0 = new LocalPartishBuilder().numRows(r0).rawDataSize(avgRowSize * r0).buildPartition(); Partish p1 = new LocalPartishBuilder().totalSize(r1 * avgRowSize / deserFactor).buildPartition(); BasicStats.Factory factory = new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf), new BasicStats.RowNumEstimator(avgRowSize)); BasicStats bs0 = factory.build(p0); BasicStats bs1 = factory.build(p1); BasicStats res = BasicStats.buildFrom(Lists.newArrayList(bs0, bs1)); assertEquals(r0 + r1, res.getNumRows()); assertEquals(avgRowSize * (r0 + r1), res.getDataSize()); }
@Test @Ignore("HIVE-18062 will fix this") public void mergedKeepsPartialStateEvenIfValuesAreSuccessfullyEstimated() { Partish p0 = new LocalPartishBuilder().numRows(10).rawDataSize(100).buildPartition(); Partish p1 = new LocalPartishBuilder().totalSize(10).buildPartition(); HiveConf conf = new HiveConf(); BasicStats.Factory factory = new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf), new BasicStats.RowNumEstimator(10)); BasicStats bs0 = factory.build(p0); BasicStats bs1 = factory.build(p1); BasicStats res = BasicStats.buildFrom(Lists.newArrayList(bs0, bs1)); assertEquals(State.PARTIAL, res.getState()); }
@Override public void apply(BasicStats stats) { long ds = stats.getRawDataSize(); if (ds <= 0) { ds = stats.getTotalSize(); // if data size is still 0 then get file size if (ds <= 0) { Path path = stats.partish.getPath(); try { ds = getFileSizeForPath(path); } catch (IOException e) { ds = 0L; } } ds = (long) (ds * deserFactor); stats.setDataSize(ds); } }
@Test public void testDataSizeEstimator() { Partish p1 = new LocalPartishBuilder().totalSize(10).buildPartition(); HiveConf conf = new HiveConf(); conf.setFloatVar(ConfVars.HIVE_STATS_DESERIALIZATION_FACTOR, 13.0f); BasicStats.Factory factory = new BasicStats.Factory(new BasicStats.DataSizeEstimator(conf)); BasicStats res = factory.build(p1); assertEquals(130, res.getDataSize()); }
public static BasicStats buildFrom(List<BasicStats> partStats) { return new BasicStats(partStats); }
@Override public void apply(BasicStats stats) { if (stats.getNumRows() == 0) { stats.setNumRows(1); } } }
public BasicStats(List<BasicStats> partStats) { partish = null; List<Long> nrIn = Lists.newArrayList(); List<Long> dsIn = Lists.newArrayList(); state = (partStats.size() == 0) ? State.COMPLETE : null; for (BasicStats ps : partStats) { nrIn.add(ps.getNumRows()); dsIn.add(ps.getDataSize()); if (state == null) { state = ps.getState(); } else { state = state.merge(ps.getState()); } } currentNumRows = StatsUtils.getSumIgnoreNegatives(nrIn); currentDataSize = StatsUtils.getSumIgnoreNegatives(dsIn); }