This is an automated email from the ASF dual-hosted git repository.

mxm pushed a commit to branch release-2.10.0
in repository https://gitbox.apache.org/repos/asf/beam.git


The following commit(s) were added to refs/heads/release-2.10.0 by this push:
     new acaec8e  [BEAM-6460] Remove cached class references upon start/shutdown
acaec8e is described below

commit acaec8e4e3d705056b6d5a712fe73d6571cf990b
Author: Maximilian Michels <m...@apache.org>
AuthorDate: Thu Jan 17 16:34:52 2019 -0500

    [BEAM-6460] Remove cached class references upon start/shutdown
    
    The Flink Classloader can only be garbage collected if the classes it 
loaded are
    not referenced anymore. Users have reported that old classes leaked through
    Jackson's TypeFactory after pipeline restart.
    
    This can only happen when
    
    - The user jar sits in the Flink classloader, e.g. YARN "cluster mode". 
Jackson
      will then be loaded from the Flink classloader.
    
    - The user uses Flink's version of a library and has set classloading to
      'parent-first' or does not bundle the library. Since libraries like 
Jackson
      are shaded in Flink this should not be an issue.
---
 .../translation/functions/FlinkDoFnFunction.java   |  7 ++-
 .../functions/FlinkStatefulDoFnFunction.java       |  7 ++-
 .../flink/translation/utils/FlinkClassloading.java | 30 +++++++++++
 .../wrappers/streaming/DoFnOperator.java           |  2 +
 .../streaming/io/UnboundedSourceWrapper.java       | 13 +++--
 .../wrappers/streaming/DoFnOperatorTest.java       | 62 +++++++++++++++++++++-
 6 files changed, 114 insertions(+), 7 deletions(-)

diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index 41f22ed..43d6f05 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -24,6 +24,7 @@ import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate;
+import org.apache.beam.runners.flink.translation.utils.FlinkClassloading;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -139,7 +140,11 @@ public class FlinkDoFnFunction<InputT, OutputT>
 
   @Override
   public void close() throws Exception {
-    doFnInvoker.invokeTeardown();
+    try {
+      doFnInvoker.invokeTeardown();
+    } finally {
+      FlinkClassloading.deleteStaticCaches();
+    }
   }
 
   static class DoFnOutputManager implements DoFnRunners.OutputManager {
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
index ca8d1ea..0046a42 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
@@ -33,6 +33,7 @@ import org.apache.beam.runners.core.TimerInternals;
 import org.apache.beam.runners.core.construction.SerializablePipelineOptions;
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate;
+import org.apache.beam.runners.flink.translation.utils.FlinkClassloading;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.transforms.DoFn;
@@ -210,6 +211,10 @@ public class FlinkStatefulDoFnFunction<K, V, OutputT>
 
   @Override
   public void close() throws Exception {
-    doFnInvoker.invokeTeardown();
+    try {
+      doFnInvoker.invokeTeardown();
+    } finally {
+      FlinkClassloading.deleteStaticCaches();
+    }
   }
 }
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkClassloading.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkClassloading.java
new file mode 100644
index 0000000..a114f40
--- /dev/null
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/FlinkClassloading.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.flink.translation.utils;
+
+import com.fasterxml.jackson.databind.type.TypeFactory;
+
+/** Utilities for dealing with classloading. */
+public class FlinkClassloading {
+
+  public static void deleteStaticCaches() {
+    // Clear cache to get rid of any references to the Flink Classloader
+    // See https://jira.apache.org/jira/browse/BEAM-6460
+    TypeFactory.defaultInstance().clearCache();
+  }
+}
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index 8bdcd56..1cd2734 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -56,6 +56,7 @@ import 
org.apache.beam.runners.core.construction.SerializablePipelineOptions;
 import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate;
 import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import org.apache.beam.runners.flink.translation.utils.FlinkClassloading;
 import 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals;
 import 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals;
 import 
