You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by florianschmidt1994 <gi...@git.apache.org> on 2018/04/03 12:51:19 UTC

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

GitHub user florianschmidt1994 opened a pull request:

    https://github.com/apache/flink/pull/5807

    [FLINK-8982][E2E Tests] Add test for known failure of queryable state

     ## What is the purpose of the change
    Add an end-to-end test to verify that the changes that @kl0u introduced in https://github.com/apache/flink/pull/5691 fix a known issue with concurrent access to queryable state, by verifying that access to queryable state works as expected.
    
    ## Brief change log
    - Add flink app with queryable state the continuously updates mapstate
    - Add queryable state client that periodically queries map state
    - Add end-to-end test that runs client against app and verifies that no unexpected exceptions occur
    - Integrate end-to-end test in testsuite
    
    ## Verifying this change
    This change added tests and can be verified as follows:
    - Run `./run-pre-commit-tests.sh`
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): no
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
      - The serializers: no
      - The runtime per-record code paths (performance sensitive): no
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
      - The S3 file system connector: no
    
    ## Documentation
    
      - Does this pull request introduce a new feature? no
      - If yes, how is the feature documented? not applicable


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/florianschmidt1994/flink end-to-end-tests-for-queryable-state

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5807.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5807
    
----
commit a94b06728c25df9a5c71a484f49cfabb36eb1460
Author: Florian Schmidt <fl...@...>
Date:   2018-03-13T13:13:08Z

    [FLINK-8982][E2E Tests] Add test for known failure of queryable state

