Return-Path: X-Original-To: apmail-flink-issues-archive@minotaur.apache.org Delivered-To: apmail-flink-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 08299182E8 for ; Thu, 5 Nov 2015 14:07:28 +0000 (UTC) Received: (qmail 88875 invoked by uid 500); 5 Nov 2015 14:07:28 -0000 Delivered-To: apmail-flink-issues-archive@flink.apache.org Received: (qmail 88847 invoked by uid 500); 5 Nov 2015 14:07:27 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 88836 invoked by uid 99); 5 Nov 2015 14:07:27 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 Nov 2015 14:07:27 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id B73532C1F51 for ; Thu, 5 Nov 2015 14:07:27 +0000 (UTC) Date: Thu, 5 Nov 2015 14:07:27 +0000 (UTC) From: "Till Rohrmann (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (FLINK-2979) RollingSink does not work with Hadoop 2.7.1 MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 Till Rohrmann created FLINK-2979: ------------------------------------ Summary: RollingSink does not work with Hadoop 2.7.1 Key: FLINK-2979 URL: https://issues.apache.org/jira/browse/FLINK-2979 Project: Flink Issue Type: Bug Components: Streaming Connectors Affects Versions: 0.10 Reporter: Till Rohrmann When executing the {{RollingSinkFaultToleranceITCase}} with Hadoop 2.7.1, then the test either does not finish because it's stuck in an endless restart loop with the following exception {code} java.lang.Exception: Could not restore checkpointed state to operators and functions at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateLazy(StreamTask.java:414) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:208) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.Exception: Failed to restore state to function: In-Progress file hdfs://127.0.0.1:52884/string-non-rolling-out/part-0-1 was neither moved to pending nor is still in progress. at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:165) at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreStateLazy(StreamTask.java:406) ... 3 more Caused by: java.lang.RuntimeException: In-Progress file hdfs://127.0.0.1:52884/string-non-rolling-out/part-0-1 was neither moved to pending nor is still in progress. at org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:670) at org.apache.flink.streaming.connectors.fs.RollingSink.restoreState(RollingSink.java:120) at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:162) ... 4 more {code} or it fails because the number of read strings differs from the exactly-once result (some strings are read multiple times). -- This message was sent by Atlassian JIRA (v6.3.4#6332)