rbalamohan commented on a change in pull request #2620:
URL: https://github.com/apache/hive/pull/2620#discussion_r703091669



##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
##########
@@ -541,6 +570,69 @@ public Map read(Kryo kryo, Input input, Class<? extends 
Map> type) {
     }
   }
 
+  /**
+   * We use a custom {@link com.esotericsoftware.kryo.Serializer} for {@link 
MapWork} objects e.g. in
+   * order to remove useless properties in execution time.
+   */
+  private static class MapWorkSerializer extends FieldSerializer<MapWork> {
+
+    public MapWorkSerializer(Kryo kryo, Class type) {
+      super(kryo, type);
+    }
+
+    @Override
+    public void write(Kryo kryo, Output output, MapWork mapWork) {
+      filterMapworkProperties(kryo, mapWork);
+      super.write(kryo, output, mapWork);
+    }
+
+    private void filterMapworkProperties(Kryo kryo, MapWork mapWork) {
+      Configuration configuration = ((KryoWithHooks) kryo).getConf();
+      if (configuration == null || HiveConf
+          .getVar(configuration, 
HiveConf.ConfVars.HIVE_PLAN_MAPWORK_SERIALIZATION_SKIP_PROPERTIES).isEmpty()) {
+        return;
+      }
+      String[] filterProps =
+          HiveConf.getVar(configuration, 
HiveConf.ConfVars.HIVE_PLAN_MAPWORK_SERIALIZATION_SKIP_PROPERTIES).split(",");
+      for (String prop : filterProps) {
+        boolean isRegex = isRegex(prop);
+        LOG.debug("Trying to filter mapwork properties (regex: " + isRegex + 
"): " + prop);
+
+        for (Entry<Path, PartitionDesc> partDescEntry : 
mapWork.getPathToPartitionInfo().entrySet()) {
+          /*
+           * remove by regex, could be a bit more expensive because of 
iterating and matching regexes
+           * e.g.: in case of impala_intermediate_stats_chunk1, 
impala_intermediate_stats_chunk2, user only needs to
+           * configure impala_intermediate_stats_chunk.*
+           */
+          if (isRegex) {
+            Pattern pattern = Pattern.compile(prop);

Review comment:
       This can be moved out of this loop.

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
##########
@@ -573,31 +665,23 @@ public PartitionDesc read(Kryo kryo, Input input, Class<? 
extends PartitionDesc>
    * @param out  The stream to write to.
    */
   public static void serializePlan(Object plan, OutputStream out) {
-    serializePlan(plan, out, false);
-  }
-
-  public static void serializePlan(Kryo kryo, Object plan, OutputStream out) {
-    serializePlan(kryo, plan, out, false);
+    serializePlan(plan, out, null);
   }
 
-  private static void serializePlan(Object plan, OutputStream out, boolean 
cloningPlan) {
-    Kryo kryo = borrowKryo();
+  private static void serializePlan(Object plan, OutputStream out, 
Configuration configuration) {

Review comment:
       Can you add simple test case for ser/deser with these props in 
TestPlan.java?

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java
##########
@@ -541,6 +570,69 @@ public Map read(Kryo kryo, Input input, Class<? extends 
Map> type) {
     }
   }
 
+  /**
+   * We use a custom {@link com.esotericsoftware.kryo.Serializer} for {@link 
MapWork} objects e.g. in
+   * order to remove useless properties in execution time.
+   */
+  private static class MapWorkSerializer extends FieldSerializer<MapWork> {
+
+    public MapWorkSerializer(Kryo kryo, Class type) {
+      super(kryo, type);
+    }
+
+    @Override
+    public void write(Kryo kryo, Output output, MapWork mapWork) {
+      filterMapworkProperties(kryo, mapWork);
+      super.write(kryo, output, mapWork);
+    }
+
+    private void filterMapworkProperties(Kryo kryo, MapWork mapWork) {
+      Configuration configuration = ((KryoWithHooks) kryo).getConf();
+      if (configuration == null || HiveConf
+          .getVar(configuration, 
HiveConf.ConfVars.HIVE_PLAN_MAPWORK_SERIALIZATION_SKIP_PROPERTIES).isEmpty()) {
+        return;
+      }
+      String[] filterProps =
+          HiveConf.getVar(configuration, 
HiveConf.ConfVars.HIVE_PLAN_MAPWORK_SERIALIZATION_SKIP_PROPERTIES).split(",");
+      for (String prop : filterProps) {
+        boolean isRegex = isRegex(prop);
+        LOG.debug("Trying to filter mapwork properties (regex: " + isRegex + 
"): " + prop);
+
+        for (Entry<Path, PartitionDesc> partDescEntry : 
mapWork.getPathToPartitionInfo().entrySet()) {
+          /*
+           * remove by regex, could be a bit more expensive because of 
iterating and matching regexes
+           * e.g.: in case of impala_intermediate_stats_chunk1, 
impala_intermediate_stats_chunk2, user only needs to
+           * configure impala_intermediate_stats_chunk.*

Review comment:
       Can you add this example in config parameter as well? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to