----


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/5807


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187677168
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.state.MapState;
    +import org.apache.flink.api.common.state.MapStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.StateBackend;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.util.Collector;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
    +
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.Random;
    +
    +/**
    + * Streaming application that creates an {@link Email} pojo with random ids and increasing
    + * timestamps and passes it to a stateful {@link org.apache.flink.api.common.functions.FlatMapFunction},
    + * where it is exposed as queryable state.
    + */
    +public class QsStateProducer {
    +
    +	public static final String QUERYABLE_STATE_NAME = "state";
    +	public static final String STATE_NAME = "state";
    +
    +	public static void main(final String[] args) throws Exception {
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		ParameterTool tool = ParameterTool.fromArgs(args);
    +		String tmpPath = tool.getRequired("tmp-dir");
    +		String stateBackendType = tool.getRequired("state-backend");
    +
    +		StateBackend stateBackend;
    +		switch (stateBackendType) {
    +			case "rocksdb":
    +				stateBackend = new RocksDBStateBackend(tmpPath);
    +				break;
    +			case "fs":
    +				stateBackend = new FsStateBackend(tmpPath);
    +				break;
    +			case "memory":
    +				stateBackend = new MemoryStateBackend();
    +				break;
    +			default:
    +				throw new RuntimeException("Unsupported state backend " + stateBackendType);
    +		}
    +
    +		env.setStateBackend(stateBackend);
    +		env.enableCheckpointing(1000L);
    +		env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
    +		env.getCheckpointConfig().setMinPauseBetweenCheckpoints(0);
    +
    +		env.addSource(new EmailSource())
    +			.keyBy(new KeySelector<Email, String>() {
    +
    +				private static final long serialVersionUID = -1480525724620425363L;
    +
    +				@Override
    +				public String getKey(Email value) throws Exception {
    +					return "";
    +				}
    +			})
    +			.flatMap(new MyFlatMap());
    +
    +		env.execute();
    +	}
    +
    +	private static class EmailSource extends RichSourceFunction<Email> {
    +
    +		private static final long serialVersionUID = -7286937645300388040L;
    +
    +		private Random random;
    --- End diff --
    
    The `random` should be `transient`


---

[GitHub] flink issue #5807: [FLINK-8982][E2E Tests] Add test for known failure of que...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/5807
  
    Thanks! I addressed the changes in the last two commits


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827698
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/EmailInformation.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Javadoc.
    + */
    +public class EmailInformation implements Serializable {
    +
    +	private static final long serialVersionUID = -8956979869800484909L;
    +
    +	public void setEmailId(EmailId emailId) {
    +		this.emailId = emailId;
    +	}
    +
    +	private EmailId emailId;
    +
    +//	public void setTimestamp(Instant timestamp) {
    +//		this.timestamp = timestamp;
    +//	}
    +
    +	//private Instant timestamp;
    +
    +	public void setStuff(List<String> stuff) {
    +		this.stuff = stuff;
    +	}
    +
    +	private List<String> stuff;
    +
    +	public void setAsdf(Long asdf) {
    +		this.asdf = asdf;
    +	}
    +
    +	private Long asdf = 0L;
    +
    +	private transient LabelSurrogate label;
    +
    +	public EmailInformation() {
    +
    +	}
    +
    +	public EmailInformation(Email email) {
    +		emailId = email.getEmailId();
    +	//	timestamp = email.getTimestamp();
    --- End diff --
    
    remove.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187676881
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh ---
    @@ -0,0 +1,120 @@
    +#!/usr/bin/env bash
    +################################################################################
    +#  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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +source "$(dirname "$0")"/queryable_state_base.sh
    +
    +QUERYABLE_STATE_SERVER_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateProducer.jar
    +QUERYABLE_STATE_CLIENT_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateClient.jar
    +
    +#####################
    +# Test that queryable state works as expected with HA mode when restarting a taskmanager
    +#
    +# The general outline is like this:
    +# 1. start cluster in HA mode with 1 TM
    +# 2. start a job that exposes queryable state from a mapstate with increasing num. of keys
    +# 3. query the state with a queryable state client and expect no error to occur
    +# 4. stop the TM
    +# 5. check how many keys were in our mapstate at the time of the latest snapshot
    +# 6. start a new TM
    +# 7. query the state with a queryable state client and retrieve the number of elements
    +#    in the mapstate
    +# 8. expect the number of elements in the mapstate after restart of TM to be > number of elements
    +#    at last snapshot
    +#
    +# Globals:
    +#   QUERYABLE_STATE_SERVER_JAR
    +#   QUERYABLE_STATE_CLIENT_JAR
    +# Arguments:
    +#   None
    +# Returns:
    +#   None
    +#####################
    +function run_test() {
    +    local EXIT_CODE=0
    +    local PARALLELISM=1 # parallelism of queryable state app
    +    local PORT="9069" # port of queryable state server
    +
    +    clean_stdout_files # to ensure there are no files accidentally left behind by previous tests
    +    link_queryable_state_lib
    +    start_ha_cluster
    +
    +    local JOB_ID=$(${FLINK_DIR}/bin/flink run \
    +        -p ${PARALLELISM} \
    +        -d ${QUERYABLE_STATE_SERVER_JAR} \
    +        --state-backend "rocksdb" \
    +        --tmp-dir file://${TEST_DATA_DIR} \
    +        | awk '{print $NF}' | tail -n 1)
    +
    +    wait_job_running ${JOB_ID}
    +
    +    sleep 20 # sleep a little to have some state accumulated
    +
    +    SERVER=$(get_queryable_state_server_ip)
    +    PORT=$(get_queryable_state_proxy_port)
    +
    +    echo SERVER: ${SERVER}
    +    echo PORT: ${PORT}
    +
    +    java -jar ${QUERYABLE_STATE_CLIENT_JAR} \
    +        --host ${SERVER} \
    +        --port ${PORT} \
    +        --iterations 1 \
    +        --job-id ${JOB_ID}
    +
    +    if [ $? != 0 ]; then
    +        echo "An error occurred when executing queryable state client"
    +        exit 1
    +    fi
    +
    +    kill_random_taskmanager
    +
    +    latest_snapshot_count=$(cat $FLINK_DIR/log/*out* | grep "on snapshot" | tail -n 1 | awk '{print $4}')
    +    echo "Latest snapshot count was ${latest_snapshot_count}"
    +
    +    sleep 10 # this is a little longer than the heartbeat timeout so that the TM is gone
    +
    +    start_and_wait_for_tm
    +
    +    wait_job_running ${JOB_ID}
    +
    --- End diff --
    
    Instead of just waiting for the job to be running, it is safer to ask through `REST` for the successful checkpoints for the job right after killing the TM, and then expecting to see more successful checkpoints after the new TM is up. This is safer because it guarantees that the backend is initialized properly and can be done similarly to how it is done in the case of the `test_ha.sh`.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187675012
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/pom.xml ---
    @@ -0,0 +1,134 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!--
    +    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.
    +    -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0"
    +		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +	<parent>
    +		<artifactId>flink-end-to-end-tests</artifactId>
    +		<groupId>org.apache.flink</groupId>
    +		<version>1.6-SNAPSHOT</version>
    +	</parent>
    +	<modelVersion>4.0.0</modelVersion>
    +
    +	<artifactId>flink-queryable-state-test_${scala.binary.version}</artifactId>
    +	<name>flink-queryable-state-test</name>
    +
    +	<packaging>jar</packaging>
    +
    +	<dependencies>
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-core</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
    +			<version>${project.version}</version>
    +		</dependency>
    +		<dependency>
    +			<groupId>org.apache.flink</groupId>
    +			<artifactId>flink-statebackend-rocksdb_2.11</artifactId>
    +			<version>1.6-SNAPSHOT</version>
    --- End diff --
    
    the version should be `${project.version}`


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827723
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/EmailInformation.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Javadoc.
    + */
    +public class EmailInformation implements Serializable {
    +
    +	private static final long serialVersionUID = -8956979869800484909L;
    +
    +	public void setEmailId(EmailId emailId) {
    +		this.emailId = emailId;
    +	}
    +
    +	private EmailId emailId;
    +
    +//	public void setTimestamp(Instant timestamp) {
    +//		this.timestamp = timestamp;
    +//	}
    +
    +	//private Instant timestamp;
    +
    +	public void setStuff(List<String> stuff) {
    +		this.stuff = stuff;
    +	}
    +
    +	private List<String> stuff;
    +
    +	public void setAsdf(Long asdf) {
    +		this.asdf = asdf;
    +	}
    +
    +	private Long asdf = 0L;
    +
    +	private transient LabelSurrogate label;
    +
    +	public EmailInformation() {
    +
    +	}
    +
    +	public EmailInformation(Email email) {
    +		emailId = email.getEmailId();
    +	//	timestamp = email.getTimestamp();
    +		stuff = new ArrayList<>();
    +		stuff.add("1");
    +		stuff.add("2");
    +		stuff.add("3");
    +		label = email.getLabel();
    +	}
    +
    +	public EmailId getEmailId() {
    +		return emailId;
    +	}
    +
    +//	//public Instant getTimestamp() {
    +//		return timestamp;
    +//	}
    +
    +	public List<String> getStuff() {
    +		return stuff;
    +	}
    +
    +	public Long getAsdf() {
    +		return asdf;
    +	}
    +
    +	public LabelSurrogate getLabel() {
    +		return label;
    +	}
    +
    +	public void setLabel(LabelSurrogate label) {
    +		this.label = label;
    +	}
    +
    +	@Override
    +	public boolean equals(Object o) {
    +		if (this == o) {
    +			return true;
    +		}
    +		if (o == null || getClass() != o.getClass()) {
    +			return false;
    +		}
    +		EmailInformation that = (EmailInformation) o;
    +		return Objects.equals(emailId, that.emailId) &&
    +//				Objects.equals(timestamp, that.timestamp) &&
    --- End diff --
    
    remove.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187674099
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java ---
    @@ -0,0 +1,182 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.state.MapState;
    +import org.apache.flink.api.common.state.MapStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.runtime.state.FunctionInitializationContext;
    +import org.apache.flink.runtime.state.FunctionSnapshotContext;
    +import org.apache.flink.runtime.state.StateBackend;
    +import org.apache.flink.runtime.state.filesystem.FsStateBackend;
    +import org.apache.flink.runtime.state.memory.MemoryStateBackend;
    +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.util.Collector;
    +
    +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;
    +
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.Random;
    +
    +/**
    + * Streaming application that creates an {@link Email} pojo with random ids and increasing
    + * timestamps and passes it to a stateful {@link org.apache.flink.api.common.functions.FlatMapFunction},
    + * where it is exposed as queryable state.
    + */
    +public class QsStateProducer {
    +
    +	public static final String QUERYABLE_STATE_NAME = "state";
    +	public static final String STATE_NAME = "state";
    +
    +	public static void main(final String[] args) throws Exception {
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		ParameterTool tool = ParameterTool.fromArgs(args);
    +		String tmpPath = tool.getRequired("tmp-dir");
    +		String stateBackendType = tool.getRequired("state-backend");
    +
    +		StateBackend stateBackend;
    +		switch (stateBackendType) {
    +			case "rocksdb":
    +				stateBackend = new RocksDBStateBackend(tmpPath);
    +				break;
    +			case "fs":
    +				stateBackend = new FsStateBackend(tmpPath);
    +				break;
    +			case "memory":
    +				stateBackend = new MemoryStateBackend();
    +				break;
    +			default:
    +				throw new RuntimeException("Unsupported state backend " + stateBackendType);
    +		}
    +
    +		env.setStateBackend(stateBackend);
    +		env.enableCheckpointing(1000L);
    +		env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
    +		env.getCheckpointConfig().setMinPauseBetweenCheckpoints(0);
    +
    +		env.addSource(new EmailSource())
    +			.keyBy(new KeySelector<Email, String>() {
    +
    +				private static final long serialVersionUID = -1480525724620425363L;
    +
    +				@Override
    +				public String getKey(Email value) throws Exception {
    +					return "";
    +				}
    +			})
    +			.flatMap(new MyFlatMap());
    +
    +		env.execute();
    +	}
    +
    +	private static class EmailSource extends RichSourceFunction<Email> {
    +
    +		private static final long serialVersionUID = -7286937645300388040L;
    +
    +		private Random random;
    +		private volatile boolean isRunning = true;
    +
    +		@Override
    +		public void open(Configuration parameters) throws Exception {
    +			super.open(parameters);
    +			this.random = new Random();
    +		}
    +
    +		@Override
    +		public void run(SourceContext<Email> ctx) throws Exception {
    +			// Sleep for 10 seconds on start to allow time to copy jobid
    +			for (int i = 0; i < 100 && isRunning; i++) {
    +				Thread.sleep(100L);
    +			}
    +
    +			int types = LabelSurrogate.Type.values().length;
    +
    +			while (isRunning) {
    +				int r = random.nextInt(100);
    +
    +				final EmailId emailId = new EmailId(Integer.toString(random.nextInt()));
    +				final Instant timestamp = Instant.now().minus(Duration.ofDays(1L));
    +				final String foo = String.format("foo #%d", r);
    +				final LabelSurrogate label = new LabelSurrogate(LabelSurrogate.Type.values()[r % types], "bar");
    +
    --- End diff --
    
    Have to take the `checkpointLock` before emitting:
    
    ```
    synchronized (ctx.getCheckpointLock()) {
    	ctx.collect(new Email(emailId, timestamp, foo, label));
    }
    ```


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827678
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/EmailInformation.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Javadoc.
    + */
    +public class EmailInformation implements Serializable {
    +
    +	private static final long serialVersionUID = -8956979869800484909L;
    +
    +	public void setEmailId(EmailId emailId) {
    +		this.emailId = emailId;
    +	}
    +
    +	private EmailId emailId;
    +
    +//	public void setTimestamp(Instant timestamp) {
    +//		this.timestamp = timestamp;
    +//	}
    +
    +	//private Instant timestamp;
    +
    +	public void setStuff(List<String> stuff) {
    +		this.stuff = stuff;
    +	}
    +
    +	private List<String> stuff;
    +
    +	public void setAsdf(Long asdf) {
    +		this.asdf = asdf;
    +	}
    +
    +	private Long asdf = 0L;
    +
    +	private transient LabelSurrogate label;
    +
    +	public EmailInformation() {
    +
    +	}
    +
    +	public EmailInformation(Email email) {
    +		emailId = email.getEmailId();
    +	//	timestamp = email.getTimestamp();
    +		stuff = new ArrayList<>();
    +		stuff.add("1");
    +		stuff.add("2");
    +		stuff.add("3");
    +		label = email.getLabel();
    +	}
    +
    +	public EmailId getEmailId() {
    +		return emailId;
    +	}
    +
    +//	//public Instant getTimestamp() {
    --- End diff --
    
    remove.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178828372
  
    --- Diff: flink-end-to-end-tests/run-pre-commit-tests.sh ---
    @@ -37,6 +37,14 @@ echo "Flink distribution directory: $FLINK_DIR"
     
     EXIT_CODE=0
     
    --- End diff --
    
    I would recommend to move it to  the nightly tests. Queryable state is not a core component and the normal builds are already timing out.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178829164
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsBugPoc.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.state.MapState;
    +import org.apache.flink.api.common.state.MapStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.util.Collector;
    +
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.Random;
    +
    +/**
    + * Javadoc.
    + */
    +public class QsBugPoc {
    +
    +	public static final String QUERYABLE_STATE_NAME = "state";
    +	public static final String STATE_NAME = "state";
    +
    +	public static void main(final String[] args) throws Exception {
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		RocksDBStateBackend rocksDb = new RocksDBStateBackend("file:///tmp/deleteme-rocksdb");
    +		env.setStateBackend(rocksDb);
    --- End diff --
    
    Also check for different backends, i.e. file and rocks. You can have a look to the `test_ha.sh`.


---

[GitHub] flink issue #5807: [FLINK-8982][E2E Tests] Add test for known failure of que...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/5807
  
    @kl0u can you have a look at this?


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187675826
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh ---
    @@ -0,0 +1,120 @@
    +#!/usr/bin/env bash
    +################################################################################
    +#  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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +source "$(dirname "$0")"/queryable_state_base.sh
    +
    +QUERYABLE_STATE_SERVER_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateProducer.jar
    +QUERYABLE_STATE_CLIENT_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateClient.jar
    +
    +#####################
    +# Test that queryable state works as expected with HA mode when restarting a taskmanager
    +#
    +# The general outline is like this:
    +# 1. start cluster in HA mode with 1 TM
    +# 2. start a job that exposes queryable state from a mapstate with increasing num. of keys
    +# 3. query the state with a queryable state client and expect no error to occur
    +# 4. stop the TM
    +# 5. check how many keys were in our mapstate at the time of the latest snapshot
    +# 6. start a new TM
    +# 7. query the state with a queryable state client and retrieve the number of elements
    +#    in the mapstate
    +# 8. expect the number of elements in the mapstate after restart of TM to be > number of elements
    +#    at last snapshot
    +#
    +# Globals:
    +#   QUERYABLE_STATE_SERVER_JAR
    +#   QUERYABLE_STATE_CLIENT_JAR
    +# Arguments:
    +#   None
    +# Returns:
    +#   None
    +#####################
    +function run_test() {
    +    local EXIT_CODE=0
    +    local PARALLELISM=1 # parallelism of queryable state app
    +    local PORT="9069" # port of queryable state server
    +
    +    clean_stdout_files # to ensure there are no files accidentally left behind by previous tests
    +    link_queryable_state_lib
    +    start_ha_cluster
    --- End diff --
    
    We do not need an HA cluster. A normal cluster would be enough.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827415
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/EmailInformation.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Javadoc.
    + */
    +public class EmailInformation implements Serializable {
    +
    +	private static final long serialVersionUID = -8956979869800484909L;
    +
    +	public void setEmailId(EmailId emailId) {
    +		this.emailId = emailId;
    +	}
    +
    +	private EmailId emailId;
    +
    +//	public void setTimestamp(Instant timestamp) {
    --- End diff --
    
    Remove commented methods.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827936
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/LabelSurrogate.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +/**
    + * Javadoc.
    + */
    --- End diff --
    
    This is just a placeholder comment for checkstyle verification to pass. Please write a real comment. 
    This holds also for other places.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r187676635
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_queryable_state_restart_tm.sh ---
    @@ -0,0 +1,120 @@
    +#!/usr/bin/env bash
    +################################################################################
    +#  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.
    +################################################################################
    +
    +source "$(dirname "$0")"/common.sh
    +source "$(dirname "$0")"/queryable_state_base.sh
    +
    +QUERYABLE_STATE_SERVER_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateProducer.jar
    +QUERYABLE_STATE_CLIENT_JAR=${TEST_INFRA_DIR}/../../flink-end-to-end-tests/flink-queryable-state-test/target/QsStateClient.jar
    +
    +#####################
    +# Test that queryable state works as expected with HA mode when restarting a taskmanager
    +#
    +# The general outline is like this:
    +# 1. start cluster in HA mode with 1 TM
    +# 2. start a job that exposes queryable state from a mapstate with increasing num. of keys
    +# 3. query the state with a queryable state client and expect no error to occur
    +# 4. stop the TM
    +# 5. check how many keys were in our mapstate at the time of the latest snapshot
    +# 6. start a new TM
    +# 7. query the state with a queryable state client and retrieve the number of elements
    +#    in the mapstate
    +# 8. expect the number of elements in the mapstate after restart of TM to be > number of elements
    +#    at last snapshot
    +#
    +# Globals:
    +#   QUERYABLE_STATE_SERVER_JAR
    +#   QUERYABLE_STATE_CLIENT_JAR
    +# Arguments:
    +#   None
    +# Returns:
    +#   None
    +#####################
    +function run_test() {
    +    local EXIT_CODE=0
    +    local PARALLELISM=1 # parallelism of queryable state app
    +    local PORT="9069" # port of queryable state server
    +
    +    clean_stdout_files # to ensure there are no files accidentally left behind by previous tests
    +    link_queryable_state_lib
    +    start_ha_cluster
    +
    +    local JOB_ID=$(${FLINK_DIR}/bin/flink run \
    +        -p ${PARALLELISM} \
    +        -d ${QUERYABLE_STATE_SERVER_JAR} \
    +        --state-backend "rocksdb" \
    +        --tmp-dir file://${TEST_DATA_DIR} \
    +        | awk '{print $NF}' | tail -n 1)
    +
    +    wait_job_running ${JOB_ID}
    +
    +    sleep 20 # sleep a little to have some state accumulated
    --- End diff --
    
    Instead of waiting, it is safer to ask through `REST` for the successful checkpoints for the job right after killing the TM, and then expecting to see more successful checkpoints after the new TM is up. This is safer because it guarantees that the backend is initialized properly and can be done similarly to how it is done in the case of the `test_ha.sh`.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827543
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/EmailInformation.java ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import java.io.Serializable;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Objects;
    +
    +/**
    + * Javadoc.
    + */
    +public class EmailInformation implements Serializable {
    +
    +	private static final long serialVersionUID = -8956979869800484909L;
    +
    +	public void setEmailId(EmailId emailId) {
    +		this.emailId = emailId;
    +	}
    +
    +	private EmailId emailId;
    +
    +//	public void setTimestamp(Instant timestamp) {
    +//		this.timestamp = timestamp;
    +//	}
    +
    +	//private Instant timestamp;
    --- End diff --
    
    same here (remove commented field).


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178827995
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/LabelSurrogate.java ---
    @@ -0,0 +1,65 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +/**
    + * Javadoc.
    + */
    +public class LabelSurrogate {
    +
    +	private Type type;
    +	private String foo;
    +
    +	public LabelSurrogate(Type type, String foo) {
    +		this.type = type;
    +		this.foo = foo;
    +	}
    +
    +	public Type getType() {
    +		return type;
    +	}
    +
    +	public void setType(Type type) {
    +		this.type = type;
    +	}
    +
    +	public String getFoo() {
    +		return foo;
    +	}
    +
    +	public void setFoo(String foo) {
    +		this.foo = foo;
    +	}
    +
    +	@Override
    +	public String toString() {
    +		return "LabelSurrogate{" +
    +				"type=" + type +
    +				", foo='" + foo + '\'' +
    +				'}';
    +	}
    +
    +	/**
    +	 * Javadoc.
    +	 */
    --- End diff --
    
    Same here.


---

[GitHub] flink pull request #5807: [FLINK-8982][E2E Tests] Add test for known failure...

Posted by kl0u <gi...@git.apache.org>.
Github user kl0u commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5807#discussion_r178828967
  
    --- Diff: flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsBugPoc.java ---
    @@ -0,0 +1,133 @@
    +/*
    + * 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.streaming.tests.queryablestate;
    +
    +import org.apache.flink.api.common.functions.RichFlatMapFunction;
    +import org.apache.flink.api.common.state.MapState;
    +import org.apache.flink.api.common.state.MapStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.TypeHint;
    +import org.apache.flink.api.common.typeinfo.TypeInformation;
    +import org.apache.flink.api.java.functions.KeySelector;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
    +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.util.Collector;
    +
    +import java.time.Duration;
    +import java.time.Instant;
    +import java.util.Random;
    +
    +/**
    + * Javadoc.
    + */
    +public class QsBugPoc {
    +
    +	public static final String QUERYABLE_STATE_NAME = "state";
    +	public static final String STATE_NAME = "state";
    +
    +	public static void main(final String[] args) throws Exception {
    +		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    +
    +		RocksDBStateBackend rocksDb = new RocksDBStateBackend("file:///tmp/deleteme-rocksdb");
    +		env.setStateBackend(rocksDb);
    --- End diff --
    
    The dir to checkpoint can be a parameter and here it should be a path in the `TEST_DIR` of the test itself. In addition, everything should be explicitly cleaned up, e.g. checkpoints, potential output/input data, etc.


---