[ 
https://issues.apache.org/jira/browse/FLINK-8005?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16244159#comment-16244159
 ] 

ASF GitHub Bot commented on FLINK-8005:
---------------------------------------

Github user GJL commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4980#discussion_r149703349
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
 ---
    @@ -1,157 +0,0 @@
    -/*
    - * 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.flink.runtime.taskmanager;
    -
    -import org.apache.flink.api.common.JobID;
    -import org.apache.flink.api.common.TaskInfo;
    -import org.apache.flink.configuration.Configuration;
    -import org.apache.flink.runtime.blob.BlobCacheService;
    -import org.apache.flink.runtime.blob.PermanentBlobCache;
    -import org.apache.flink.runtime.blob.TransientBlobCache;
    -import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
    -import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    -import org.apache.flink.runtime.clusterframework.types.AllocationID;
    -import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
    -import 
org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
    -import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
    -import org.apache.flink.runtime.execution.ExecutionState;
    -import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
    -import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
    -import org.apache.flink.runtime.executiongraph.JobInformation;
    -import org.apache.flink.runtime.executiongraph.TaskInformation;
    -import org.apache.flink.runtime.filecache.FileCache;
    -import org.apache.flink.runtime.io.disk.iomanager.IOManager;
    -import org.apache.flink.runtime.io.network.NetworkEnvironment;
    -import 
org.apache.flink.runtime.io.network.netty.PartitionProducerStateChecker;
    -import 
org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
    -import org.apache.flink.runtime.jobgraph.JobVertexID;
    -import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
    -import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
    -import org.apache.flink.runtime.jobgraph.tasks.StoppableTask;
    -import org.apache.flink.runtime.memory.MemoryManager;
    -import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;
    -import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
    -
    -import org.junit.Test;
    -import org.junit.runner.RunWith;
    -import org.powermock.core.classloader.annotations.PrepareForTest;
    -import org.powermock.modules.junit4.PowerMockRunner;
    -
    -import java.lang.reflect.Field;
    -import java.util.Collections;
    -import java.util.concurrent.Executor;
    -
    -import scala.concurrent.duration.FiniteDuration;
    -
    -import static org.mockito.Mockito.mock;
    -import static org.mockito.Mockito.when;
    -
    -@RunWith(PowerMockRunner.class)
    -@PrepareForTest({ TaskDeploymentDescriptor.class, JobID.class, 
FiniteDuration.class })
    -public class TaskStopTest {
    -   private Task task;
    -
    -   public void doMocking(AbstractInvokable taskMock) throws Exception {
    -
    -           TaskInfo taskInfoMock = mock(TaskInfo.class);
    -           
when(taskInfoMock.getTaskNameWithSubtasks()).thenReturn("dummyName");
    -
    -           TaskManagerRuntimeInfo tmRuntimeInfo = 
mock(TaskManagerRuntimeInfo.class);
    -           when(tmRuntimeInfo.getConfiguration()).thenReturn(new 
Configuration());
    -
    -           TaskMetricGroup taskMetricGroup = mock(TaskMetricGroup.class);
    -           
when(taskMetricGroup.getIOMetricGroup()).thenReturn(mock(TaskIOMetricGroup.class));
    -
    -           BlobCacheService blobService =
    -                   new BlobCacheService(mock(PermanentBlobCache.class), 
mock(TransientBlobCache.class));
    -
    -           task = new Task(
    -                   mock(JobInformation.class),
    -                   new TaskInformation(
    -                           new JobVertexID(),
    -                           "test task name",
    -                           1,
    -                           1,
    -                           "foobar",
    -                           new Configuration()),
    -                   mock(ExecutionAttemptID.class),
    -                   mock(AllocationID.class),
    -                   0,
    -                   0,
    -                   
Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
    -                   Collections.<InputGateDeploymentDescriptor>emptyList(),
    -                   0,
    -                   mock(TaskStateSnapshot.class),
    -                   mock(MemoryManager.class),
    -                   mock(IOManager.class),
    -                   mock(NetworkEnvironment.class),
    -                   mock(BroadcastVariableManager.class),
    -                   mock(TaskManagerActions.class),
    -                   mock(InputSplitProvider.class),
    -                   mock(CheckpointResponder.class),
    -                   blobService,
    -                   mock(LibraryCacheManager.class),
    -                   mock(FileCache.class),
    -                   tmRuntimeInfo,
    -                   taskMetricGroup,
    -                   mock(ResultPartitionConsumableNotifier.class),
    -                   mock(PartitionProducerStateChecker.class),
    -                   mock(Executor.class));
    -           Field f = task.getClass().getDeclaredField("invokable");
    -           f.setAccessible(true);
    -           f.set(task, taskMock);
    -
    -           Field f2 = task.getClass().getDeclaredField("executionState");
    -           f2.setAccessible(true);
    -           f2.set(task, ExecutionState.RUNNING);
    -   }
    -
    -   @Test(timeout = 20000)
    -   public void testStopExecution() throws Exception {
    --- End diff --
    
    This only tested that stop is invoked. Should be covered by 
`testSetsUserCodeClassLoader`.


> Snapshotting FlinkKafkaProducer011 fails due to ClassLoader issues
> ------------------------------------------------------------------
>
>                 Key: FLINK-8005
>                 URL: https://issues.apache.org/jira/browse/FLINK-8005
>             Project: Flink
>          Issue Type: Bug
>          Components: Core, Kafka Connector, State Backends, Checkpointing
>    Affects Versions: 1.4.0
>            Reporter: Gary Yao
>            Assignee: Gary Yao
>            Priority: Blocker
>             Fix For: 1.4.0
>
>
> *Problem Description*
> Classes in the user code jar cannot be loaded by the snapshot thread’s 
> context class loader ({{AppClassLoader}}).
> For example, when creating instances of {{KafkaProducer}}, Strings are 
> resolved to class objects by Kafka.
> Find below an extract from {{ConfigDef.java}}:
> {code}
> case CLASS:
>     if (value instanceof Class)
>         return value;
>     else if (value instanceof String)
>         return Class.forName(trimmed, true, 
> Utils.getContextOrKafkaClassLoader());
>     else
>         throw new ConfigException(name, value, "Expected a Class instance or 
> class name.");
> {code}
> *Exception/Stacktrace*
> {noformat}
> Caused by: java.lang.Exception: Could not complete snapshot 1 for operator 
> Source: Collection Source -> Sink: kafka-sink-1510048188383 (1/1).
>       at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:379)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.checkpointStreamOperator(StreamTask.java:1077)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$CheckpointingOperation.executeCheckpointing(StreamTask.java:1026)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.checkpointState(StreamTask.java:659)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:595)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpoint(StreamTask.java:526)
>       ... 7 more
> Caused by: org.apache.kafka.common.config.ConfigException: Invalid value 
> org.apache.kafka.common.serialization.ByteArraySerializer for configuration 
> key.serializer: Class 
> org.apache.kafka.common.serialization.ByteArraySerializer could not be found.
>       at 
> org.apache.kafka.common.config.ConfigDef.parseType(ConfigDef.java:715)
>       at 
> org.apache.kafka.common.config.ConfigDef.parseValue(ConfigDef.java:460)
>       at org.apache.kafka.common.config.ConfigDef.parse(ConfigDef.java:453)
>       at 
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:62)
>       at 
> org.apache.kafka.common.config.AbstractConfig.<init>(AbstractConfig.java:75)
>       at 
> org.apache.kafka.clients.producer.ProducerConfig.<init>(ProducerConfig.java:360)
>       at 
> org.apache.kafka.clients.producer.KafkaProducer.<init>(KafkaProducer.java:288)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaProducer.<init>(FlinkKafkaProducer.java:114)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.initProducer(FlinkKafkaProducer011.java:913)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.initTransactionalProducer(FlinkKafkaProducer011.java:904)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.createOrGetProducerFromPool(FlinkKafkaProducer011.java:637)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.beginTransaction(FlinkKafkaProducer011.java:613)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.beginTransaction(FlinkKafkaProducer011.java:94)
>       at 
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.beginTransactionInternal(TwoPhaseCommitSinkFunction.java:359)
>       at 
> org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.snapshotState(TwoPhaseCommitSinkFunction.java:294)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011.snapshotState(FlinkKafkaProducer011.java:756)
>       at 
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.trySnapshotFunctionState(StreamingFunctionUtils.java:118)
>       at 
> org.apache.flink.streaming.util.functions.StreamingFunctionUtils.snapshotFunctionState(StreamingFunctionUtils.java:99)
>       at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.snapshotState(AbstractUdfStreamOperator.java:90)
>       at 
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:357)
>       ... 12 more
> {noformat}
> *How to reproduce*
> Note that the problem only appears when a job is deployed on a cluster. 
> # Build Flink 1.4
> # Build test job https://github.com/GJL/flink-kafka011-producer-test with 
> {{mvn -o clean install -Pbuild-jar}}
> # Start job:
> {noformat}
> bin/flink run -c com.garyyao.StreamingJob 
> /pathto/flink-kafka011-producer/target/flink-kafka011-producer-1.0-SNAPSHOT.jar
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to