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

    https://github.com/apache/flink/pull/4353#discussion_r133026125
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/TaskStateSnapshot.java
 ---
    @@ -0,0 +1,139 @@
    +/*
    + * 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.checkpoint;
    +
    +import org.apache.flink.runtime.jobgraph.OperatorID;
    +import org.apache.flink.runtime.state.CompositeStateHandle;
    +import org.apache.flink.runtime.state.SharedStateRegistry;
    +import org.apache.flink.runtime.state.StateUtil;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.Set;
    +
    +/**
    + * This class encapsulates state handles to the snapshots of all operator 
instances executed within one task. A task
    + * can run multiple operator instances as a result of operator chaining, 
and all operator instances from the chain can
    + * register their state under their operator id. Each operator instance is 
a physical execution responsible for
    + * processing a partition of the data that goes through a logical 
operator. This partitioning happens to parallelize
    + * execution of logical operators, e.g. distributing a map function.
    + *
    + * <p>One instance of this class contains the information that one task 
will send to acknowledge a checkpoint request by
    + * the checkpoint coordinator. Tasks run operator instances in parallel, 
so the union of all
    + * {@link TaskStateSnapshot} that are collected by the checkpoint 
coordinator from all tasks represent the whole
    + * state of a job at the time of the checkpoint.
    + *
    + * <p>This class should be called TaskState once the old class with this 
name that we keep for backwards
    + * compatibility goes away.
    + */
    +public class TaskStateSnapshot implements CompositeStateHandle {
    +
    +   private static final long serialVersionUID = 1L;
    +
    +   /** Mapping from an operator id to the state of one subtask of this 
operator */
    +   private final Map<OperatorID, OperatorSubtaskState> 
subtaskStatesByOperatorID;
    --- End diff --
    
    Hmm, I think if we consider default load factors and for large sizes, I 
would pick a min >30% hit rate linear array scan over 100% hit rate random 
access iteration. For all expected sizes (in cache) in this class, it should 
not matter. LHM also consumes a bit more memory. I would tend to keep it this 
way.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to