org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals;
@@ -391,6 +392,7 @@ public class DoFnOperator<InputT, OutputT> extends 
AbstractStreamOperator<Window
       super.dispose();
       checkFinishBundleTimer.cancel(true);
     } finally {
+      FlinkClassloading.deleteStaticCaches();
       if (bundleStarted) {
         invokeFinishBundle();
       }
diff --git 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 9562a1e..bfe79d3 100644
--- 
a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ 
b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -26,6 +26,7 @@ import org.apache.beam.runners.flink.FlinkPipelineOptions;
 import org.apache.beam.runners.flink.metrics.FlinkMetricContainer;
 import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil;
 import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.utils.FlinkClassloading;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.SerializableCoder;
@@ -333,11 +334,15 @@ public class UnboundedSourceWrapper<OutputT, 
CheckpointMarkT extends UnboundedSo
 
   @Override
   public void close() throws Exception {
-    super.close();
-    if (localReaders != null) {
-      for (UnboundedSource.UnboundedReader<OutputT> reader : localReaders) {
-        reader.close();
+    try {
+      super.close();
+      if (localReaders != null) {
+        for (UnboundedSource.UnboundedReader<OutputT> reader : localReaders) {
+          reader.close();
+        }
       }
+    } finally {
+      FlinkClassloading.deleteStaticCaches();
     }
   }
 
diff --git 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
index ed1630c..d134bad 100644
--- 
a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
+++ 
b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
@@ -18,11 +18,15 @@
 package org.apache.beam.runners.flink.translation.wrappers.streaming;
 
 import static 
org.apache.beam.runners.flink.translation.wrappers.streaming.StreamRecordStripper.stripStreamRecordFromWindowedValue;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.emptyIterable;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.is;
 import static org.hamcrest.collection.IsIterableContainingInOrder.contains;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertThat;
 
+import com.fasterxml.jackson.databind.type.TypeFactory;
+import com.fasterxml.jackson.databind.util.LRUMap;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Optional;
@@ -82,6 +86,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+import org.mockito.internal.util.reflection.Whitebox;
 
 /** Tests for {@link DoFnOperator}. */
 @RunWith(JUnit4.class)
@@ -1200,6 +1205,61 @@ public class DoFnOperatorTest {
     newHarness.close();
   }
 
+  /**
+   * Ensures Jackson cache is cleaned to get rid of any references to the 
Flink Classloader. See
+   * https://jira.apache.org/jira/browse/BEAM-6460
+   */
+  @Test
+  public void testRemoveCachedClassReferences() throws Exception {
+
+    FlinkPipelineOptions options = 
PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+    options.setParallelism(4);
+
+    TupleTag<String> outputTag = new TupleTag<>("main-output");
+    WindowedValue.ValueOnlyWindowedValueCoder<String> windowedValueCoder =
+        WindowedValue.getValueOnlyCoder(StringUtf8Coder.of());
+    IdentityDoFn<String> doFn =
+        new IdentityDoFn<String>() {
+          @FinishBundle
+          public void finishBundle(FinishBundleContext context) {
+            context.output(
+                "finishBundle", BoundedWindow.TIMESTAMP_MIN_VALUE, 
GlobalWindow.INSTANCE);
+          }
+        };
+
+    DoFnOperator.MultiOutputOutputManagerFactory<String> outputManagerFactory =
+        new DoFnOperator.MultiOutputOutputManagerFactory(
+            outputTag,
+            WindowedValue.getFullCoder(StringUtf8Coder.of(), 
GlobalWindow.Coder.INSTANCE));
+
+    DoFnOperator<String, String> doFnOperator =
+        new DoFnOperator<>(
+            doFn,
+            "stepName",
+            windowedValueCoder,
+            null,
+            Collections.emptyMap(),
+            outputTag,
+            Collections.emptyList(),
+            outputManagerFactory,
+            WindowingStrategy.globalDefault(),
+            new HashMap<>(), /* side-input mapping */
+            Collections.emptyList(), /* side inputs */
+            options,
+            null,
+            null);
+
+    OneInputStreamOperatorTestHarness<WindowedValue<String>, 
WindowedValue<String>> testHarness =
+        new OneInputStreamOperatorTestHarness<>(doFnOperator);
+
+    LRUMap typeCache =
+        (LRUMap) Whitebox.getInternalState(TypeFactory.defaultInstance(), 
"_typeCache");
+    assertThat(typeCache.size(), greaterThan(0));
+    testHarness.open();
+    testHarness.close();
+    assertThat(typeCache.size(), is(0));
+  }
+
   private Iterable<WindowedValue<String>> stripStreamRecord(Iterable<?> input) 
{
     return FluentIterable.from(input)
         .filter(o -> o instanceof StreamRecord)

Reply via email to