[GitHub] incubator-apex-malhar pull request: MLHR-1897 #comment added manag...

classic Classic list List threaded Threaded
1 message Options
Reply | Threaded
Open this post in threaded view
|

[GitHub] incubator-apex-malhar pull request: MLHR-1897 #comment added manag...

DT-Priyanka
Github user chandnisingh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53709037
 
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,529 @@
    +/**
    + * 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 com.datatorrent.lib.state.managed;
    +
    +import java.io.IOException;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context.DAGContext;
    +import com.datatorrent.api.Context.OperatorContext;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.annotation.Stateless;
    +import com.datatorrent.common.util.NameableThreadFactory;
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.fileaccess.TFileImpl;
    +import com.datatorrent.lib.util.comparator.SliceComparator;
    +import com.datatorrent.netlet.util.DTThrowable;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * An abstract implementation of managed state.<br/>
    + *
    + * The important sub-components here are:
    + * <ol>
    + *   <li>
    + *     {@link #dataManager}: writes incremental checkpoints in window files and transfers data from window
    + *     files to bucket files.
    + *   </li>
    + *   <li>
    + *     {@link #bucketsMetaDataManager}: a bucket on disk is sub-divided into time-buckets. This manages meta-bucket
    + *     information (list of {@link com.datatorrent.lib.state.managed.BucketsMetaDataManager.TimeBucketMeta}) per bucket.
    + *   </li>
    + *   <li>
    + *     {@link #timeBucketAssigner}: assigns time-buckets to keys and manages the time boundaries.
    + *   </li>
    + *   <li>
    + *     {@link #stateTracker}: tracks the size of data in memory and requests buckets to free memory when enough memory
    + *     is not available.
    + *   </li>
    + *   <li>
    + *     {@link #fileAccess}: pluggable file system abstraction.
    + *   </li>
    + * </ol>
    + *
    + * The implementations of put, getSync and getAsync here use windowId as the time field to derive timeBucket of a key.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Component<OperatorContext>, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxMemorySize;
    +
    +  protected int numBuckets;
    +
    +  private int incrementalCheckpointWindowCount = DAGContext.CHECKPOINT_WINDOW_COUNT.defaultValue;
    +
    +  @NotNull
    +  private FileAccess fileAccess = new TFileImpl.DTFileImpl();
    +  @NotNull
    +  protected TimeBucketAssigner timeBucketAssigner = new TimeBucketAssigner();
    +
    +  protected Bucket[] buckets;
    +
    +  @Min(1)
    +  private int numReaders = 1;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private BucketsDataManager dataManager = new BucketsDataManager(this);
    +
    +  private final BucketsMetaDataManager bucketsMetaDataManager = new BucketsMetaDataManager(this);
    +
    +  private transient int operatorId;
    +  private transient long windowId;
    +
    +  private transient int windowCount;
    +
    +  private transient long largestRecoveryWindow;
    +  protected transient boolean replay;
    +
    +  @NotNull
    +  protected Comparator<Slice> keyComparator = new SliceComparator();
    +
    +  protected final transient AtomicReference<Throwable> throwable = new AtomicReference<>();
    +
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration checkStateSizeInterval = Duration.millis(
    +      DAGContext.STREAMING_WINDOW_SIZE_MILLIS.defaultValue * OperatorContext.APPLICATION_WINDOW_COUNT.defaultValue);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration durationPreventingFreeingSpace;
    +
    +  private transient StateTracker stateTracker = new StateTracker(this);
    +
    +  //accessible to StateTracker
    +  final transient Object commitLock = new Object();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorId = context.getId();
    +    fileAccess.init();
    +    timeBucketAssigner.register(dataManager);
    +    timeBucketAssigner.setup(context);
    +
    +    numBuckets = getNumBuckets();
    +    buckets = new Bucket[numBuckets];
    +
    +    Preconditions.checkArgument(numReaders <= numBuckets, "no. of readers cannot exceed no. of buckets");
    +    //setup state data manager
    +    dataManager.setup(context);
    +
    +    //recovering data for window files to bucket
    +    try {
    +      Map<Long, Map<Slice, Bucket.BucketedValue>> recovered = dataManager.load(operatorId);
    +      if (recovered != null && !recovered.isEmpty()) {
    +
    +        for (Map.Entry<Long, Map<Slice, Bucket.BucketedValue>> entry : recovered.entrySet()) {
    +          int bucketIdx = prepareBucket(entry.getKey());
    +
    +          for (Map.Entry<Slice, Bucket.BucketedValue> dataEntry : entry.getValue().entrySet()) {
    +            buckets[bucketIdx].put(dataEntry.getKey(), dataEntry.getValue().getTimeBucket(),
    +                dataEntry.getValue().getValue());
    +          }
    +        }
    +      }
    +    } catch (IOException e) {
    +      throw new RuntimeException("recovering", e);
    +    }
    +
    +    largestRecoveryWindow = dataManager.getLargestRecoveryWindow();
    +    long activationWindow = context.getValue(OperatorContext.ACTIVATION_WINDOW_ID);
    +
    +    if (activationWindow != Stateless.WINDOW_ID && largestRecoveryWindow <= activationWindow) {
    +      replay = true;
    +    }
    +
    +    readerService = Executors.newFixedThreadPool(numReaders, new NameableThreadFactory("managedStateReaders"));
    +    stateTracker.setup(context);
    +  }
    +
    +  /**
    +   * Gets the number of buckets which is required during setup to create the array of buckets.<br/>
    +   * {@link ManagedStateImpl} provides num of buckets which is injected using a property.<br/>
    +   * {@link TimeManagedStateImpl} provides num of buckets which are calculated based on time settings.
    +   *
    +   * @return number of buckets.
    +   */
    +  public abstract int getNumBuckets();
    +
    +  public void beginWindow(long l)
    --- End diff --
   
    From @amberarrow
     descriptive parameter name like 'windowId' ? (also in TimeBucketAssigner)
   
   
    --------
    will change


---
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 [hidden email] or file a JIRA ticket
with INFRA.
---