42
42
import java .util .concurrent .Executors ;
43
43
import java .util .concurrent .atomic .AtomicInteger ;
44
44
import java .util .function .Function ;
45
+ import java .util .function .Predicate ;
45
46
import java .util .stream .Collectors ;
46
47
import java .util .stream .Stream ;
47
48
import org .apache .commons .collections .MapUtils ;
@@ -637,20 +638,26 @@ private boolean writeColStats(List<ColumnStatistics> colStats, Table tbl) {
637
638
long snapshotId = tbl .currentSnapshot ().snapshotId ();
638
639
long snapshotSequenceNumber = tbl .currentSnapshot ().sequenceNumber ();
639
640
640
- colStats .forEach (statsObj -> {
641
- byte [] serializeColStats = SerializationUtils .serialize (statsObj );
642
- puffinWriter .add (
643
- new Blob (
644
- ColumnStatisticsObj .class .getSimpleName (),
645
- ImmutableList .of (1 ),
646
- snapshotId ,
647
- snapshotSequenceNumber ,
648
- ByteBuffer .wrap (serializeColStats ),
649
- PuffinCompressionCodec .NONE ,
650
- ImmutableMap .of ("partition" ,
651
- String .valueOf (statsObj .getStatsDesc ().getPartName ()))
652
- ));
653
- });
641
+ colStats .forEach (stats -> {
642
+ boolean isTblLevel = stats .getStatsDesc ().isIsTblLevel ();
643
+
644
+ for (Serializable statsObj : (isTblLevel ) ? stats .getStatsObj () : Collections .singletonList (stats )) {
645
+ byte [] serializeColStats = SerializationUtils .serialize (statsObj );
646
+ puffinWriter .add (
647
+ new Blob (
648
+ ColumnStatisticsObj .class .getSimpleName (),
649
+ ImmutableList .of ((isTblLevel ) ? tbl .spec ().schema ().findField (
650
+ ((ColumnStatisticsObj ) statsObj ).getColName ()).fieldId () : 1 ),
651
+ snapshotId ,
652
+ snapshotSequenceNumber ,
653
+ ByteBuffer .wrap (serializeColStats ),
654
+ PuffinCompressionCodec .NONE ,
655
+ (isTblLevel ) ?
656
+ ImmutableMap .of ("specId" , String .valueOf (tbl .spec ().specId ())) :
657
+ ImmutableMap .of ("partition" , String .valueOf (stats .getStatsDesc ().getPartName ()))
658
+ ));
659
+ }});
660
+
654
661
puffinWriter .finish ();
655
662
656
663
statisticsFile =
@@ -693,17 +700,27 @@ private boolean canProvideColStats(Table table, long snapshotId) {
693
700
}
694
701
695
702
@ Override
696
- public List <ColumnStatisticsObj > getColStatistics (org .apache .hadoop .hive .ql .metadata .Table hmsTable ) {
703
+ public List <ColumnStatisticsObj > getColStatistics (org .apache .hadoop .hive .ql .metadata .Table hmsTable , List < String > colNames ) {
697
704
Table table = IcebergTableUtil .getTable (conf , hmsTable .getTTable ());
698
705
Snapshot snapshot = IcebergTableUtil .getTableSnapshot (table , hmsTable );
699
706
700
- ColumnStatistics emptyStats = new ColumnStatistics ();
701
707
if (snapshot != null ) {
702
- return IcebergTableUtil .getColStatsPath (table , snapshot .snapshotId ())
703
- .map (statsPath -> readColStats (table , statsPath , null ).get (0 ))
704
- .orElse (emptyStats ).getStatsObj ();
708
+
709
+ Predicate <BlobMetadata > filter ;
710
+ if (colNames != null ) {
711
+ Set <String > columns = Sets .newHashSet (colNames );
712
+ filter = metadata -> {
713
+ int specId = Integer .parseInt (metadata .properties ().get ("specId" ));
714
+ String column = table .specs ().get (specId ).schema ().findColumnName (metadata .inputFields ().get (0 ));
715
+ return columns .contains (column );
716
+ };
717
+ } else {
718
+ filter = null ;
719
+ }
720
+
721
+ return IcebergTableUtil .readColStats (table , snapshot .snapshotId (), filter );
705
722
}
706
- return emptyStats . getStatsObj ();
723
+ return Lists . newArrayList ();
707
724
}
708
725
709
726
@ Override
@@ -720,9 +737,10 @@ public AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table hms
720
737
MetastoreConf .ConfVars .STATS_NDV_DENSITY_FUNCTION );
721
738
double ndvTuner = MetastoreConf .getDoubleVar (getConf (), MetastoreConf .ConfVars .STATS_NDV_TUNER );
722
739
723
- List <ColumnStatistics > partStats = IcebergTableUtil .getColStatsPath (table , snapshot .snapshotId ())
724
- .map (statsPath -> readColStats (table , statsPath , Sets .newHashSet (partNames )))
725
- .orElse (Collections .emptyList ());
740
+ Set <String > partitions = Sets .newHashSet (partNames );
741
+ Predicate <BlobMetadata > filter = metadata -> partitions .contains (metadata .properties ().get ("partition" ));
742
+
743
+ List <ColumnStatistics > partStats = IcebergTableUtil .readColStats (table , snapshot .snapshotId (), filter );
726
744
727
745
partStats .forEach (colStats ->
728
746
colStats .getStatsObj ().removeIf (statsObj -> !colNames .contains (statsObj .getColName ())));
@@ -736,30 +754,6 @@ public AggrStats getAggrColStatsFor(org.apache.hadoop.hive.ql.metadata.Table hms
736
754
return new AggrStats (colStatsList , partStats .size ());
737
755
}
738
756
739
- private List <ColumnStatistics > readColStats (Table table , Path statsPath , Set <String > partNames ) {
740
- List <ColumnStatistics > colStats = Lists .newArrayList ();
741
-
742
- try (PuffinReader reader = Puffin .read (table .io ().newInputFile (statsPath .toString ())).build ()) {
743
- List <BlobMetadata > blobMetadata = reader .fileMetadata ().blobs ();
744
-
745
- if (partNames != null ) {
746
- blobMetadata = blobMetadata .stream ()
747
- .filter (metadata -> partNames .contains (metadata .properties ().get ("partition" )))
748
- .collect (Collectors .toList ());
749
- }
750
- Iterator <ByteBuffer > it = Iterables .transform (reader .readAll (blobMetadata ), Pair ::second ).iterator ();
751
- LOG .info ("Using col stats from : {}" , statsPath );
752
-
753
- while (it .hasNext ()) {
754
- byte [] byteBuffer = ByteBuffers .toByteArray (it .next ());
755
- colStats .add (SerializationUtils .deserialize (byteBuffer ));
756
- }
757
- } catch (Exception e ) {
758
- LOG .warn (" Unable to read col stats: " , e );
759
- }
760
- return colStats ;
761
- }
762
-
763
757
@ Override
764
758
public boolean canComputeQueryUsingStats (Partish partish ) {
765
759
org .apache .hadoop .hive .ql .metadata .Table hmsTable = partish .getTable ();
@@ -799,22 +793,24 @@ private boolean shouldRewriteColStats(Table tbl) {
799
793
private void checkAndMergeColStats (List <ColumnStatistics > statsNew , Table tbl ) throws InvalidObjectException {
800
794
Long previousSnapshotId = tbl .currentSnapshot ().parentId ();
801
795
if (previousSnapshotId != null && canProvideColStats (tbl , previousSnapshotId )) {
802
- List <ColumnStatistics > statsOld = IcebergTableUtil .getColStatsPath (tbl , previousSnapshotId )
803
- .map (statsPath -> readColStats (tbl , statsPath , null ))
804
- .orElse (Collections .emptyList ());
805
796
806
797
boolean isTblLevel = statsNew .get (0 ).getStatsDesc ().isIsTblLevel ();
807
798
Map <String , ColumnStatistics > oldStatsMap = Maps .newHashMap ();
808
799
800
+ List <?> statsOld = IcebergTableUtil .readColStats (tbl , previousSnapshotId , null );
801
+
809
802
if (!isTblLevel ) {
810
- for (ColumnStatistics statsObjOld : statsOld ) {
803
+ for (ColumnStatistics statsObjOld : ( List < ColumnStatistics >) statsOld ) {
811
804
oldStatsMap .put (statsObjOld .getStatsDesc ().getPartName (), statsObjOld );
812
805
}
806
+ } else {
807
+ statsOld = Collections .singletonList (
808
+ new ColumnStatistics (null , (List <ColumnStatisticsObj >) statsOld ));
813
809
}
814
810
for (ColumnStatistics statsObjNew : statsNew ) {
815
811
String partitionKey = statsObjNew .getStatsDesc ().getPartName ();
816
812
ColumnStatistics statsObjOld = isTblLevel ?
817
- statsOld .get (0 ) : oldStatsMap .get (partitionKey );
813
+ ( ColumnStatistics ) statsOld .get (0 ) : oldStatsMap .get (partitionKey );
818
814
819
815
if (statsObjOld != null && statsObjOld .getStatsObjSize () != 0 && !statsObjNew .getStatsObj ().isEmpty ()) {
820
816
MetaStoreServerUtils .mergeColStats (statsObjNew , statsObjOld );
0 commit comments