You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by chandnisingh <gi...@git.apache.org> on 2015/12/22 03:25:56 UTC

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

GitHub user chandnisingh opened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/145

    [For Review Only] MLHR-1897 #comment added managed state 

    Still working on tests and will add documentation. 
    
    @amberarrow @ilooner @PramodSSImmaneni 
    Please review

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

    $ git pull https://github.com/chandnisingh/incubator-apex-malhar MLHR-1897

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

    https://github.com/apache/incubator-apex-malhar/pull/145.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 #145
    
----
commit 1ef879b55437c343d9fce414bd9d3c41db2da154
Author: Chandni Singh <cs...@apache.org>
Date:   2015-12-13T11:13:08Z

    MLHR-1897 #comment added managed state

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48512994
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    --- End diff --
    
    Javadoc?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56759046
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    protected transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsFileSystem.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +    }
    +
    +    protected DefaultBucket(long bucketId)
    +    {
    +      this.bucketId = bucketId;
    +    }
    +
    +    @Override
    +    public void setup(@NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traverse the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      for (Long window : recoveredData.descendingKeySet()) {
    +        //traverse the reccovered data in reverse order
    +        bucketedValue = recoveredData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    --- End diff --
    
    Will fix this. However the expectation here is that the latest value is retrieved always. If the latest time-bucket is known then the process to read it is faster otherwise reading all time-buckets is slower. This is again missing from the documentation  and I will add to it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56766113
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    --- End diff --
    
    return null. => return expired token.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53725996
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,505 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends Component<Context.OperatorContext>
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    private final transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsMetaDataManager.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      managedStateContext = null;
    +    }
    +
    +    protected DefaultBucket(long bucketId, @NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.bucketId = bucketId;
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext);
    +    }
    +
    +    @Override
    +    public void setup(Context.OperatorContext context)
    +    {
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traversing the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    +        }
    +        return valSlice;
    +      } else {
    +        //search all the time buckets
    +        try {
    +          if (cachedBucketMetas == null) {
    +            cachedBucketMetas = managedStateContext.getBucketsMetaDataManager().getAllTimeBuckets(bucketId);
    +          }
    +
    +          for (BucketsMetaDataManager.ImmutableTimeBucketMeta immutableTimeBucketMeta : cachedBucketMetas) {
    +
    +            if (managedStateContext.getKeyComparator().compare(key, immutableTimeBucketMeta.getFirstKey()) >= 0) {
    +              //keys in the time bucket files are sorted so if the first key in the file is greater than the key being
    +              //searched, the key will not be present in that file.
    +              Slice valSlice = getKeyFromTimeBucketReader(key, immutableTimeBucketMeta.getTimeBucketId());
    +              if (valSlice != null) {
    +                BucketedValue bucketedValue = new BucketedValue(immutableTimeBucketMeta.getTimeBucketId(), valSlice);
    +                readCache.put(key, bucketedValue);
    +                return valSlice;
    +              }
    +            }
    +          }
    +
    +        } catch (IOException e) {
    +          throw new RuntimeException("get time-buckets " + bucketId, e);
    +        }
    +      }
    +      return null;
    +    }
    +
    +    @Override
    +    public Slice get(Slice key, long timeBucket, ReadSource readSource)
    +    {
    +      switch (readSource) {
    +        case MEMORY:
    +          return getFromMemory(key);
    +        case READERS:
    +          return getFromReaders(key, timeBucket);
    +        case ALL:
    +        default:
    +          Slice value = getFromMemory(key);
    +          if (value != null) {
    +            return value;
    +          }
    +          return getFromReaders(key, timeBucket);
    +      }
    +    }
    +
    +    /**
    +     * Returns the value for the key from a time-bucket reader
    +     * @param key        key
    +     * @param timeBucket time bucket
    +     * @return value if key is found in the time bucket; false otherwise
    +     */
    +    private Slice getKeyFromTimeBucketReader(Slice key, long timeBucket)
    +    {
    +      FileAccess.FileReader fileReader = readers.get(timeBucket);
    +      if (fileReader != null) {
    +        return readKey(fileReader, key, timeBucket);
    +      }
    +      //file reader is not loaded and is null
    +      try {
    +        if (loadFileReader(timeBucket)) {
    +          return readKey(readers.get(timeBucket), key, timeBucket);
    +        }
    +        return null;
    +      } catch (IOException e) {
    +        throw new RuntimeException("while loading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private Slice readKey(FileAccess.FileReader fileReader, Slice key, long timeBucket)
    +    {
    +      Slice valSlice = new Slice(null, 0, 0);
    +      try {
    +        if (fileReader.seek(key)) {
    +          fileReader.next(dummyGetKey, valSlice);
    +          return valSlice;
    +        } else {
    +          return null;
    +        }
    +      } catch (IOException e) {
    +        throw new RuntimeException("reading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private boolean loadFileReader(long timeBucketId) throws IOException
    +    {
    +      BucketsMetaDataManager.ImmutableTimeBucketMeta tbm = managedStateContext.getBucketsMetaDataManager()
    +          .getTimeBucketMeta(bucketId, timeBucketId);
    +
    +      if (tbm != null) {
    +        FileAccess.FileReader reader = managedStateContext.getFileAccess().getReader(bucketId,
    +            BucketsDataManager.getFileName(timeBucketId));
    +        readers.put(timeBucketId, reader);
    +        sizeInBytes.getAndAdd(tbm.getSizeInBytes());
    +        return true;
    +      }
    +      return false;
    +    }
    +
    +    @Override
    +    public void put(Slice key, long timeBucket, Slice value)
    +    {
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue == null) {
    +        bucketedValue = new BucketedValue();
    +        flash.put(key, bucketedValue);
    +        sizeInBytes.getAndAdd(key.length);
    +        sizeInBytes.getAndAdd(Long.SIZE);
    +      }
    +      if (timeBucket > bucketedValue.getTimeBucket()) {
    +
    +        if (bucketedValue.getValue() != null) {
    +          //overriding the value so need to subtract the size of previous value
    +          sizeInBytes.getAndAdd(-bucketedValue.getValue().length);
    +        }
    +
    +        sizeInBytes.getAndAdd(value.length);
    --- End diff --
    
    From @amberarrow 
    Can avoid 2 calls to getAndAdd() with something like:
    ```java
    int inc = null == bucketedValue.getValue() ? value.length : value.length - bucketedValue.getValue().length;
    sizeInBytes.getAndAdd(inc);
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56767189
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.lib.state.TimeSlicedBucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * This implementation of {@link ManagedState} lets the client to specify the time for each key. The value of time
    + * is used to derive the time-bucket of a key.
    + */
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public class ManagedTimeStateImpl extends AbstractManagedStateImpl implements TimeSlicedBucketedState
    +{
    +  public ManagedTimeStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, long time, Slice key, Slice value)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return BucketedState.EXPIRED;
    +    }
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    --- End diff --
    
    This synchronized block and the assignment to `bucket` are duplicated in multiple places (3 or 4); could it be factored out to a helper method in the base class ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765586
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    --- End diff --
    
    These contrasting statements should also include the third implementation class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53717920
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    +      }
    +    }
    +  };
    +
    +  private final transient Object lock = new Object();
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    if (!initialized) {
    +      if (bucketSpan == null) {
    +        bucketSpan = Duration.millis(context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +            context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS));
    +      }
    +      Calendar calendar = Calendar.getInstance();
    +      long now = calendar.getTimeInMillis();
    +      fixedStartTime = now - expireBefore.getMillis();
    +      startTime = fixedStartTime;
    +
    +      bucketSpanMillis = bucketSpan.getMillis();
    +      numBuckets = (int)((expireBefore.getMillis() + bucketSpanMillis - 1) / bucketSpanMillis);
    +      endTime = startTime + (numBuckets * bucketSpanMillis);
    +
    +      initialized = true;
    +    }
    +    windowBoundedService = new WindowBoundedService(bucketSpanMillis, expiryTask);
    +    windowBoundedService.setup(context);
    +  }
    +
    +  public void beginWindow(long l)
    +  {
    +    windowBoundedService.beginWindow(l);
    +  }
    +
    +  public void endWindow()
    +  {
    +    windowBoundedService.endWindow();
    +  }
    +
    +  /**
    +   * Get the bucket key for the long value.
    +   *
    +   * @param value value from which bucket key is derived.
    +   * @return -1 if value is already expired; bucket key otherwise.
    +   */
    +  public long getTimeBucketFor(long value)
    +  {
    +    long lstart;
    +    long lend;
    +    synchronized (lock) {
    +      lstart = startTime;
    +      lend = endTime;
    +    }
    +    if (value < lstart) {
    +      return -1;
    +    }
    +    long diffFromStart = value - fixedStartTime;
    +    long key = diffFromStart / bucketSpanMillis;
    +    if (value > lend) {
    +      long move = ((value - lend) / bucketSpanMillis + 1) * bucketSpanMillis;
    +      synchronized (lock) {
    +        startTime = lstart + move;
    +        endTime = lend + move;
    +      }
    --- End diff --
    
    >> Can (value - lend) ever exceed bucketSpanMillis ?
    Yes. ```startTime``` is initialized during setup() and ```endTime = startTime + numBucket*bucketSpan```. 
    
    However ```value```  can come from the event. This time in the event can be ahead of ```endTime``` (we have seen this in a real application).  This triggers the whole boundary to move ahead.
    
    >> Second, it seems like this assignment could overwrite the assignment made to startTime and endTime by the expiryTask thread in some scenarios, for example: (a) We copy the values in the first synchronized block above (b) The expiryTask thread increments values by bucketSpanMillis (c) We overwrite those values by the values computed here. Is this OK ?
    
    We haven't seen a problem so far with this in BucketManager because the time in the event is given a priority over expiry task. However here there maybe an issue because the listener (```BucketsDataManager```) is notified to purge data before start time.
    
    Will club (a) and (c) in one block 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48513818
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Operator, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxCacheSize;
    +
    +  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;
    +
    +  private StateTracker tracker;
    +
    +  @Min(1)
    +  private int numReaders = 10;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private final 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);
    +
    +  private transient StateTracker stateTracker;
    +
    +  private final transient Object commitLock = new Object();
    +
    +  @SuppressWarnings("unchecked")
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorId = context.getId();
    +    fileAccess.init();
    +    timeBucketAssigner.register(dataManager);
    +    timeBucketAssigner.setup(context);
    +
    +    numBuckets = getNumBuckets();
    +    buckets = (Bucket[])Array.newInstance(Bucket.class, 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 = new StateTracker(this, throwable);
    +    stateTracker.setup(context);
    +  }
    +
    +  public abstract int getNumBuckets();
    --- End diff --
    
    Javadoc?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53716067
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    +      }
    +    }
    +  };
    +
    +  private final transient Object lock = new Object();
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    if (!initialized) {
    +      if (bucketSpan == null) {
    +        bucketSpan = Duration.millis(context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +            context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS));
    +      }
    +      Calendar calendar = Calendar.getInstance();
    +      long now = calendar.getTimeInMillis();
    --- End diff --
    
    From @amberarrow 
    System.currentTimeMillis() ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56944460
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    --- End diff --
    
    @ilooner suggested to keep initial time and add millis corresponding to app window to it. So made that 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56767040
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,93 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    +    if (timeBucket != -1) {
    +      int bucketIdx = prepareBucket(bucketId);
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +
    +    synchronized (bucket) {
    +
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    --- End diff --
    
    buckets[bucketIdx] => bucket


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56732445
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * 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 javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    --- End diff --
    
    These are configurable properties so they need to be just set once. Shouldn't be change after application is deployed



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56852888
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    --- End diff --
    
    window id is derived from time and its increasing with time. TimeBucketAssigner reference instant can be configured. If TimeBucketAssigner is configured correctly windowId will not be less than start. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57081100
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.lib.state.TimeSlicedBucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * This implementation of {@link ManagedState} lets the client to specify the time for each key. The value of time
    + * is used to derive the time-bucket of a key.
    + */
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public class ManagedTimeStateImpl extends AbstractManagedStateImpl implements TimeSlicedBucketedState
    +{
    +  public ManagedTimeStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, long time, Slice key, Slice value)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return BucketedState.EXPIRED;
    +    }
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    --- End diff --
    
    getSync and getAsync both return different types of object - Slice and Future<Slice>.
    I can create 2 helper methods for each and reduce the code a little.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765550
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    --- End diff --
    
    `ManagedTimeStateImpl` has a time parameter so seems like this should be `ManagedStateImpl`.
    
    This documentation would be easier to read with a 3-column table -- one column for each of the implementation classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765265
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * An expired value. In some implementations where bucketId is time related then the event can be old and
    +   * the get methods- getSync & getAsync return this fixed slice instance.<br/>
    +   * In the usages, comparisons with EXPIRED should be made using <code>==</code> instead of <code>equals</code>.
    +   */
    +  Slice EXPIRED = new Slice(null, -1, -1);
    +
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key      key
    +   * @param value    value
    +   */
    --- End diff --
    
    value => value (must not be null)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56775333
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,93 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    +    if (timeBucket != -1) {
    +      int bucketIdx = prepareBucket(bucketId);
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    --- End diff --
    
    This method should also acquire the bucket lock incase the user mixes Synchronous and Asynchronous get calls.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57094062
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -51,63 +51,57 @@ public void put(long bucketId, long time, Slice key, Slice value)
         }
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Slice getSync(long bucketId, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
    -
    -    synchronized (bucket) {
    -      return bucket.get(key, -1, Bucket.ReadSource.ALL);
    -    }
    +    return getValueFromBucketSync(bucketId, -1, key);
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Slice getSync(long bucketId, long time, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
         long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
         if (timeBucket == -1) {
           //time is expired so no point in looking further.
           return BucketedState.EXPIRED;
         }
    +    return getValueFromBucketSync(bucketId, timeBucket, key);
    +  }
     
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  private Slice getValueFromBucketSync(long bucketId, long timeBucket, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
         Bucket bucket = buckets[bucketIdx];
         synchronized (bucket) {
           return bucket.get(key, timeBucket, Bucket.ReadSource.ALL);
         }
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Future<Slice> getAsync(long bucketId, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
    -    synchronized (bucket) {
    -      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    -      if (cachedVal != null) {
    -        return Futures.immediateFuture(cachedVal);
    -      }
    -      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    -    }
    +    return getValueFromBucketAsync(bucketId, -1, key);
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Future<Slice> getAsync(long bucketId, long time, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
         long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
         if (timeBucket == -1) {
           //time is expired so no point in looking further.
           return Futures.immediateFuture(BucketedState.EXPIRED);
         }
    +    return getValueFromBucketAsync(bucketId, timeBucket, key);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  private Future<Slice> getValueFromBucketAsync(long bucketId, long timeBucket, Slice key)
    --- End diff --
    
    I moved it to the AbstractManagedStateImpl however can't make it static because they need to access some non-static members and methods - ```buckets``` array and ```prepareBucket()```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765254
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    --- End diff --
    
    Add: A key must not be null.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56738629
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return null;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return null;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    Long purgedTimeBucket;
    +
    +    //tear down all the purged time buckets
    +    while (null != (purgedTimeBucket = purgedTimeBuckets.poll())) {
    +      int purgedTimeBucketIdx = getBucketIdx(purgedTimeBucket);
    +      if (buckets[purgedTimeBucketIdx] != null && buckets[purgedTimeBucketIdx].getBucketId() == purgedTimeBucket) {
    +        buckets[purgedTimeBucketIdx].teardown();
    +        buckets[purgedTimeBucketIdx] = null;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected void handleBucketConflict(int bucketIdx, long newBucketId)
    +  {
    +    Preconditions.checkArgument(buckets[bucketIdx].getBucketId() < newBucketId, "new time bucket should have a value"
    +        + " greater than the old time bucket");
    +    //Time buckets are purged periodically so here a bucket conflict is expected and so we just ignore conflicts.
    +    buckets[bucketIdx].teardown();
    +    buckets[bucketIdx] = newBucket(newBucketId);
    +    buckets[bucketIdx].setup(this);
    +  }
    +
    +  @Override
    +  public void purgeTimeBucketsLessThanEqualTo(long timeBucket)
    +  {
    +    purgedTimeBuckets.add(timeBucket);
    +    super.purgeTimeBucketsLessThanEqualTo(timeBucket);
    +  }
    +
    +  /**
    +   * This uses operator id instead of bucket id as the name of parent folder of time-buckets. This is because
    +   * multiple partitions may work on same time-buckets.
    +   */
    +  public static class TimeUnifiedBucketsFileSystem extends BucketsFileSystem
    --- End diff --
    
    Can this be private ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48513504
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Operator, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxCacheSize;
    +
    +  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;
    +
    +  private StateTracker tracker;
    +
    +  @Min(1)
    +  private int numReaders = 10;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private final 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);
    +
    +  private transient StateTracker stateTracker;
    +
    +  private final transient Object commitLock = new Object();
    +
    +  @SuppressWarnings("unchecked")
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorId = context.getId();
    +    fileAccess.init();
    +    timeBucketAssigner.register(dataManager);
    +    timeBucketAssigner.setup(context);
    +
    +    numBuckets = getNumBuckets();
    +    buckets = (Bucket[])Array.newInstance(Bucket.class, 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 = new StateTracker(this, throwable);
    +    stateTracker.setup(context);
    +  }
    +
    +  public abstract int getNumBuckets();
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    if (throwable.get() != null) {
    +      throw DTThrowable.wrapIfChecked(throwable.get());
    +    }
    +
    +    windowCount++;
    +    timeBucketAssigner.beginWindow(l);
    +    if (replay && l > largestRecoveryWindow) {
    +      replay = false;
    +    }
    +  }
    +
    +  @Override
    +  public void put(long groupId, Slice key, Slice value)
    +  {
    +    if (replay) {
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(groupId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  /**
    +   * Prepares the bucket and returns its index.
    +   * @param groupId bucket key
    +   * @return  bucket index
    +   */
    +  protected int prepareBucket(long groupId)
    +  {
    +    stateTracker.bucketAccessed(groupId);
    +    int bucketIdx = getBucketIdx(groupId);
    +
    +    Bucket bucket = buckets[bucketIdx];
    +    if (bucket == null) {
    +      //bucket is not in memory
    +      bucket = newBucket(groupId);
    +      buckets[bucketIdx] = bucket;
    +    }
    +    return  bucketIdx;
    +  }
    +
    +  protected int getBucketIdx(long bucketKey)
    +  {
    +    return (int)(bucketKey % numBuckets);
    +  }
    +
    +  Bucket getBucket(long groupId)
    +  {
    +    return buckets[getBucketIdx(groupId)];
    +  }
    +
    +  protected Bucket newBucket(long groupId)
    +  {
    +    return new Bucket.DefaultBucket(groupId, this);
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    timeBucketAssigner.endWindow();
    +    if (!replay && windowCount == incrementalCheckpointWindowCount) {
    +      checkpointDifference();
    +      windowCount = 0;
    +    }
    +  }
    +
    +  protected void checkpointDifference()
    +  {
    +    Map<Long, Map<Slice, Bucket.BucketedValue>> flashData = Maps.newHashMap();
    +
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    +        Map<Slice, Bucket.BucketedValue> flashDataForBucket = bucket.checkpoint(windowId);
    +        if (!flashDataForBucket.isEmpty()) {
    +          flashData.put(bucket.getBucketKey(), flashDataForBucket);
    +        }
    +      }
    +    }
    +    if (!flashData.isEmpty()) {
    +      try {
    +        dataManager.save(flashData, operatorId, windowId);
    +      } catch (IOException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    synchronized (commitLock) {
    +      try {
    +        for (Bucket bucket : buckets) {
    +          if (bucket != null) {
    +            bucket.committed(l);
    +          }
    +        }
    +        dataManager.committed(operatorId, l);
    +      } catch (IOException e) {
    +        throw new RuntimeException("committing " + l, e);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    dataManager.teardown();
    +    timeBucketAssigner.teardown();
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    +        bucket.teardown();
    +      }
    +    }
    +    stateTracker.teardown();
    +  }
    +
    +  @Override
    +  public void setCacheSize(long bytes)
    +  {
    +    maxCacheSize = bytes;
    +  }
    +
    +  public long getCacheSize()
    +  {
    +    return maxCacheSize;
    +  }
    +
    +  public void setFileAccess(@NotNull FileAccess fileAccess)
    +  {
    +    this.fileAccess = Preconditions.checkNotNull(fileAccess);
    +  }
    +
    +  public void setKeyComparator(@NotNull Comparator<Slice> keyComparator)
    +  {
    +    this.keyComparator = Preconditions.checkNotNull(keyComparator);
    +  }
    +
    +  @Override
    +  public FileAccess getFileAccess()
    +  {
    +    return fileAccess;
    +  }
    +
    +  public void setTimeBucketAssigner(@NotNull TimeBucketAssigner timeBucketAssigner)
    +  {
    +    this.timeBucketAssigner = Preconditions.checkNotNull(timeBucketAssigner);
    +  }
    +
    +  public TimeBucketAssigner getTimeBucketAssigner()
    +  {
    +    return timeBucketAssigner;
    +  }
    +
    +  @Override
    +  public Comparator<Slice> getKeyComparator()
    +  {
    +    return keyComparator;
    +  }
    +
    +  @Override
    +  public BucketsMetaDataManager getBucketsMetaDataManager()
    +  {
    +    return bucketsMetaDataManager;
    +  }
    +
    +  public int getNumReaders()
    --- End diff --
    
    Haven't completed the documentation yet. There are multiple threads that can load different buckets from disk. This configures the number of threads in the pool. This is why synchronizing on a bucket in the async call so that fetching of a single bucket cannot is done by a single thread.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56730030
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * 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 javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by the expiry task asynchronously.<br/>
    + * The boundaries move only between an application window to ensure consistency of a checkpoint. Checkpoint will happen
    + * at application window boundaries so if we do not restrict moving start and end within an app window boundary, it may
    + * happen that old value of 'start' is saved with the new value of 'end'.
    + * <p/>
    + *
    + * The boundaries can also be moved by {@link #getTimeBucketFor(long)}. The time which is passed as an argument to this
    + * method can be ahead of <code>end</code>. This means that the corresponding event is a future event
    + * (wrt TimeBucketAssigner) and cannot be ignored. Therefore it is accounted by sliding boundaries further.
    + */
    +public class TimeBucketAssigner implements ManagedStateComponent
    +{
    +  @NotNull
    +  private Instant referenceInstant = new Instant();
    +
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long start;
    +  private long end;
    +  private int numBuckets;
    +  private transient long fixedStart;
    +  private transient long lowestTimeBucket;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  private transient PurgeListener purgeListener = null;
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        start += bucketSpanMillis;
    +        end += bucketSpanMillis;
    +        if (purgeListener != null) {
    +          purgeListener.purgeTimeBucketsLessThanEqualTo(lowestTimeBucket++);
    +        }
    +      }
    +    }
    +  };
    +
    +  private final transient Object lock = new Object();
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    Context.OperatorContext context = managedStateContext.getOperatorContext();
    +    fixedStart = referenceInstant.getMillis() - expireBefore.getMillis();
    +
    +    if (!initialized) {
    +      if (bucketSpan == null) {
    +        bucketSpan = Duration.millis(context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +            context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS));
    +      }
    +      start = fixedStart;
    +      bucketSpanMillis = bucketSpan.getMillis();
    +      numBuckets = (int)((expireBefore.getMillis() + bucketSpanMillis - 1) / bucketSpanMillis);
    +      end = start + (numBuckets * bucketSpanMillis);
    +
    +      initialized = true;
    +    }
    +    lowestTimeBucket = (start - fixedStart) / bucketSpanMillis;
    +    windowBoundedService = new WindowBoundedService(bucketSpanMillis, expiryTask);
    +    windowBoundedService.setup(context);
    +  }
    +
    +  public void beginWindow(long windowId)
    +  {
    +    windowBoundedService.beginWindow(windowId);
    +  }
    +
    +  public void endWindow()
    +  {
    +    windowBoundedService.endWindow();
    +  }
    +
    +  /**
    +   * Get the bucket key for the long value.
    +   *
    +   * @param value value from which bucket key is derived.
    +   * @return -1 if value is already expired; bucket key otherwise.
    +   */
    +  public long getTimeBucketFor(long value)
    +  {
    +    synchronized (lock) {
    +      if (value < start) {
    +        return -1;
    +      }
    +      long diffFromStart = value - fixedStart;
    +      long key = diffFromStart / bucketSpanMillis;
    +      if (value > end) {
    +        long move = ((value - end) / bucketSpanMillis + 1) * bucketSpanMillis;
    +        start += move;
    +        end += move;
    +      }
    +      return key;
    +    }
    +  }
    +
    +  public void setPurgeListener(@NotNull PurgeListener purgeListener)
    +  {
    +    this.purgeListener = Preconditions.checkNotNull(purgeListener, "purge listener");
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    windowBoundedService.teardown();
    +  }
    +
    +  /**
    +   * @return number of buckets.
    +   */
    +  public int getNumBuckets()
    +  {
    +    return numBuckets;
    +  }
    +
    +  /**
    +   * @return reference instant
    +   */
    +  public Instant getReferenceInstant()
    +  {
    +    return referenceInstant;
    +  }
    +
    +  /**
    +   * Sets the reference instant (by default the system time when the streaming app is created).
    +   * This instant with {@link #expireBefore} is used to calculate the {@link #start} and {@link #end}.
    +   *
    +   * @param referenceInstant
    +   */
    +  public void setReferenceInstant(Instant referenceInstant)
    +  {
    +    this.referenceInstant = referenceInstant;
    +  }
    +
    +  /**
    +   * @return duration before which the data is expired.
    +   */
    +  public Duration getExpireBefore()
    +  {
    +    return expireBefore;
    +  }
    +
    +  /**
    +   * Sets the duration which denotes expiry. Any event with time before this duration is considered to be expired.
    +   * @param expireBefore duration
    +   */
    +  public void setExpireBefore(Duration expireBefore)
    +  {
    +    this.expireBefore = expireBefore;
    +  }
    +
    +  /**
    +   * @return time-bucket span
    +   */
    +  public Duration getBucketSpan()
    +  {
    +    return bucketSpan;
    +  }
    +
    +  /**
    +   * Sets the length of a time bucket.
    +   * @param bucketSpan length of time bucket
    +   */
    +  public void setBucketSpan(Duration bucketSpan)
    +  {
    +    this.bucketSpan = bucketSpan;
    +  }
    +
    +  /**
    +   * The listeners are informed when the time slides and time buckets which are older than the smallest time bucket
    --- End diff --
    
    listeners are => listener is


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56779211
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    --- End diff --
    
    There is another thread reading from the row so I think I need to synchronize on the MutableTimeBucket in the row.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53726983
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/StateTracker.java ---
    @@ -0,0 +1,153 @@
    +/**
    + * 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.PriorityQueue;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Tracks the size of state in memory and evicts buckets.
    + */
    +class StateTracker extends TimerTask implements Component<Context.OperatorContext>
    --- End diff --
    
    Most of the classes that I have added implement Component. The reason for making them implement Component is that the ```setup(...)``` and ```teardown(...)``` also becomes part of their definition.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56770642
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    +    Preconditions.checkNotNull(timeBuckets, "timeBuckets");
    +    String tmpFileName = getTmpFileName();
    +
    +    try (DataOutputStream dos = getOutputStream(bucketId, tmpFileName)) {
    +      dos.writeInt(timeBuckets.size());
    +      for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBuckets.entrySet()) {
    +        MutableTimeBucketMeta tbm = entry.getValue();
    +        dos.writeLong(tbm.getTimeBucketId());
    +        dos.writeLong(tbm.getSizeInBytes());
    +        dos.writeLong(tbm.getLastTransferredWindowId());
    +        dos.writeInt(tbm.getFirstKey().length);
    +        dos.write(tbm.getFirstKey().toByteArray());
    +      }
    +
    +    }
    +    rename(bucketId, tmpFileName, META_FILE_NAME);
    +  }
    +
    +  protected void deleteTimeBucketsLessThanEqualTo(long latestExpiredTimeBucket) throws IOException
    +  {
    +    LOG.debug("delete files before {}", latestExpiredTimeBucket);
    +
    +    for (long bucketName : bucketNamesOnFS) {
    +      RemoteIterator<LocatedFileStatus> timeBucketsIterator = listFiles(bucketName);
    +      boolean emptyBucket = true;
    +      while (timeBucketsIterator.hasNext()) {
    +        LocatedFileStatus timeBucketStatus = timeBucketsIterator.next();
    +
    +        String timeBucketStr = timeBucketStatus.getPath().getName();
    +        if (timeBucketStr.equals(BucketsFileSystem.META_FILE_NAME) || timeBucketStr.endsWith(".tmp")) {
    +          //ignoring meta and tmp files
    +          continue;
    +        }
    +        long timeBucket = Long.parseLong(timeBucketStr);
    +
    +        if (timeBucket <= latestExpiredTimeBucket) {
    +          LOG.debug("deleting bucket {} time-bucket {}", timeBucket);
    +          delete(bucketName, timeBucketStatus.getPath().getName());
    +
    +          invalidateTimeBucket(bucketName, timeBucket);
    +        } else {
    +          emptyBucket = false;
    +        }
    +      }
    +      if (emptyBucket) {
    +        LOG.debug("deleting bucket {}", bucketName);
    +        deleteBucket(bucketName);
    +      }
    +    }
    +  }
    +
    +  void invalidateTimeBucket(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      timeBucketsMeta.remove(bucketId, timeBucketId);
    +    }
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +  }
    +
    +  private static class TimeBucketMeta implements Comparable<TimeBucketMeta>
    +  {
    +    protected final long bucketId;
    +    protected final long timeBucketId;
    +    protected long lastTransferredWindowId = -1;
    +    protected long sizeInBytes;
    +    protected Slice firstKey;
    +
    +    private TimeBucketMeta()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      timeBucketId = -1;
    +    }
    +
    +    private TimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      this.bucketId = bucketId;
    +      this.timeBucketId = timeBucketId;
    +    }
    +
    +    public long getLastTransferredWindowId()
    +    {
    +      return lastTransferredWindowId;
    +    }
    +
    +    public long getSizeInBytes()
    +    {
    +      return this.sizeInBytes;
    +    }
    +
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    public long getTimeBucketId()
    +    {
    +      return timeBucketId;
    +    }
    +
    +    public Slice getFirstKey()
    +    {
    +      return firstKey;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof TimeBucketMeta)) {
    +        return false;
    +      }
    +
    +      TimeBucketMeta that = (TimeBucketMeta)o;
    +
    +      return bucketId == that.bucketId && timeBucketId == that.timeBucketId;
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(bucketId, timeBucketId);
    +    }
    +
    +    @Override
    +    public int compareTo(@NotNull TimeBucketMeta o)
    +    {
    +      if (bucketId < o.bucketId) {
    +        return -1;
    +      }
    +      if (bucketId > o.bucketId) {
    +        return 1;
    +      }
    +      if (timeBucketId < o.timeBucketId) {
    +        return -1;
    +      }
    +      if (timeBucketId > o.timeBucketId) {
    +        return 1;
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /**
    +   * Represents time bucket meta information which can be changed.
    +   * The updates to an instance and read/creation of {@link #immutableTimeBucketMeta} belonging to it are synchronized
    +   * as different threads are updating and reading from it.<br/>
    +   *
    +   * The instance is updated when data from window files are transferred to bucket files and
    +   * {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket} reads the immutable time bucket meta.
    +   */
    +  static class MutableTimeBucketMeta extends TimeBucketMeta
    +  {
    +    private transient ImmutableTimeBucketMeta immutableTimeBucketMeta;
    +
    +    private volatile boolean changed;
    +
    +    MutableTimeBucketMeta()
    +    {
    +      //for kryo
    +    }
    +
    +    public MutableTimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      super(bucketId, timeBucketId);
    +    }
    +
    +    synchronized void updateTimeBucketMeta(long lastTransferredWindow, long bytes, @NotNull Slice firstKey)
    +    {
    +      changed = true;
    +      this.lastTransferredWindowId = lastTransferredWindow;
    +      this.sizeInBytes = bytes;
    +      this.firstKey = Preconditions.checkNotNull(firstKey, "first key");
    +    }
    +
    +    synchronized ImmutableTimeBucketMeta getImmutableTimeBucketMeta()
    +    {
    +      if (immutableTimeBucketMeta == null || changed) {
    +        immutableTimeBucketMeta = new ImmutableTimeBucketMeta(getBucketId(), getTimeBucketId(),
    +            getLastTransferredWindowId(), getSizeInBytes(), getFirstKey());
    +        changed = false;
    +      }
    +      return immutableTimeBucketMeta;
    +    }
    +
    +  }
    +
    +  /**
    +   * This serves the readers - {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket}.
    +   * It is accessible outside the package unlike {@link MutableTimeBucketMeta} that can be only modified by
    +   * BucketPersistor.
    --- End diff --
    
    BucketPersistor ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56771890
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    --- End diff --
    
    synchronized block could end here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57083270
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.lib.state.TimeSlicedBucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * This implementation of {@link ManagedState} lets the client to specify the time for each key. The value of time
    + * is used to derive the time-bucket of a key.
    + */
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public class ManagedTimeStateImpl extends AbstractManagedStateImpl implements TimeSlicedBucketedState
    +{
    +  public ManagedTimeStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, long time, Slice key, Slice value)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return BucketedState.EXPIRED;
    +    }
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    --- End diff --
    
    Yes, I think it's worth doing for 2 reasons: (a) Reduces code duplication thus helping with maintainability and (b) Helps readability since the reader wouldn't have to visually compare code fragments to confirm that all locations are doing the same thing.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53726516
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/StateTracker.java ---
    @@ -0,0 +1,153 @@
    +/**
    + * 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.PriorityQueue;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Tracks the size of state in memory and evicts buckets.
    + */
    +class StateTracker extends TimerTask implements Component<Context.OperatorContext>
    --- End diff --
    
    From @amberarrow 
    Why does this class need to implement Component ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56718885
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,60 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in a bucket identified by bucketId. Fetching a key can be expensive if the key
    +   * is not in memory and is present on disk. This fetches the key synchronously. <br/>
    +   * {@link #getAsync(long, Slice)} is recommended for efficiently reading the value of a key.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @return        value
    +   */
    +  Slice getSync(long bucketId, Slice key);
    +
    +  /**
    +   * Returns the future using which the value is obtained.<br/>
    +   * If the key is present in memory, then the future has its value set when constructed.
    --- End diff --
    
    Suggest omitting "in memory" or replacing with "in the bucket" (multiple places)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57335712
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,499 @@
    +/**
    + * 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.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.base.Throwables;
    +import com.google.common.collect.Maps;
    +
    +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.Slice;
    +
    +/**
    + * An abstract implementation of managed state.<br/>
    + *
    + * The important sub-components here are:
    + * <ol>
    + *   <li>
    + *     {@link #checkpointManager}: writes incremental checkpoints in window files and transfers data from window
    + *     files to bucket files.
    + *   </li>
    + *   <li>
    + *     {@link #bucketsFileSystem}: a bucket on disk is sub-divided into time-buckets. This manages meta-bucket
    + *     information (list of {@link BucketsFileSystem.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}: plug-able 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.CheckpointNotificationListener, ManagedStateContext,
    +    TimeBucketAssigner.PurgeListener
    +{
    +  private long maxMemorySize;
    +
    +  protected int numBuckets;
    +
    +  @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
    +  protected IncrementalCheckpointManager checkpointManager = new IncrementalCheckpointManager();
    +
    +  @NotNull
    +  protected BucketsFileSystem bucketsFileSystem = new BucketsFileSystem();
    +
    +  protected transient OperatorContext operatorContext;
    +  protected transient long windowId;
    +
    +  @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();
    +
    +  //accessible to StateTracker
    +  final transient Object commitLock = new Object();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorContext = context;
    +    fileAccess.init();
    +
    +    timeBucketAssigner.setPurgeListener(this);
    +
    +    //setup all the managed state components
    +    timeBucketAssigner.setup(this);
    +    checkpointManager.setup(this);
    +    bucketsFileSystem.setup(this);
    +
    +    if (buckets == null) {
    +      //create buckets array only once at start when it is not created.
    +      numBuckets = getNumBuckets();
    +      buckets = new Bucket[numBuckets];
    +    }
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    +        bucket.setup(this);
    +      }
    +    }
    +
    +    Preconditions.checkArgument(numReaders <= numBuckets, "no. of readers cannot exceed no. of buckets");
    +
    +    stateTracker.setup(this);
    +    long activationWindow = context.getValue(OperatorContext.ACTIVATION_WINDOW_ID);
    +
    +    if (activationWindow != Stateless.WINDOW_ID) {
    +      //delete all the wal files with windows > activationWindow.
    +      //All the wal files with windows <= activationWindow are loaded and kept separately as recovered data.
    +      try {
    +        for (long recoveredWindow : checkpointManager.getWindowIds(operatorContext.getId())) {
    +          if (recoveredWindow <= activationWindow) {
    +            @SuppressWarnings("unchecked")
    +            Map<Long, Map<Slice, Bucket.BucketedValue>> recoveredData =
    +                (Map<Long, Map<Slice, Bucket.BucketedValue>>)checkpointManager.load(operatorContext.getId(), windowId);
    --- End diff --
    
    windowId => recoveredWindow ?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56734755
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * 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 javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    --- End diff --
    
    It's worth adding a comment to that effect; otherwise, users may try to do something clever like increasing the span when data is sparse and decreasing it when data is dense.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56752821
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -29,11 +31,17 @@
     public interface BucketedState
     {
       /**
    +   * An expired value. In some implementations where bucketId is time related then the event can be old and
    +   * the get methods- getSync & getAsync return this special slice instance.
    +   */
    +  Slice EXPIRED = new Slice(Ints.toByteArray(-1));
    --- End diff --
    
    I can use Slice(null, -1, -1). In the usages, comparisons should be made using ```==``` and not ```equals```.
    
    For every expired event I didn't want to instantiate a Slice object. So even if I extend Slice, I will re-use a special instance of slice so thought why not use just a special instance.
    
    There is a problem with all the approaches I can think of which is that Slice cannot have an immutable extension because all the members are publicly exposed without getters and setters.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56775791
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    protected transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsFileSystem.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +    }
    +
    +    protected DefaultBucket(long bucketId)
    +    {
    +      this.bucketId = bucketId;
    +    }
    +
    +    @Override
    +    public void setup(@NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traverse the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      for (Long window : recoveredData.descendingKeySet()) {
    +        //traverse the reccovered data in reverse order
    +        bucketedValue = recoveredData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    +        }
    +        return valSlice;
    +      } else {
    +        //search all the time buckets
    +        try {
    +          if (cachedBucketMetas == null) {
    +            cachedBucketMetas = managedStateContext.getBucketsFileSystem().getAllTimeBuckets(bucketId);
    +          }
    +
    +          for (BucketsFileSystem.ImmutableTimeBucketMeta immutableTimeBucketMeta : cachedBucketMetas) {
    +
    +            if (managedStateContext.getKeyComparator().compare(key, immutableTimeBucketMeta.getFirstKey()) >= 0) {
    +              //keys in the time bucket files are sorted so if the first key in the file is greater than the key being
    +              //searched, the key will not be present in that file.
    +              Slice valSlice = getKeyFromTimeBucketReader(key, immutableTimeBucketMeta.getTimeBucketId());
    +              if (valSlice != null) {
    +                BucketedValue bucketedValue = new BucketedValue(immutableTimeBucketMeta.getTimeBucketId(), valSlice);
    +                readCache.put(key, bucketedValue);
    +                return valSlice;
    +              }
    +            }
    +          }
    +
    +        } catch (IOException e) {
    +          throw new RuntimeException("get time-buckets " + bucketId, e);
    +        }
    +      }
    +      return null;
    +    }
    +
    +    @Override
    +    public Slice get(Slice key, long timeBucket, ReadSource readSource)
    +    {
    +      switch (readSource) {
    +        case MEMORY:
    +          return getFromMemory(key);
    +        case READERS:
    +          return getFromReaders(key, timeBucket);
    +        case ALL:
    +        default:
    +          Slice value = getFromMemory(key);
    +          if (value != null) {
    +            return value;
    +          }
    +          return getFromReaders(key, timeBucket);
    +      }
    +    }
    +
    +    /**
    +     * Returns the value for the key from a time-bucket reader
    +     * @param key        key
    +     * @param timeBucket time bucket
    +     * @return value if key is found in the time bucket; false otherwise
    +     */
    +    private Slice getKeyFromTimeBucketReader(Slice key, long timeBucket)
    +    {
    +      FileAccess.FileReader fileReader = readers.get(timeBucket);
    +      if (fileReader != null) {
    +        return readKey(fileReader, key, timeBucket);
    +      }
    +      //file reader is not loaded and is null
    +      try {
    +        if (loadFileReader(timeBucket)) {
    +          return readKey(readers.get(timeBucket), key, timeBucket);
    +        }
    +        return null;
    +      } catch (IOException e) {
    +        throw new RuntimeException("while loading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private Slice readKey(FileAccess.FileReader fileReader, Slice key, long timeBucket)
    +    {
    +      Slice valSlice = new Slice(null, 0, 0);
    +      try {
    +        if (fileReader.seek(key)) {
    +          fileReader.next(dummyGetKey, valSlice);
    +          return valSlice;
    +        } else {
    +          return null;
    +        }
    +      } catch (IOException e) {
    +        throw new RuntimeException("reading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private boolean loadFileReader(long timeBucketId) throws IOException
    +    {
    +      BucketsFileSystem.ImmutableTimeBucketMeta tbm = managedStateContext.getBucketsFileSystem()
    +          .getTimeBucketMeta(bucketId, timeBucketId);
    +
    +      if (tbm != null) {
    +        FileAccess.FileReader reader = managedStateContext.getBucketsFileSystem().getReader(bucketId,
    +            BucketsFileSystem.getFileName(timeBucketId));
    +        readers.put(timeBucketId, reader);
    +        sizeInBytes.getAndAdd(tbm.getSizeInBytes());
    +        return true;
    +      }
    +      return false;
    +    }
    +
    +    @Override
    +    public void put(Slice key, long timeBucket, Slice value)
    +    {
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue == null) {
    +        bucketedValue = new BucketedValue();
    +        flash.put(key, bucketedValue);
    --- End diff --
    
    Should be a concurrent hash map


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53715681
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    +      }
    +    }
    +  };
    +
    +  private final transient Object lock = new Object();
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    if (!initialized) {
    +      if (bucketSpan == null) {
    +        bucketSpan = Duration.millis(context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +            context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS));
    +      }
    +      Calendar calendar = Calendar.getInstance();
    +      long now = calendar.getTimeInMillis();
    +      fixedStartTime = now - expireBefore.getMillis();
    +      startTime = fixedStartTime;
    +
    +      bucketSpanMillis = bucketSpan.getMillis();
    +      numBuckets = (int)((expireBefore.getMillis() + bucketSpanMillis - 1) / bucketSpanMillis);
    +      endTime = startTime + (numBuckets * bucketSpanMillis);
    +
    +      initialized = true;
    +    }
    +    windowBoundedService = new WindowBoundedService(bucketSpanMillis, expiryTask);
    +    windowBoundedService.setup(context);
    +  }
    +
    +  public void beginWindow(long l)
    +  {
    +    windowBoundedService.beginWindow(l);
    +  }
    +
    +  public void endWindow()
    +  {
    +    windowBoundedService.endWindow();
    +  }
    +
    +  /**
    +   * Get the bucket key for the long value.
    +   *
    +   * @param value value from which bucket key is derived.
    +   * @return -1 if value is already expired; bucket key otherwise.
    +   */
    +  public long getTimeBucketFor(long value)
    +  {
    +    long lstart;
    +    long lend;
    +    synchronized (lock) {
    +      lstart = startTime;
    +      lend = endTime;
    +    }
    +    if (value < lstart) {
    +      return -1;
    +    }
    +    long diffFromStart = value - fixedStartTime;
    +    long key = diffFromStart / bucketSpanMillis;
    +    if (value > lend) {
    +      long move = ((value - lend) / bucketSpanMillis + 1) * bucketSpanMillis;
    +      synchronized (lock) {
    +        startTime = lstart + move;
    +        endTime = lend + move;
    +      }
    --- End diff --
    
    From @amberarrow 
    
    Not clear what we're trying to achieve with the code above; could you explain ? Can (value - lend) ever exceed bucketSpanMillis ?
    Second, it seems like this assignment could overwrite the assignment made to startTime and endTime by the expiryTask thread in some scenarios, for example: (a) We copy the values in the first synchronized block above (b) The expiryTask thread increments values by bucketSpanMillis (c) We overwrite those values by the values computed here. Is this OK ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56845821
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    --- End diff --
    
    `getTimeBucketFor` expects its argument to be a timestamp somewhere near the current time; the `windowId`, at least when run in local mode, seems to be a much larger value, e.g.
    windowId = 6264531193618759682, current = 1458574832005, diff = 6264529735043927677
    
    So, how is this expected to work ? Do we have any platform guarantees about the windowId ? If the windowId happens be less than `start` would we not always get a -1 back ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56779610
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    --- End diff --
    
    Will make the suggested change, i.e. synchronize block cover most of the function


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56759001
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    protected transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsFileSystem.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +    }
    +
    +    protected DefaultBucket(long bucketId)
    +    {
    +      this.bucketId = bucketId;
    +    }
    +
    +    @Override
    +    public void setup(@NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traverse the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      for (Long window : recoveredData.descendingKeySet()) {
    +        //traverse the reccovered data in reverse order
    +        bucketedValue = recoveredData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    --- End diff --
    
    If the user does a get for key "A" time bucket 1 first. Then that value will be put in the read cache. If the user then does a get for key "A" time bucket 2, then the time bucket 1 value is retrieved from the read cache and returned. So the value for an older timebucket is returned.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48512962
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,49 @@
    +/*
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +public interface BucketedState
    --- End diff --
    
    Javadoc?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53715458
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    --- End diff --
    
    From @amberarrow 
    
    Seems like the operator thread could also move them in getTimeBucketFor(). Are startTime and endTime intended to bound the current time bucket ? If so, it's worth stating it here.
    
    
    -----
    ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57249937
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,60 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in a bucket identified by bucketId. Fetching a key can be expensive if the key
    +   * is not in memory and is present on disk. This fetches the key synchronously. <br/>
    +   * {@link #getAsync(long, Slice)} is recommended for efficiently reading the value of a key.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @return        value
    +   */
    +  Slice getSync(long bucketId, Slice key);
    +
    +  /**
    +   * Returns the future using which the value is obtained.<br/>
    +   * If the key is present in memory, then the future has its value set when constructed.
    --- End diff --
    
    Oh, I misunderstood the comment.
    
    If the intent is to say "If the key is present in memory, then the future has its value set when constructed; if not the value is set some time after construction"
    (where I've added the part after the semicolon), then isn't this an implementation detail that is better placed in the implementation file ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48513206
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Operator, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxCacheSize;
    +
    +  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;
    +
    +  private StateTracker tracker;
    +
    +  @Min(1)
    +  private int numReaders = 10;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private final 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);
    +
    +  private transient StateTracker stateTracker;
    +
    +  private final transient Object commitLock = new Object();
    +
    +  @SuppressWarnings("unchecked")
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorId = context.getId();
    +    fileAccess.init();
    +    timeBucketAssigner.register(dataManager);
    +    timeBucketAssigner.setup(context);
    +
    +    numBuckets = getNumBuckets();
    +    buckets = (Bucket[])Array.newInstance(Bucket.class, 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 = new StateTracker(this, throwable);
    +    stateTracker.setup(context);
    +  }
    +
    +  public abstract int getNumBuckets();
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    if (throwable.get() != null) {
    +      throw DTThrowable.wrapIfChecked(throwable.get());
    +    }
    +
    +    windowCount++;
    +    timeBucketAssigner.beginWindow(l);
    +    if (replay && l > largestRecoveryWindow) {
    +      replay = false;
    +    }
    +  }
    +
    +  @Override
    +  public void put(long groupId, Slice key, Slice value)
    +  {
    +    if (replay) {
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(groupId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    Bucket bucket = buckets[bucketIdx];
    --- End diff --
    
    Why the local variable?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56754928
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/TimeSlicedBucketedState.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A type of bucketed state where a bucket's data is further divided into time buckets. This requires
    + * time per key to figure out which time bucket a particular key belongs to.
    + * <p/>
    + * The time value eases purging of aged key/value pair.
    + */
    +public interface TimeSlicedBucketedState
    +{
    +  /**
    +   * Sets the value of a key in the bucket identified by bucketId. Time is used to derive which time bucket (within
    +   * the main bucket) a key belongs to.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param time    time associated with the key.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, long time, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in the bucket identified by bucketId. This will search for the key in all the
    +   * time buckets.</br>
    +   *
    --- End diff --
    
    The latest value will be returned. Will update the java docs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53727076
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/StateTracker.java ---
    @@ -0,0 +1,153 @@
    +/**
    + * 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.PriorityQueue;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +import java.util.concurrent.ConcurrentHashMap;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +
    +/**
    + * Tracks the size of state in memory and evicts buckets.
    + */
    +class StateTracker extends TimerTask implements Component<Context.OperatorContext>
    +{
    +  //bucket id -> last time the bucket was accessed
    +  private final transient ConcurrentHashMap<Long, Long> bucketAccessTimes = new ConcurrentHashMap<>();
    +
    +  private transient PriorityQueue<Map.Entry<Long, Long>> bucketHeap;
    --- End diff --
    
    From @amberarrow 
    One last suggestion to think about: ConcurrentSkipListSet may work better and also avoid re-populating and clearing the heap each time since it is thread-safe and maintains sorted order. Feel free to stay with the current version if you prefer it.
    
    The synchronization cost is sometimes unavoidable when multiple threads are involved. For example, TimeBucketAssigner.getTimeBucketFor() is called by all the put, getSync and getAsync methods and yet has 2 synchronized blocks.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r54147037
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,101 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    --- End diff --
    
    @PramodSSImmaneni  
    This is a comment/question about design and large state management design document was shared couple of months back with dev@apex. 
    
    IMO the pull request if for code review. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48513313
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Operator, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxCacheSize;
    +
    +  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;
    +
    +  private StateTracker tracker;
    +
    +  @Min(1)
    +  private int numReaders = 10;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private final 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);
    +
    +  private transient StateTracker stateTracker;
    +
    +  private final transient Object commitLock = new Object();
    +
    +  @SuppressWarnings("unchecked")
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorId = context.getId();
    +    fileAccess.init();
    +    timeBucketAssigner.register(dataManager);
    +    timeBucketAssigner.setup(context);
    +
    +    numBuckets = getNumBuckets();
    +    buckets = (Bucket[])Array.newInstance(Bucket.class, 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 = new StateTracker(this, throwable);
    +    stateTracker.setup(context);
    +  }
    +
    +  public abstract int getNumBuckets();
    +
    +  @Override
    +  public void beginWindow(long l)
    +  {
    +    if (throwable.get() != null) {
    +      throw DTThrowable.wrapIfChecked(throwable.get());
    +    }
    +
    +    windowCount++;
    +    timeBucketAssigner.beginWindow(l);
    +    if (replay && l > largestRecoveryWindow) {
    +      replay = false;
    +    }
    +  }
    +
    +  @Override
    +  public void put(long groupId, Slice key, Slice value)
    +  {
    +    if (replay) {
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(groupId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long groupId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(groupId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  /**
    +   * Prepares the bucket and returns its index.
    +   * @param groupId bucket key
    +   * @return  bucket index
    +   */
    +  protected int prepareBucket(long groupId)
    +  {
    +    stateTracker.bucketAccessed(groupId);
    +    int bucketIdx = getBucketIdx(groupId);
    +
    +    Bucket bucket = buckets[bucketIdx];
    +    if (bucket == null) {
    +      //bucket is not in memory
    +      bucket = newBucket(groupId);
    +      buckets[bucketIdx] = bucket;
    +    }
    +    return  bucketIdx;
    +  }
    +
    +  protected int getBucketIdx(long bucketKey)
    +  {
    +    return (int)(bucketKey % numBuckets);
    +  }
    +
    +  Bucket getBucket(long groupId)
    +  {
    +    return buckets[getBucketIdx(groupId)];
    +  }
    +
    +  protected Bucket newBucket(long groupId)
    +  {
    +    return new Bucket.DefaultBucket(groupId, this);
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    timeBucketAssigner.endWindow();
    +    if (!replay && windowCount == incrementalCheckpointWindowCount) {
    +      checkpointDifference();
    +      windowCount = 0;
    +    }
    +  }
    +
    +  protected void checkpointDifference()
    +  {
    +    Map<Long, Map<Slice, Bucket.BucketedValue>> flashData = Maps.newHashMap();
    +
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    +        Map<Slice, Bucket.BucketedValue> flashDataForBucket = bucket.checkpoint(windowId);
    +        if (!flashDataForBucket.isEmpty()) {
    +          flashData.put(bucket.getBucketKey(), flashDataForBucket);
    +        }
    +      }
    +    }
    +    if (!flashData.isEmpty()) {
    +      try {
    +        dataManager.save(flashData, operatorId, windowId);
    +      } catch (IOException e) {
    +        throw new RuntimeException(e);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +  }
    +
    +  @Override
    +  public void committed(long l)
    +  {
    +    synchronized (commitLock) {
    +      try {
    +        for (Bucket bucket : buckets) {
    +          if (bucket != null) {
    +            bucket.committed(l);
    +          }
    +        }
    +        dataManager.committed(operatorId, l);
    +      } catch (IOException e) {
    +        throw new RuntimeException("committing " + l, e);
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +    dataManager.teardown();
    +    timeBucketAssigner.teardown();
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    +        bucket.teardown();
    +      }
    +    }
    +    stateTracker.teardown();
    +  }
    +
    +  @Override
    +  public void setCacheSize(long bytes)
    +  {
    +    maxCacheSize = bytes;
    +  }
    +
    +  public long getCacheSize()
    +  {
    +    return maxCacheSize;
    +  }
    +
    +  public void setFileAccess(@NotNull FileAccess fileAccess)
    +  {
    +    this.fileAccess = Preconditions.checkNotNull(fileAccess);
    +  }
    +
    +  public void setKeyComparator(@NotNull Comparator<Slice> keyComparator)
    +  {
    +    this.keyComparator = Preconditions.checkNotNull(keyComparator);
    +  }
    +
    +  @Override
    +  public FileAccess getFileAccess()
    +  {
    +    return fileAccess;
    +  }
    +
    +  public void setTimeBucketAssigner(@NotNull TimeBucketAssigner timeBucketAssigner)
    +  {
    +    this.timeBucketAssigner = Preconditions.checkNotNull(timeBucketAssigner);
    +  }
    +
    +  public TimeBucketAssigner getTimeBucketAssigner()
    +  {
    +    return timeBucketAssigner;
    +  }
    +
    +  @Override
    +  public Comparator<Slice> getKeyComparator()
    +  {
    +    return keyComparator;
    +  }
    +
    +  @Override
    +  public BucketsMetaDataManager getBucketsMetaDataManager()
    +  {
    +    return bucketsMetaDataManager;
    +  }
    +
    +  public int getNumReaders()
    --- End diff --
    
    Purpose of numReaders?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56766131
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return BucketedState.EXPIRED;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    --- End diff --
    
    return null. => return expired token.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56754879
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/TimeSlicedBucketedState.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A type of bucketed state where a bucket's data is further divided into time buckets. This requires
    + * time per key to figure out which time bucket a particular key belongs to.
    + * <p/>
    + * The time value eases purging of aged key/value pair.
    + */
    +public interface TimeSlicedBucketedState
    +{
    +  /**
    +   * Sets the value of a key in the bucket identified by bucketId. Time is used to derive which time bucket (within
    +   * the main bucket) a key belongs to.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param time    time associated with the key.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, long time, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in the bucket identified by bucketId. This will search for the key in all the
    +   * time buckets.</br>
    +   *
    --- End diff --
    
    If a key is present in multiple time buckets, which value is returned?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56770280
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    +    Preconditions.checkNotNull(timeBuckets, "timeBuckets");
    +    String tmpFileName = getTmpFileName();
    +
    +    try (DataOutputStream dos = getOutputStream(bucketId, tmpFileName)) {
    +      dos.writeInt(timeBuckets.size());
    +      for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBuckets.entrySet()) {
    +        MutableTimeBucketMeta tbm = entry.getValue();
    +        dos.writeLong(tbm.getTimeBucketId());
    +        dos.writeLong(tbm.getSizeInBytes());
    +        dos.writeLong(tbm.getLastTransferredWindowId());
    +        dos.writeInt(tbm.getFirstKey().length);
    +        dos.write(tbm.getFirstKey().toByteArray());
    +      }
    +
    +    }
    +    rename(bucketId, tmpFileName, META_FILE_NAME);
    +  }
    +
    +  protected void deleteTimeBucketsLessThanEqualTo(long latestExpiredTimeBucket) throws IOException
    +  {
    +    LOG.debug("delete files before {}", latestExpiredTimeBucket);
    +
    +    for (long bucketName : bucketNamesOnFS) {
    +      RemoteIterator<LocatedFileStatus> timeBucketsIterator = listFiles(bucketName);
    +      boolean emptyBucket = true;
    +      while (timeBucketsIterator.hasNext()) {
    +        LocatedFileStatus timeBucketStatus = timeBucketsIterator.next();
    +
    +        String timeBucketStr = timeBucketStatus.getPath().getName();
    +        if (timeBucketStr.equals(BucketsFileSystem.META_FILE_NAME) || timeBucketStr.endsWith(".tmp")) {
    +          //ignoring meta and tmp files
    +          continue;
    +        }
    +        long timeBucket = Long.parseLong(timeBucketStr);
    +
    +        if (timeBucket <= latestExpiredTimeBucket) {
    +          LOG.debug("deleting bucket {} time-bucket {}", timeBucket);
    +          delete(bucketName, timeBucketStatus.getPath().getName());
    +
    +          invalidateTimeBucket(bucketName, timeBucket);
    +        } else {
    +          emptyBucket = false;
    +        }
    +      }
    +      if (emptyBucket) {
    +        LOG.debug("deleting bucket {}", bucketName);
    +        deleteBucket(bucketName);
    +      }
    +    }
    +  }
    +
    +  void invalidateTimeBucket(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      timeBucketsMeta.remove(bucketId, timeBucketId);
    +    }
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +  }
    +
    +  private static class TimeBucketMeta implements Comparable<TimeBucketMeta>
    +  {
    +    protected final long bucketId;
    +    protected final long timeBucketId;
    +    protected long lastTransferredWindowId = -1;
    +    protected long sizeInBytes;
    +    protected Slice firstKey;
    +
    +    private TimeBucketMeta()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      timeBucketId = -1;
    +    }
    +
    +    private TimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      this.bucketId = bucketId;
    +      this.timeBucketId = timeBucketId;
    +    }
    +
    +    public long getLastTransferredWindowId()
    +    {
    +      return lastTransferredWindowId;
    +    }
    +
    +    public long getSizeInBytes()
    +    {
    +      return this.sizeInBytes;
    +    }
    +
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    public long getTimeBucketId()
    +    {
    +      return timeBucketId;
    +    }
    +
    +    public Slice getFirstKey()
    +    {
    +      return firstKey;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof TimeBucketMeta)) {
    +        return false;
    +      }
    +
    +      TimeBucketMeta that = (TimeBucketMeta)o;
    +
    +      return bucketId == that.bucketId && timeBucketId == that.timeBucketId;
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(bucketId, timeBucketId);
    +    }
    +
    +    @Override
    +    public int compareTo(@NotNull TimeBucketMeta o)
    +    {
    +      if (bucketId < o.bucketId) {
    +        return -1;
    +      }
    +      if (bucketId > o.bucketId) {
    +        return 1;
    +      }
    +      if (timeBucketId < o.timeBucketId) {
    +        return -1;
    +      }
    +      if (timeBucketId > o.timeBucketId) {
    +        return 1;
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /**
    +   * Represents time bucket meta information which can be changed.
    +   * The updates to an instance and read/creation of {@link #immutableTimeBucketMeta} belonging to it are synchronized
    +   * as different threads are updating and reading from it.<br/>
    +   *
    +   * The instance is updated when data from window files are transferred to bucket files and
    +   * {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket} reads the immutable time bucket meta.
    +   */
    +  static class MutableTimeBucketMeta extends TimeBucketMeta
    +  {
    +    private transient ImmutableTimeBucketMeta immutableTimeBucketMeta;
    +
    +    private volatile boolean changed;
    +
    +    MutableTimeBucketMeta()
    +    {
    +      //for kryo
    +    }
    +
    +    public MutableTimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      super(bucketId, timeBucketId);
    +    }
    +
    +    synchronized void updateTimeBucketMeta(long lastTransferredWindow, long bytes, @NotNull Slice firstKey)
    +    {
    +      changed = true;
    +      this.lastTransferredWindowId = lastTransferredWindow;
    +      this.sizeInBytes = bytes;
    +      this.firstKey = Preconditions.checkNotNull(firstKey, "first key");
    +    }
    +
    +    synchronized ImmutableTimeBucketMeta getImmutableTimeBucketMeta()
    +    {
    +      if (immutableTimeBucketMeta == null || changed) {
    +        immutableTimeBucketMeta = new ImmutableTimeBucketMeta(getBucketId(), getTimeBucketId(),
    +            getLastTransferredWindowId(), getSizeInBytes(), getFirstKey());
    +        changed = false;
    +      }
    +      return immutableTimeBucketMeta;
    +    }
    +
    +  }
    +
    +  /**
    +   * This serves the readers - {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket}.
    +   * It is accessible outside the package unlike {@link MutableTimeBucketMeta} that can be only modified by
    +   * BucketPersistor.
    +   */
    +  public static final class ImmutableTimeBucketMeta extends TimeBucketMeta
    --- End diff --
    
    Do we need this class ?
    `TimeBucketMeta` itself seems immutable since it has no setters and private constructors, so could we not just use that instead ?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56752949
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -29,11 +31,17 @@
     public interface BucketedState
     {
       /**
    +   * An expired value. In some implementations where bucketId is time related then the event can be old and
    +   * the get methods- getSync & getAsync return this special slice instance.
    +   */
    +  Slice EXPIRED = new Slice(Ints.toByteArray(-1));
    --- End diff --
    
    Using a fixed instance is fine. Worth documenting the `==` part.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56758801
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    --- End diff --
    
    can recovered data be treated the same as checkpointed data?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

Posted by chandnisingh <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56750393
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -29,11 +31,17 @@
     public interface BucketedState
     {
       /**
    +   * An expired value. In some implementations where bucketId is time related then the event can be old and
    +   * the get methods- getSync & getAsync return this special slice instance.
    +   */
    +  Slice EXPIRED = new Slice(Ints.toByteArray(-1));
    --- End diff --
    
    Couldn't -1 be a valid user value ?
    Might be better to use invalid slices, e.g. Slice(null, -1, -1). Or, if you want to be more user friendly, extend Slice and add an isExpired() method which checks offset and length against -1.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56767061
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.lib.state.TimeSlicedBucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * This implementation of {@link ManagedState} lets the client to specify the time for each key. The value of time
    + * is used to derive the time-bucket of a key.
    + */
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public class ManagedTimeStateImpl extends AbstractManagedStateImpl implements TimeSlicedBucketedState
    +{
    +  public ManagedTimeStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, long time, Slice key, Slice value)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return BucketedState.EXPIRED;
    +    }
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    --- End diff --
    
    buckets[bucketIdx] => bucket


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765314
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/TimeSlicedBucketedState.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A type of bucketed state where a bucket's data is further divided into time buckets. This requires
    + * time per key to figure out which time bucket a particular key belongs to.
    --- End diff --
    
    Add: The key must not be null.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56769401
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,499 @@
    +/**
    + * 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.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.base.Throwables;
    +import com.google.common.collect.Maps;
    +
    +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.Slice;
    +
    +/**
    + * An abstract implementation of managed state.<br/>
    + *
    + * The important sub-components here are:
    + * <ol>
    + *   <li>
    + *     {@link #checkpointManager}: writes incremental checkpoints in window files and transfers data from window
    + *     files to bucket files.
    + *   </li>
    + *   <li>
    + *     {@link #bucketsFileSystem}: a bucket on disk is sub-divided into time-buckets. This manages meta-bucket
    + *     information (list of {@link BucketsFileSystem.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}: plug-able 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.
    --- End diff --
    
    This comment seems to belong in ManagedStateImpl


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56779057
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    --- End diff --
    
    Modification to a particular row is just being done by one thread which is the thread that transfers data from a window file to data files. I will double check and add a comment 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56767049
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -0,0 +1,125 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.api.annotation.OperatorAnnotation;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.lib.state.TimeSlicedBucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * This implementation of {@link ManagedState} lets the client to specify the time for each key. The value of time
    + * is used to derive the time-bucket of a key.
    + */
    +@OperatorAnnotation(checkpointableWithinAppWindow = false)
    +public class ManagedTimeStateImpl extends AbstractManagedStateImpl implements TimeSlicedBucketedState
    +{
    +  public ManagedTimeStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, long time, Slice key, Slice value)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket != -1) {
    +      buckets[bucketIdx].put(key, timeBucket, value);
    +    }
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    return buckets[bucketIdx].get(key, -1, Bucket.ReadSource.ALL);
    +  }
    +
    +  @Override
    +  public Slice getSync(long bucketId, long time, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so no point in looking further.
    +      return BucketedState.EXPIRED;
    +    }
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long bucketId, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    --- End diff --
    
    buckets[bucketIdx] => bucket


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57086936
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeStateImpl.java ---
    @@ -51,63 +51,57 @@ public void put(long bucketId, long time, Slice key, Slice value)
         }
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Slice getSync(long bucketId, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
    -
    -    synchronized (bucket) {
    -      return bucket.get(key, -1, Bucket.ReadSource.ALL);
    -    }
    +    return getValueFromBucketSync(bucketId, -1, key);
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Slice getSync(long bucketId, long time, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
         long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
         if (timeBucket == -1) {
           //time is expired so no point in looking further.
           return BucketedState.EXPIRED;
         }
    +    return getValueFromBucketSync(bucketId, timeBucket, key);
    +  }
     
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  private Slice getValueFromBucketSync(long bucketId, long timeBucket, Slice key)
    +  {
    +    int bucketIdx = prepareBucket(bucketId);
         Bucket bucket = buckets[bucketIdx];
         synchronized (bucket) {
           return bucket.get(key, timeBucket, Bucket.ReadSource.ALL);
         }
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Future<Slice> getAsync(long bucketId, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
    -    synchronized (bucket) {
    -      Slice cachedVal = buckets[bucketIdx].get(key, -1, Bucket.ReadSource.MEMORY);
    -      if (cachedVal != null) {
    -        return Futures.immediateFuture(cachedVal);
    -      }
    -      return readerService.submit(new KeyFetchTask(bucket, key, -1, throwable));
    -    }
    +    return getValueFromBucketAsync(bucketId, -1, key);
       }
     
    -  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
       @Override
       public Future<Slice> getAsync(long bucketId, long time, Slice key)
       {
    -    int bucketIdx = prepareBucket(bucketId);
    -    Bucket bucket = buckets[bucketIdx];
         long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
         if (timeBucket == -1) {
           //time is expired so no point in looking further.
           return Futures.immediateFuture(BucketedState.EXPIRED);
         }
    +    return getValueFromBucketAsync(bucketId, timeBucket, key);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  private Future<Slice> getValueFromBucketAsync(long bucketId, long timeBucket, Slice key)
    --- End diff --
    
    If these new functions getValueFromBucketAsync() and getValueFromBucketSync() are moved to the base class and made static, could they also be used in the other 2 implementation classes ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57241107
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,499 @@
    +/**
    + * 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.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.base.Throwables;
    +import com.google.common.collect.Maps;
    +
    +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.Slice;
    +
    +/**
    + * An abstract implementation of managed state.<br/>
    + *
    + * The important sub-components here are:
    + * <ol>
    + *   <li>
    + *     {@link #checkpointManager}: writes incremental checkpoints in window files and transfers data from window
    + *     files to bucket files.
    + *   </li>
    + *   <li>
    + *     {@link #bucketsFileSystem}: a bucket on disk is sub-divided into time-buckets. This manages meta-bucket
    + *     information (list of {@link BucketsFileSystem.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}: plug-able 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.CheckpointNotificationListener, ManagedStateContext,
    +    TimeBucketAssigner.PurgeListener
    +{
    +  private long maxMemorySize;
    +
    +  protected int numBuckets;
    +
    +  @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
    +  protected IncrementalCheckpointManager checkpointManager = new IncrementalCheckpointManager();
    +
    +  @NotNull
    +  protected BucketsFileSystem bucketsFileSystem = new BucketsFileSystem();
    +
    +  protected transient OperatorContext operatorContext;
    +  protected transient long windowId;
    +
    +  @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();
    +
    +  //accessible to StateTracker
    +  final transient Object commitLock = new Object();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorContext = context;
    +    fileAccess.init();
    +
    +    timeBucketAssigner.setPurgeListener(this);
    +
    +    //setup all the managed state components
    +    timeBucketAssigner.setup(this);
    +    checkpointManager.setup(this);
    +    bucketsFileSystem.setup(this);
    +
    +    if (buckets == null) {
    +      //create buckets array only once at start when it is not created.
    +      numBuckets = getNumBuckets();
    +      buckets = new Bucket[numBuckets];
    +    }
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    --- End diff --
    
    Yes it is possible. Let's say user specified numBuckets = 4. This will create ```buckets``` array of size 4. However if we don't see a key for bucket 3 then ```buckets[2]``` will never be initialized to a a bucket. We are following a lazy initialization strategy.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57247940
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    --- End diff --
    
    I think having the table in javadoc is better since it will be visible on our API reference page.
    Here is one example (though not a real fancy table):
    https://docs.oracle.com/javase/7/docs/jre/api/net/httpserver/spec/com/sun/net/httpserver/HttpServer.html
    
    You can put it in one implementation class and just have links in the other 2.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56776122
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    +    Preconditions.checkNotNull(timeBuckets, "timeBuckets");
    +    String tmpFileName = getTmpFileName();
    +
    +    try (DataOutputStream dos = getOutputStream(bucketId, tmpFileName)) {
    +      dos.writeInt(timeBuckets.size());
    +      for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBuckets.entrySet()) {
    +        MutableTimeBucketMeta tbm = entry.getValue();
    +        dos.writeLong(tbm.getTimeBucketId());
    +        dos.writeLong(tbm.getSizeInBytes());
    +        dos.writeLong(tbm.getLastTransferredWindowId());
    +        dos.writeInt(tbm.getFirstKey().length);
    +        dos.write(tbm.getFirstKey().toByteArray());
    +      }
    +
    +    }
    +    rename(bucketId, tmpFileName, META_FILE_NAME);
    +  }
    +
    +  protected void deleteTimeBucketsLessThanEqualTo(long latestExpiredTimeBucket) throws IOException
    +  {
    +    LOG.debug("delete files before {}", latestExpiredTimeBucket);
    +
    +    for (long bucketName : bucketNamesOnFS) {
    +      RemoteIterator<LocatedFileStatus> timeBucketsIterator = listFiles(bucketName);
    +      boolean emptyBucket = true;
    +      while (timeBucketsIterator.hasNext()) {
    +        LocatedFileStatus timeBucketStatus = timeBucketsIterator.next();
    +
    +        String timeBucketStr = timeBucketStatus.getPath().getName();
    +        if (timeBucketStr.equals(BucketsFileSystem.META_FILE_NAME) || timeBucketStr.endsWith(".tmp")) {
    +          //ignoring meta and tmp files
    +          continue;
    +        }
    +        long timeBucket = Long.parseLong(timeBucketStr);
    +
    +        if (timeBucket <= latestExpiredTimeBucket) {
    +          LOG.debug("deleting bucket {} time-bucket {}", timeBucket);
    +          delete(bucketName, timeBucketStatus.getPath().getName());
    +
    +          invalidateTimeBucket(bucketName, timeBucket);
    +        } else {
    +          emptyBucket = false;
    +        }
    +      }
    +      if (emptyBucket) {
    +        LOG.debug("deleting bucket {}", bucketName);
    +        deleteBucket(bucketName);
    +      }
    +    }
    +  }
    +
    +  void invalidateTimeBucket(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      timeBucketsMeta.remove(bucketId, timeBucketId);
    +    }
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +  }
    +
    +  private static class TimeBucketMeta implements Comparable<TimeBucketMeta>
    +  {
    +    protected final long bucketId;
    +    protected final long timeBucketId;
    +    protected long lastTransferredWindowId = -1;
    +    protected long sizeInBytes;
    +    protected Slice firstKey;
    +
    +    private TimeBucketMeta()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      timeBucketId = -1;
    +    }
    +
    +    private TimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      this.bucketId = bucketId;
    +      this.timeBucketId = timeBucketId;
    +    }
    +
    +    public long getLastTransferredWindowId()
    +    {
    +      return lastTransferredWindowId;
    +    }
    +
    +    public long getSizeInBytes()
    +    {
    +      return this.sizeInBytes;
    +    }
    +
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    public long getTimeBucketId()
    +    {
    +      return timeBucketId;
    +    }
    +
    +    public Slice getFirstKey()
    +    {
    +      return firstKey;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof TimeBucketMeta)) {
    +        return false;
    +      }
    +
    +      TimeBucketMeta that = (TimeBucketMeta)o;
    +
    +      return bucketId == that.bucketId && timeBucketId == that.timeBucketId;
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(bucketId, timeBucketId);
    +    }
    +
    +    @Override
    +    public int compareTo(@NotNull TimeBucketMeta o)
    +    {
    +      if (bucketId < o.bucketId) {
    +        return -1;
    +      }
    +      if (bucketId > o.bucketId) {
    +        return 1;
    +      }
    +      if (timeBucketId < o.timeBucketId) {
    +        return -1;
    +      }
    +      if (timeBucketId > o.timeBucketId) {
    +        return 1;
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /**
    +   * Represents time bucket meta information which can be changed.
    +   * The updates to an instance and read/creation of {@link #immutableTimeBucketMeta} belonging to it are synchronized
    +   * as different threads are updating and reading from it.<br/>
    +   *
    +   * The instance is updated when data from window files are transferred to bucket files and
    +   * {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket} reads the immutable time bucket meta.
    +   */
    +  static class MutableTimeBucketMeta extends TimeBucketMeta
    +  {
    +    private transient ImmutableTimeBucketMeta immutableTimeBucketMeta;
    +
    +    private volatile boolean changed;
    +
    +    MutableTimeBucketMeta()
    +    {
    +      //for kryo
    +    }
    +
    +    public MutableTimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      super(bucketId, timeBucketId);
    +    }
    +
    +    synchronized void updateTimeBucketMeta(long lastTransferredWindow, long bytes, @NotNull Slice firstKey)
    +    {
    +      changed = true;
    +      this.lastTransferredWindowId = lastTransferredWindow;
    +      this.sizeInBytes = bytes;
    +      this.firstKey = Preconditions.checkNotNull(firstKey, "first key");
    +    }
    +
    +    synchronized ImmutableTimeBucketMeta getImmutableTimeBucketMeta()
    +    {
    +      if (immutableTimeBucketMeta == null || changed) {
    +        immutableTimeBucketMeta = new ImmutableTimeBucketMeta(getBucketId(), getTimeBucketId(),
    +            getLastTransferredWindowId(), getSizeInBytes(), getFirstKey());
    +        changed = false;
    +      }
    +      return immutableTimeBucketMeta;
    +    }
    +
    +  }
    +
    +  /**
    +   * This serves the readers - {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket}.
    +   * It is accessible outside the package unlike {@link MutableTimeBucketMeta} that can be only modified by
    +   * BucketPersistor.
    +   */
    +  public static final class ImmutableTimeBucketMeta extends TimeBucketMeta
    --- End diff --
    
    Checking if this could be removed. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57248568
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,60 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in a bucket identified by bucketId. Fetching a key can be expensive if the key
    +   * is not in memory and is present on disk. This fetches the key synchronously. <br/>
    +   * {@link #getAsync(long, Slice)} is recommended for efficiently reading the value of a key.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @return        value
    +   */
    +  Slice getSync(long bucketId, Slice key);
    +
    +  /**
    +   * Returns the future using which the value is obtained.<br/>
    +   * If the key is present in memory, then the future has its value set when constructed.
    --- End diff --
    
    A bucket includes all the  key/values -  both in-memory and on-disk belonging to that bucket.  So if I say key is present in the bucket that could mean that the key could be in bucket cache or in the data files belonging to the bucket. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53716480
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    +      }
    +    }
    +  };
    +
    +  private final transient Object lock = new Object();
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    if (!initialized) {
    +      if (bucketSpan == null) {
    +        bucketSpan = Duration.millis(context.getValue(Context.OperatorContext.APPLICATION_WINDOW_COUNT) *
    +            context.getValue(Context.DAGContext.STREAMING_WINDOW_SIZE_MILLIS));
    +      }
    +      Calendar calendar = Calendar.getInstance();
    +      long now = calendar.getTimeInMillis();
    --- End diff --
    
    Done


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56729305
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,236 @@
    +/**
    + * 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 javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +import org.joda.time.Instant;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.base.Preconditions;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    --- End diff --
    
    Are the values of `expireBefore` and `bucketSpan` intended to be fixed after initialization or are they modifiable ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53726381
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,505 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends Component<Context.OperatorContext>
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    private final transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsMetaDataManager.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      managedStateContext = null;
    +    }
    +
    +    protected DefaultBucket(long bucketId, @NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.bucketId = bucketId;
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext);
    +    }
    +
    +    @Override
    +    public void setup(Context.OperatorContext context)
    +    {
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traversing the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    +        }
    +        return valSlice;
    +      } else {
    +        //search all the time buckets
    +        try {
    +          if (cachedBucketMetas == null) {
    +            cachedBucketMetas = managedStateContext.getBucketsMetaDataManager().getAllTimeBuckets(bucketId);
    +          }
    +
    +          for (BucketsMetaDataManager.ImmutableTimeBucketMeta immutableTimeBucketMeta : cachedBucketMetas) {
    +
    +            if (managedStateContext.getKeyComparator().compare(key, immutableTimeBucketMeta.getFirstKey()) >= 0) {
    +              //keys in the time bucket files are sorted so if the first key in the file is greater than the key being
    +              //searched, the key will not be present in that file.
    +              Slice valSlice = getKeyFromTimeBucketReader(key, immutableTimeBucketMeta.getTimeBucketId());
    +              if (valSlice != null) {
    +                BucketedValue bucketedValue = new BucketedValue(immutableTimeBucketMeta.getTimeBucketId(), valSlice);
    +                readCache.put(key, bucketedValue);
    +                return valSlice;
    +              }
    +            }
    +          }
    +
    +        } catch (IOException e) {
    +          throw new RuntimeException("get time-buckets " + bucketId, e);
    +        }
    +      }
    +      return null;
    +    }
    +
    +    @Override
    +    public Slice get(Slice key, long timeBucket, ReadSource readSource)
    +    {
    +      switch (readSource) {
    +        case MEMORY:
    +          return getFromMemory(key);
    +        case READERS:
    +          return getFromReaders(key, timeBucket);
    +        case ALL:
    +        default:
    +          Slice value = getFromMemory(key);
    +          if (value != null) {
    +            return value;
    +          }
    +          return getFromReaders(key, timeBucket);
    +      }
    +    }
    +
    +    /**
    +     * Returns the value for the key from a time-bucket reader
    +     * @param key        key
    +     * @param timeBucket time bucket
    +     * @return value if key is found in the time bucket; false otherwise
    +     */
    +    private Slice getKeyFromTimeBucketReader(Slice key, long timeBucket)
    +    {
    +      FileAccess.FileReader fileReader = readers.get(timeBucket);
    +      if (fileReader != null) {
    +        return readKey(fileReader, key, timeBucket);
    +      }
    +      //file reader is not loaded and is null
    +      try {
    +        if (loadFileReader(timeBucket)) {
    +          return readKey(readers.get(timeBucket), key, timeBucket);
    +        }
    +        return null;
    +      } catch (IOException e) {
    +        throw new RuntimeException("while loading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private Slice readKey(FileAccess.FileReader fileReader, Slice key, long timeBucket)
    +    {
    +      Slice valSlice = new Slice(null, 0, 0);
    +      try {
    +        if (fileReader.seek(key)) {
    +          fileReader.next(dummyGetKey, valSlice);
    +          return valSlice;
    +        } else {
    +          return null;
    +        }
    +      } catch (IOException e) {
    +        throw new RuntimeException("reading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private boolean loadFileReader(long timeBucketId) throws IOException
    +    {
    +      BucketsMetaDataManager.ImmutableTimeBucketMeta tbm = managedStateContext.getBucketsMetaDataManager()
    +          .getTimeBucketMeta(bucketId, timeBucketId);
    +
    +      if (tbm != null) {
    +        FileAccess.FileReader reader = managedStateContext.getFileAccess().getReader(bucketId,
    +            BucketsDataManager.getFileName(timeBucketId));
    +        readers.put(timeBucketId, reader);
    +        sizeInBytes.getAndAdd(tbm.getSizeInBytes());
    +        return true;
    +      }
    +      return false;
    +    }
    +
    +    @Override
    +    public void put(Slice key, long timeBucket, Slice value)
    +    {
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue == null) {
    +        bucketedValue = new BucketedValue();
    +        flash.put(key, bucketedValue);
    +        sizeInBytes.getAndAdd(key.length);
    +        sizeInBytes.getAndAdd(Long.SIZE);
    +      }
    +      if (timeBucket > bucketedValue.getTimeBucket()) {
    +
    +        if (bucketedValue.getValue() != null) {
    +          //overriding the value so need to subtract the size of previous value
    +          sizeInBytes.getAndAdd(-bucketedValue.getValue().length);
    +        }
    +
    +        sizeInBytes.getAndAdd(value.length);
    --- End diff --
    
    Ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: [For Review Only] MLHR-1897 #c...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r48513062
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,487 @@
    +/**
    + * 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.lang.reflect.Array;
    +import java.util.Comparator;
    +import java.util.Map;
    +import java.util.Timer;
    +import java.util.TimerTask;
    +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.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.MinMaxPriorityQueue;
    +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;
    +
    +/**
    + * Not thread safe.
    + */
    +public abstract class AbstractManagedStateImpl
    +    implements ManagedState, Operator, Operator.CheckpointListener, ManagedStateContext
    +{
    +  private long maxCacheSize;
    +
    +  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;
    +
    +  private StateTracker tracker;
    +
    +  @Min(1)
    +  private int numReaders = 10;
    +  @NotNull
    +  protected transient ExecutorService readerService;
    +
    +  @NotNull
    +  private final 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);
    +
    +  private transient StateTracker stateTracker;
    +
    +  private final transient Object commitLock = new Object();
    +
    +  @SuppressWarnings("unchecked")
    --- End diff --
    
    Can the suppress be localized?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57250223
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,60 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in a bucket identified by bucketId. Fetching a key can be expensive if the key
    +   * is not in memory and is present on disk. This fetches the key synchronously. <br/>
    +   * {@link #getAsync(long, Slice)} is recommended for efficiently reading the value of a key.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @return        value
    +   */
    +  Slice getSync(long bucketId, Slice key);
    +
    +  /**
    +   * Returns the future using which the value is obtained.<br/>
    +   * If the key is present in memory, then the future has its value set when constructed.
    --- End diff --
    
    Yeah should be placed in the implementation file. Will move it there.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r54145912
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/BucketedState.java ---
    @@ -0,0 +1,101 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A state where keys are grouped in buckets.
    + */
    +public interface BucketedState
    +{
    +  /**
    +   * Sets the value of the key in bucket identified by bucketId.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, Slice key, Slice value);
    --- End diff --
    
    Is bucketId an artifact of the implementation? The key is fine as it has a manifestation for the user within the application. This way of specification puts an implicit 2 level hierarchy. Questions arise why not 3 levels?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56786971
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,580 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public TimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<TimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<TimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<TimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    +    Preconditions.checkNotNull(timeBuckets, "timeBuckets");
    +    String tmpFileName = getTmpFileName();
    +
    +    try (DataOutputStream dos = getOutputStream(bucketId, tmpFileName)) {
    +      dos.writeInt(timeBuckets.size());
    +      for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBuckets.entrySet()) {
    +        MutableTimeBucketMeta tbm = entry.getValue();
    +        dos.writeLong(tbm.getTimeBucketId());
    +        dos.writeLong(tbm.getSizeInBytes());
    +        dos.writeLong(tbm.getLastTransferredWindowId());
    +        dos.writeInt(tbm.getFirstKey().length);
    +        dos.write(tbm.getFirstKey().toByteArray());
    +      }
    +
    +    }
    +    rename(bucketId, tmpFileName, META_FILE_NAME);
    +  }
    +
    +  protected void deleteTimeBucketsLessThanEqualTo(long latestExpiredTimeBucket) throws IOException
    +  {
    +    LOG.debug("delete files before {}", latestExpiredTimeBucket);
    +
    +    for (long bucketName : bucketNamesOnFS) {
    +      RemoteIterator<LocatedFileStatus> timeBucketsIterator = listFiles(bucketName);
    +      boolean emptyBucket = true;
    +      while (timeBucketsIterator.hasNext()) {
    +        LocatedFileStatus timeBucketStatus = timeBucketsIterator.next();
    +
    +        String timeBucketStr = timeBucketStatus.getPath().getName();
    +        if (timeBucketStr.equals(BucketsFileSystem.META_FILE_NAME) || timeBucketStr.endsWith(".tmp")) {
    +          //ignoring meta and tmp files
    +          continue;
    +        }
    +        long timeBucket = Long.parseLong(timeBucketStr);
    +
    +        if (timeBucket <= latestExpiredTimeBucket) {
    +          LOG.debug("deleting bucket {} time-bucket {}", timeBucket);
    +          delete(bucketName, timeBucketStatus.getPath().getName());
    +
    --- End diff --
    
    The time bucket data file is deleted then the meta file is updated. If the operator fails after deleting the time bucket but before the meta file is updated, the operator could load the meta data for a deleted time bucket on recovery. Could that cause a problem? Could the update of the Meta file be done before the delete of the time bucket?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56775624
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    +
    +    //Data serialized/deserialized from bucket data files
    +    private final transient Map<Slice, BucketedValue> readCache = Maps.newConcurrentMap();
    +
    +    //TimeBucket -> FileReaders
    +    private final transient Map<Long, FileAccess.FileReader> readers = Maps.newTreeMap();
    +
    +    protected transient ManagedStateContext managedStateContext;
    +
    +    private AtomicLong sizeInBytes = new AtomicLong(0);
    +
    +    private final transient Slice dummyGetKey = new Slice(null, 0, 0);
    +
    +    private transient TreeSet<BucketsFileSystem.ImmutableTimeBucketMeta> cachedBucketMetas;
    +
    +    private DefaultBucket()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +    }
    +
    +    protected DefaultBucket(long bucketId)
    +    {
    +      this.bucketId = bucketId;
    +    }
    +
    +    @Override
    +    public void setup(@NotNull ManagedStateContext managedStateContext)
    +    {
    +      this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +    }
    +
    +    @Override
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    @Override
    +    public long getSizeInBytes()
    +    {
    +      return sizeInBytes.longValue();
    +    }
    +
    +    private Slice getFromMemory(Slice key)
    +    {
    +      //search the cache for key
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      for (Long window : checkpointedData.descendingKeySet()) {
    +        //traverse the checkpointed data in reverse order
    +        bucketedValue = checkpointedData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      for (Long window : recoveredData.descendingKeySet()) {
    +        //traverse the reccovered data in reverse order
    +        bucketedValue = recoveredData.get(window).get(key);
    +        if (bucketedValue != null) {
    +          return bucketedValue.getValue();
    +        }
    +      }
    +
    +      bucketedValue = committedData.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +
    +      bucketedValue = readCache.get(key);
    +      if (bucketedValue != null) {
    +        return bucketedValue.getValue();
    +      }
    +      return null;
    +    }
    +
    +    private Slice getFromReaders(Slice key, long timeBucket)
    +    {
    +      if (timeBucket != -1) {
    +        Slice valSlice = getKeyFromTimeBucketReader(key, timeBucket);
    +        if (valSlice != null) {
    +          BucketedValue bucketedValue = new BucketedValue(timeBucket, valSlice);
    +          readCache.put(key, bucketedValue);
    +        }
    +        return valSlice;
    +      } else {
    +        //search all the time buckets
    +        try {
    +          if (cachedBucketMetas == null) {
    +            cachedBucketMetas = managedStateContext.getBucketsFileSystem().getAllTimeBuckets(bucketId);
    +          }
    +
    +          for (BucketsFileSystem.ImmutableTimeBucketMeta immutableTimeBucketMeta : cachedBucketMetas) {
    +
    +            if (managedStateContext.getKeyComparator().compare(key, immutableTimeBucketMeta.getFirstKey()) >= 0) {
    +              //keys in the time bucket files are sorted so if the first key in the file is greater than the key being
    +              //searched, the key will not be present in that file.
    +              Slice valSlice = getKeyFromTimeBucketReader(key, immutableTimeBucketMeta.getTimeBucketId());
    +              if (valSlice != null) {
    +                BucketedValue bucketedValue = new BucketedValue(immutableTimeBucketMeta.getTimeBucketId(), valSlice);
    +                readCache.put(key, bucketedValue);
    +                return valSlice;
    +              }
    +            }
    +          }
    +
    +        } catch (IOException e) {
    +          throw new RuntimeException("get time-buckets " + bucketId, e);
    +        }
    +      }
    +      return null;
    +    }
    +
    +    @Override
    +    public Slice get(Slice key, long timeBucket, ReadSource readSource)
    +    {
    +      switch (readSource) {
    +        case MEMORY:
    +          return getFromMemory(key);
    +        case READERS:
    +          return getFromReaders(key, timeBucket);
    +        case ALL:
    +        default:
    +          Slice value = getFromMemory(key);
    +          if (value != null) {
    +            return value;
    +          }
    +          return getFromReaders(key, timeBucket);
    +      }
    +    }
    +
    +    /**
    +     * Returns the value for the key from a time-bucket reader
    +     * @param key        key
    +     * @param timeBucket time bucket
    +     * @return value if key is found in the time bucket; false otherwise
    +     */
    +    private Slice getKeyFromTimeBucketReader(Slice key, long timeBucket)
    +    {
    +      FileAccess.FileReader fileReader = readers.get(timeBucket);
    +      if (fileReader != null) {
    +        return readKey(fileReader, key, timeBucket);
    +      }
    +      //file reader is not loaded and is null
    +      try {
    +        if (loadFileReader(timeBucket)) {
    +          return readKey(readers.get(timeBucket), key, timeBucket);
    +        }
    +        return null;
    +      } catch (IOException e) {
    +        throw new RuntimeException("while loading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private Slice readKey(FileAccess.FileReader fileReader, Slice key, long timeBucket)
    +    {
    +      Slice valSlice = new Slice(null, 0, 0);
    +      try {
    +        if (fileReader.seek(key)) {
    +          fileReader.next(dummyGetKey, valSlice);
    +          return valSlice;
    +        } else {
    +          return null;
    +        }
    +      } catch (IOException e) {
    +        throw new RuntimeException("reading " + bucketId + ", " + timeBucket, e);
    +      }
    +    }
    +
    +    private boolean loadFileReader(long timeBucketId) throws IOException
    +    {
    +      BucketsFileSystem.ImmutableTimeBucketMeta tbm = managedStateContext.getBucketsFileSystem()
    +          .getTimeBucketMeta(bucketId, timeBucketId);
    +
    +      if (tbm != null) {
    +        FileAccess.FileReader reader = managedStateContext.getBucketsFileSystem().getReader(bucketId,
    +            BucketsFileSystem.getFileName(timeBucketId));
    +        readers.put(timeBucketId, reader);
    +        sizeInBytes.getAndAdd(tbm.getSizeInBytes());
    +        return true;
    +      }
    +      return false;
    +    }
    +
    +    @Override
    +    public void put(Slice key, long timeBucket, Slice value)
    +    {
    +      BucketedValue bucketedValue = flash.get(key);
    +      if (bucketedValue == null) {
    +        bucketedValue = new BucketedValue();
    +        flash.put(key, bucketedValue);
    --- End diff --
    
    A put operation can modify flash in one thread while an asynchronous get operation can do a get from flash in another thread. Could this cause a problem?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53704929
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    --- End diff --
    
    From @amberarrow 
    Do we need a set here, i.e. do we expect more than one listener ? With a set, several questions come up:
    1. We should document the fact that the listener should ensure that set membership works correctly by implementing hashCode()/equals().
    2. The purgeTimeBucketsBefore() method should not take a long time since it is run with a lock held in the expiry task.
    3. Do we need an unregister() method to remove a listener ?
    
    All these go away if we limit it to just 1 listener.
    ---------------
     
    Made the 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56781864
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return expired slice.
    +      return BucketedState.EXPIRED;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +
    +    synchronized (bucket) {
    +      return bucket.get(key, timeBucket, Bucket.ReadSource.ALL);
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    Long purgedTimeBucket;
    +
    +    //tear down all the purged time buckets
    +    while (null != (purgedTimeBucket = purgedTimeBuckets.poll())) {
    +      int purgedTimeBucketIdx = getBucketIdx(purgedTimeBucket);
    +      if (buckets[purgedTimeBucketIdx] != null && buckets[purgedTimeBucketIdx].getBucketId() == purgedTimeBucket) {
    +        buckets[purgedTimeBucketIdx].teardown();
    +        buckets[purgedTimeBucketIdx] = null;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected void handleBucketConflict(int bucketIdx, long newBucketId)
    +  {
    +    Preconditions.checkArgument(buckets[bucketIdx].getBucketId() < newBucketId, "new time bucket should have a value"
    +        + " greater than the old time bucket");
    +    //Time buckets are purged periodically so here a bucket conflict is expected and so we just ignore conflicts.
    +    buckets[bucketIdx].teardown();
    +    buckets[bucketIdx] = newBucket(newBucketId);
    +    buckets[bucketIdx].setup(this);
    +  }
    +
    +  @Override
    +  public void purgeTimeBucketsLessThanEqualTo(long timeBucket)
    +  {
    +    purgedTimeBuckets.add(timeBucket);
    +    super.purgeTimeBucketsLessThanEqualTo(timeBucket);
    +  }
    +
    +  /**
    +   * This uses operator id instead of bucket id as the name of parent folder of time-buckets. This is because
    +   * multiple partitions may work on same time-buckets.
    +   */
    +  public static class TimeUnifiedBucketsFileSystem extends BucketsFileSystem
    +  {
    --- End diff --
    
    Missing Override annotations on methods for this class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57247730
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    --- End diff --
    
    How about I create a markdown file in this package with a tabular structure highlighting the differences?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53714476
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    --- End diff --
    
    There will be only one listener. Notification is within the synchronized block because ```startTime``` is modified by the operator thread as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56766289
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return BucketedState.EXPIRED;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    --- End diff --
    
    buckets[bucketIdx] => bucket


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r53714252
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/TimeBucketAssigner.java ---
    @@ -0,0 +1,223 @@
    +/**
    + * 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.util.Calendar;
    +import java.util.Set;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.joda.time.Duration;
    +
    +import com.esotericsoftware.kryo.serializers.FieldSerializer;
    +import com.esotericsoftware.kryo.serializers.JavaSerializer;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.api.Component;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.lib.appdata.query.WindowBoundedService;
    +
    +/**
    + * Keeps track of time buckets.<br/>
    + *
    + * The data of a bucket is further divided into time-buckets. This component controls the length of time buckets,
    + * which time-bucket an event falls into and sliding the time boundaries.
    + * <p/>
    + *
    + * The configuration {@link #expireBefore} and {@link #bucketSpan}  are used to calculate number of time-buckets.
    + * For eg. if <code>expireBefore = 1 hour</code> and <code>bucketSpan = 30 minutes</code>, then <code>
    + *   numBuckets = 60 minutes/ 30 minutes = 2 </code>.
    + * <p/>
    + *
    + * The time boundaries- start and end, periodically move by span of a single time-bucket. Any event with time < start
    + * is expired. These boundaries slide between application window by another thread and not the operator thread.
    + */
    +public class TimeBucketAssigner implements Component<Context.OperatorContext>
    +{
    +  @NotNull
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration expireBefore = Duration.standardDays(2);
    +
    +  @FieldSerializer.Bind(JavaSerializer.class)
    +  private Duration bucketSpan;
    +
    +  private long bucketSpanMillis;
    +
    +  private long fixedStartTime;
    +  private long startTime;
    +  private long endTime;
    +  private int numBuckets;
    +
    +  private boolean initialized;
    +
    +  private transient WindowBoundedService windowBoundedService;
    +
    +  @NotNull
    +  private final transient Set<Listener> listeners = Sets.newHashSet();
    +
    +  private final transient Runnable expiryTask = new Runnable()
    +  {
    +    @Override
    +    public void run()
    +    {
    +      synchronized (lock) {
    +        startTime += bucketSpanMillis;
    +        endTime += bucketSpanMillis;
    +        for (Listener listener : listeners) {
    +          listener.purgeTimeBucketsBefore(startTime);
    +        }
    --- End diff --
    
    From @amberarrow 
    Does this loop need to be inside the synchronized block ?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56758813
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/Bucket.java ---
    @@ -0,0 +1,534 @@
    +/**
    + * 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.Iterator;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import com.google.common.annotations.VisibleForTesting;
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A bucket that groups events.
    + */
    +public interface Bucket extends ManagedStateComponent
    +{
    +  /**
    +   * @return bucket id
    +   */
    +  long getBucketId();
    +
    +  /**
    +   *
    +   * @return size of bucket in memory.
    +   */
    +  long getSizeInBytes();
    +
    +  /**
    +   * Get value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket time bucket of the key if known; -1 otherwise.
    +   * @param source     source to read from
    +   * @return value of the key.
    +   */
    +  Slice get(Slice key, long timeBucket, ReadSource source);
    +
    +  /**
    +   * Set value of a key.
    +   *
    +   * @param key        key.
    +   * @param timeBucket timeBucket of the key.
    +   * @param value      value of the key.
    +   */
    +  void put(Slice key, long timeBucket, Slice value);
    +
    +  /**
    +   * Triggers the bucket to checkpoint. Returns the non checkpointed data so far.
    +   *
    +   * @return non checkpointed data.
    +   */
    +  Map<Slice, BucketedValue> checkpoint(long windowId);
    +
    +  /**
    +   * Triggers the bucket to commit data till provided window id.
    +   *
    +   * @param windowId window id
    +   */
    +  void committed(long windowId);
    +
    +  /**
    +   * Triggers bucket to free memory which is already persisted in bucket data files.
    +   *
    +   * @return amount of memory freed in bytes.
    +   * @throws IOException
    +   */
    +  long freeMemory() throws IOException;
    +
    +  /**
    +   * Allows the bucket to process/cache data which is recovered (from window files) after failure.
    +   *
    +   * @param windowId largest recovery window
    +   * @param recoveredData recovered data
    +   */
    +  void recoveredData(long windowId, Map<Slice, Bucket.BucketedValue> recoveredData);
    +
    +  enum ReadSource
    +  {
    +    MEMORY,      //state in memory in key/value form
    +    READERS,     //these are streams in which the key will be searched and serialized.
    +    ALL          //both the above states.
    +  }
    +
    +  class BucketedValue
    +  {
    +    private long timeBucket;
    +    private Slice value;
    +
    +    protected BucketedValue()
    +    {
    +    }
    +
    +    protected BucketedValue(long timeBucket, Slice value)
    +    {
    +      this.timeBucket = timeBucket;
    +      this.value = value;
    +    }
    +
    +    protected long getTimeBucket()
    +    {
    +      return timeBucket;
    +    }
    +
    +    protected void setTimeBucket(long timeBucket)
    +    {
    +      this.timeBucket = timeBucket;
    +    }
    +
    +    public Slice getValue()
    +    {
    +      return value;
    +    }
    +
    +    public void setValue(Slice value)
    +    {
    +      this.value = value;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof BucketedValue)) {
    +        return false;
    +      }
    +
    +      BucketedValue that = (BucketedValue)o;
    +
    +      return timeBucket == that.timeBucket && value.equals(that.value);
    +
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(timeBucket, value);
    +    }
    +  }
    +
    +  /**
    +   * Default bucket.<br/>
    +   * Not thread-safe.
    +   */
    +  class DefaultBucket implements Bucket
    +  {
    +    private final long bucketId;
    +
    +    //Key -> Ordered values
    +    private Map<Slice, BucketedValue> flash = Maps.newHashMap();
    +
    +    //Data persisted in write ahead logs. window -> bucket
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> checkpointedData = Maps.newTreeMap();
    +
    +    //Data persisted in bucket data files
    +    private final transient Map<Slice, BucketedValue> committedData = Maps.newHashMap();
    +
    +    //Data recovered
    +    private final transient TreeMap<Long, Map<Slice, BucketedValue>> recoveredData = Maps.newTreeMap();
    --- End diff --
    
    Yes. I think it can be. I will make the 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56766102
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    --- End diff --
    
    return null => ignore key/value


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57266129
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -226,6 +232,15 @@ protected int prepareBucket(long bucketId)
         return bucketIdx;
       }
     
    +  protected void putInBucket(long bucketId, long timeBucket, Slice key, Slice value)
    +  {
    +    if (timeBucket != -1) {
    +      //time bucket is invalid data is not stored
    --- End diff --
    
    Suggest moving comment above the if and rephrasing:
    // if time bucket is invalid, data is not stored


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r57243863
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    --- End diff --
    
    How about I create a markdown file in this package with a tabular structure highlighting the differences?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56765323
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/TimeSlicedBucketedState.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A type of bucketed state where a bucket's data is further divided into time buckets. This requires
    + * time per key to figure out which time bucket a particular key belongs to.
    + * <p/>
    + * The time value eases purging of aged key/value pair.
    + */
    +public interface TimeSlicedBucketedState
    +{
    +  /**
    +   * Sets the value of a key in the bucket identified by bucketId. Time is used to derive which time bucket (within
    +   * the main bucket) a key belongs to.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param time    time associated with the key.
    +   * @param key     key
    +   * @param value   value
    --- End diff --
    
    value => value (must not be null)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56776146
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    +    Preconditions.checkNotNull(timeBuckets, "timeBuckets");
    +    String tmpFileName = getTmpFileName();
    +
    +    try (DataOutputStream dos = getOutputStream(bucketId, tmpFileName)) {
    +      dos.writeInt(timeBuckets.size());
    +      for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBuckets.entrySet()) {
    +        MutableTimeBucketMeta tbm = entry.getValue();
    +        dos.writeLong(tbm.getTimeBucketId());
    +        dos.writeLong(tbm.getSizeInBytes());
    +        dos.writeLong(tbm.getLastTransferredWindowId());
    +        dos.writeInt(tbm.getFirstKey().length);
    +        dos.write(tbm.getFirstKey().toByteArray());
    +      }
    +
    +    }
    +    rename(bucketId, tmpFileName, META_FILE_NAME);
    +  }
    +
    +  protected void deleteTimeBucketsLessThanEqualTo(long latestExpiredTimeBucket) throws IOException
    +  {
    +    LOG.debug("delete files before {}", latestExpiredTimeBucket);
    +
    +    for (long bucketName : bucketNamesOnFS) {
    +      RemoteIterator<LocatedFileStatus> timeBucketsIterator = listFiles(bucketName);
    +      boolean emptyBucket = true;
    +      while (timeBucketsIterator.hasNext()) {
    +        LocatedFileStatus timeBucketStatus = timeBucketsIterator.next();
    +
    +        String timeBucketStr = timeBucketStatus.getPath().getName();
    +        if (timeBucketStr.equals(BucketsFileSystem.META_FILE_NAME) || timeBucketStr.endsWith(".tmp")) {
    +          //ignoring meta and tmp files
    +          continue;
    +        }
    +        long timeBucket = Long.parseLong(timeBucketStr);
    +
    +        if (timeBucket <= latestExpiredTimeBucket) {
    +          LOG.debug("deleting bucket {} time-bucket {}", timeBucket);
    +          delete(bucketName, timeBucketStatus.getPath().getName());
    +
    +          invalidateTimeBucket(bucketName, timeBucket);
    +        } else {
    +          emptyBucket = false;
    +        }
    +      }
    +      if (emptyBucket) {
    +        LOG.debug("deleting bucket {}", bucketName);
    +        deleteBucket(bucketName);
    +      }
    +    }
    +  }
    +
    +  void invalidateTimeBucket(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      timeBucketsMeta.remove(bucketId, timeBucketId);
    +    }
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +  }
    +
    +  private static class TimeBucketMeta implements Comparable<TimeBucketMeta>
    +  {
    +    protected final long bucketId;
    +    protected final long timeBucketId;
    +    protected long lastTransferredWindowId = -1;
    +    protected long sizeInBytes;
    +    protected Slice firstKey;
    +
    +    private TimeBucketMeta()
    +    {
    +      //for kryo
    +      bucketId = -1;
    +      timeBucketId = -1;
    +    }
    +
    +    private TimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      this.bucketId = bucketId;
    +      this.timeBucketId = timeBucketId;
    +    }
    +
    +    public long getLastTransferredWindowId()
    +    {
    +      return lastTransferredWindowId;
    +    }
    +
    +    public long getSizeInBytes()
    +    {
    +      return this.sizeInBytes;
    +    }
    +
    +    public long getBucketId()
    +    {
    +      return bucketId;
    +    }
    +
    +    public long getTimeBucketId()
    +    {
    +      return timeBucketId;
    +    }
    +
    +    public Slice getFirstKey()
    +    {
    +      return firstKey;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o)
    +    {
    +      if (this == o) {
    +        return true;
    +      }
    +      if (!(o instanceof TimeBucketMeta)) {
    +        return false;
    +      }
    +
    +      TimeBucketMeta that = (TimeBucketMeta)o;
    +
    +      return bucketId == that.bucketId && timeBucketId == that.timeBucketId;
    +    }
    +
    +    @Override
    +    public int hashCode()
    +    {
    +      return Objects.hash(bucketId, timeBucketId);
    +    }
    +
    +    @Override
    +    public int compareTo(@NotNull TimeBucketMeta o)
    +    {
    +      if (bucketId < o.bucketId) {
    +        return -1;
    +      }
    +      if (bucketId > o.bucketId) {
    +        return 1;
    +      }
    +      if (timeBucketId < o.timeBucketId) {
    +        return -1;
    +      }
    +      if (timeBucketId > o.timeBucketId) {
    +        return 1;
    +      }
    +      return 0;
    +    }
    +  }
    +
    +  /**
    +   * Represents time bucket meta information which can be changed.
    +   * The updates to an instance and read/creation of {@link #immutableTimeBucketMeta} belonging to it are synchronized
    +   * as different threads are updating and reading from it.<br/>
    +   *
    +   * The instance is updated when data from window files are transferred to bucket files and
    +   * {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket} reads the immutable time bucket meta.
    +   */
    +  static class MutableTimeBucketMeta extends TimeBucketMeta
    +  {
    +    private transient ImmutableTimeBucketMeta immutableTimeBucketMeta;
    +
    +    private volatile boolean changed;
    +
    +    MutableTimeBucketMeta()
    +    {
    +      //for kryo
    +    }
    +
    +    public MutableTimeBucketMeta(long bucketId, long timeBucketId)
    +    {
    +      super(bucketId, timeBucketId);
    +    }
    +
    +    synchronized void updateTimeBucketMeta(long lastTransferredWindow, long bytes, @NotNull Slice firstKey)
    +    {
    +      changed = true;
    +      this.lastTransferredWindowId = lastTransferredWindow;
    +      this.sizeInBytes = bytes;
    +      this.firstKey = Preconditions.checkNotNull(firstKey, "first key");
    +    }
    +
    +    synchronized ImmutableTimeBucketMeta getImmutableTimeBucketMeta()
    +    {
    +      if (immutableTimeBucketMeta == null || changed) {
    +        immutableTimeBucketMeta = new ImmutableTimeBucketMeta(getBucketId(), getTimeBucketId(),
    +            getLastTransferredWindowId(), getSizeInBytes(), getFirstKey());
    +        changed = false;
    +      }
    +      return immutableTimeBucketMeta;
    +    }
    +
    +  }
    +
    +  /**
    +   * This serves the readers - {@link com.datatorrent.lib.state.managed.Bucket.DefaultBucket}.
    +   * It is accessible outside the package unlike {@link MutableTimeBucketMeta} that can be only modified by
    +   * BucketPersistor.
    --- End diff --
    
    BucketFileSystem. Left over from refactoring.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56768167
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/AbstractManagedStateImpl.java ---
    @@ -0,0 +1,499 @@
    +/**
    + * 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.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.base.Throwables;
    +import com.google.common.collect.Maps;
    +
    +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.Slice;
    +
    +/**
    + * An abstract implementation of managed state.<br/>
    + *
    + * The important sub-components here are:
    + * <ol>
    + *   <li>
    + *     {@link #checkpointManager}: writes incremental checkpoints in window files and transfers data from window
    + *     files to bucket files.
    + *   </li>
    + *   <li>
    + *     {@link #bucketsFileSystem}: a bucket on disk is sub-divided into time-buckets. This manages meta-bucket
    + *     information (list of {@link BucketsFileSystem.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}: plug-able 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.CheckpointNotificationListener, ManagedStateContext,
    +    TimeBucketAssigner.PurgeListener
    +{
    +  private long maxMemorySize;
    +
    +  protected int numBuckets;
    +
    +  @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
    +  protected IncrementalCheckpointManager checkpointManager = new IncrementalCheckpointManager();
    +
    +  @NotNull
    +  protected BucketsFileSystem bucketsFileSystem = new BucketsFileSystem();
    +
    +  protected transient OperatorContext operatorContext;
    +  protected transient long windowId;
    +
    +  @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();
    +
    +  //accessible to StateTracker
    +  final transient Object commitLock = new Object();
    +
    +  @Override
    +  public void setup(OperatorContext context)
    +  {
    +    operatorContext = context;
    +    fileAccess.init();
    +
    +    timeBucketAssigner.setPurgeListener(this);
    +
    +    //setup all the managed state components
    +    timeBucketAssigner.setup(this);
    +    checkpointManager.setup(this);
    +    bucketsFileSystem.setup(this);
    +
    +    if (buckets == null) {
    +      //create buckets array only once at start when it is not created.
    +      numBuckets = getNumBuckets();
    +      buckets = new Bucket[numBuckets];
    +    }
    +    for (Bucket bucket : buckets) {
    +      if (bucket != null) {
    --- End diff --
    
    Is it possible for bucket to be null here ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56853086
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedStateImpl.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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.util.concurrent.Future;
    +
    +import javax.validation.constraints.Min;
    +
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Basic implementation of {@link ManagedState} where window is used to sub-group key of a particular bucket.<br/>
    + *
    + */
    +public class ManagedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +
    +  public ManagedStateImpl()
    +  {
    +    this.numBuckets = 1;
    +  }
    +
    +  @Override
    +  public void put(long bucketId, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(windowId);
    --- End diff --
    
    I think I need to make have better default configuration for TimeBucketAssigner in this case


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56772431
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/BucketsFileSystem.java ---
    @@ -0,0 +1,573 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.Collections;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Set;
    +import java.util.TreeMap;
    +import java.util.TreeSet;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +import javax.annotation.Nullable;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Ordering;
    +import com.google.common.collect.Sets;
    +import com.google.common.collect.Table;
    +import com.google.common.collect.TreeBasedTable;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * Persists bucket data on disk and maintains meta information about the buckets.
    + * <p/>
    + *
    + * Each bucket has a meta-data file and the format of that is :<br/>
    + * <ol>
    + * <li>total number of time-buckets (int)</li>
    + * <li>For each time bucket
    + * <ol>
    + * <li>time bucket key (long)</li>
    + * <li>size of data (sum of bytes) (long)</li>
    + * <li>last transferred window id (long)</li>
    + * <li>length of the first key in the time-bucket file (int)</li>
    + * <li>first key in the time-bucket file (byte[])</li>
    + * </ol>
    + * </li>
    + * </ol>
    + * <p/>
    + * Meta data information is updated by {@link IncrementalCheckpointManager}. Any updates are restricted to the package.
    + */
    +public class BucketsFileSystem implements ManagedStateComponent
    +{
    +  public static final String META_FILE_NAME = "_META";
    +
    +  private final transient TreeBasedTable<Long, Long, MutableTimeBucketMeta> timeBucketsMeta = TreeBasedTable.create();
    +
    +  //Check-pointed set of all buckets this instance has written to.
    +  protected final Set<Long> bucketNamesOnFS = new ConcurrentSkipListSet<>();
    +
    +  protected transient ManagedStateContext managedStateContext;
    +
    +  @Override
    +  public void setup(@NotNull ManagedStateContext managedStateContext)
    +  {
    +    this.managedStateContext = Preconditions.checkNotNull(managedStateContext, "managed state context");
    +  }
    +
    +  protected FileAccess.FileWriter getWriter(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getWriter(bucketId, fileName);
    +  }
    +
    +  protected FileAccess.FileReader getReader(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getReader(bucketId, fileName);
    +  }
    +
    +  protected void rename(long bucketId, String fromName, String toName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().rename(bucketId, fromName, toName);
    +  }
    +
    +  protected DataOutputStream getOutputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getOutputStream(bucketId, fileName);
    +  }
    +
    +  protected DataInputStream getInputStream(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().getInputStream(bucketId, fileName);
    +  }
    +
    +  protected boolean exists(long bucketId, String fileName) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().exists(bucketId, fileName);
    +  }
    +
    +  protected RemoteIterator<LocatedFileStatus> listFiles(long bucketId) throws IOException
    +  {
    +    return managedStateContext.getFileAccess().listFiles(bucketId);
    +  }
    +
    +  protected void delete(long bucketId, String fileName) throws IOException
    +  {
    +    managedStateContext.getFileAccess().delete(bucketId, fileName);
    +  }
    +
    +  protected void deleteBucket(long bucketId) throws IOException
    +  {
    +    managedStateContext.getFileAccess().deleteBucket(bucketId);
    +  }
    +
    +  /**
    +   * Saves data to a bucket. The data consists of key/values of all time-buckets of a particular bucket.
    +   *
    +   * @param windowId        window id
    +   * @param bucketId        bucket id
    +   * @param data            data of all time-buckets
    +   * @throws IOException
    +   */
    +  protected void writeBucketData(long windowId, long bucketId, Map<Slice, Bucket.BucketedValue> data) throws IOException
    +  {
    +    Table<Long, Slice, Bucket.BucketedValue> timeBucketedKeys = TreeBasedTable.create(Ordering.<Long>natural(),
    +        managedStateContext.getKeyComparator());
    +
    +    for (Map.Entry<Slice, Bucket.BucketedValue> entry : data.entrySet()) {
    +      long timeBucketId = entry.getValue().getTimeBucket();
    +      timeBucketedKeys.put(timeBucketId, entry.getKey(), entry.getValue());
    +    }
    +
    +    for (long timeBucket : timeBucketedKeys.rowKeySet()) {
    +      BucketsFileSystem.MutableTimeBucketMeta tbm = getOrCreateTimeBucketMeta(bucketId, timeBucket);
    +      addBucketName(bucketId);
    +
    +      long dataSize = 0;
    +      Slice firstKey = null;
    +
    +      FileAccess.FileWriter fileWriter;
    +      String tmpFileName = getTmpFileName();
    +      if (tbm.getLastTransferredWindowId() == -1) {
    +        //A new time bucket so we append all the key/values to the new file
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue().getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      } else {
    +        //the time bucket existed so we need to read the file and then re-write it
    +        TreeMap<Slice, Slice> fileData = new TreeMap<>(managedStateContext.getKeyComparator());
    +        FileAccess.FileReader fileReader = getReader(bucketId, getFileName(timeBucket));
    +        fileReader.readFully(fileData);
    +        fileReader.close();
    +
    +        for (Map.Entry<Slice, Bucket.BucketedValue> entry : timeBucketedKeys.row(timeBucket).entrySet()) {
    +          fileData.put(entry.getKey(), entry.getValue().getValue());
    +        }
    +
    +        fileWriter = getWriter(bucketId, tmpFileName);
    +        for (Map.Entry<Slice, Slice> entry : fileData.entrySet()) {
    +          Slice key = entry.getKey();
    +          Slice value = entry.getValue();
    +
    +          dataSize += key.length;
    +          dataSize += value.length;
    +
    +          fileWriter.append(key.toByteArray(), value.toByteArray());
    +          if (firstKey == null) {
    +            firstKey = key;
    +          }
    +        }
    +      }
    +      fileWriter.close();
    +      rename(bucketId, tmpFileName, getFileName(timeBucket));
    +      tbm.updateTimeBucketMeta(windowId, dataSize, firstKey);
    +    }
    +
    +    updateBucketMetaFile(bucketId);
    +  }
    +
    +  /**
    +   * Retrieves the time bucket meta of a particular time-bucket. If the time bucket doesn't exist then a new one
    +   * is created.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return time bucket meta of the time bucket
    +   * @throws IOException
    +   */
    +  @NotNull
    +  MutableTimeBucketMeta getOrCreateTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm == null) {
    +        tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +        timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +      }
    +      return tbm;
    +    }
    +  }
    +
    +  protected void addBucketName(long bucketId)
    +  {
    +    bucketNamesOnFS.add(bucketId);
    +  }
    +
    +  /**
    +   * Returns the time bucket meta of a particular time-bucket which is immutable.
    +   *
    +   * @param bucketId     bucket id
    +   * @param timeBucketId time bucket id
    +   * @return immutable time bucket meta
    +   * @throws IOException
    +   */
    +  @Nullable
    +  public ImmutableTimeBucketMeta getTimeBucketMeta(long bucketId, long timeBucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      MutableTimeBucketMeta tbm = timeBucketMetaHelper(bucketId, timeBucketId);
    +      if (tbm != null) {
    +        return tbm.getImmutableTimeBucketMeta();
    +      }
    +      return null;
    +    }
    +  }
    +
    +  private MutableTimeBucketMeta timeBucketMetaHelper(long bucketId, long timeBucketId) throws IOException
    +  {
    +    MutableTimeBucketMeta tbm = timeBucketsMeta.get(bucketId, timeBucketId);
    +    if (tbm != null) {
    +      return tbm;
    +    }
    +    if (exists(bucketId, META_FILE_NAME)) {
    +      try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +        //Load meta info of all the time buckets of the bucket identified by bucketId.
    +        loadBucketMetaFile(bucketId, dis);
    +      }
    +    } else {
    +      return null;
    +    }
    +    return timeBucketsMeta.get(bucketId, timeBucketId);
    +  }
    +
    +  /**
    +   * Returns the meta information of all the time buckets in the bucket in descending order - latest to oldest.
    +   *
    +   * @param bucketId bucket id
    +   * @return all the time buckets in order - latest to oldest
    +   */
    +  public TreeSet<ImmutableTimeBucketMeta> getAllTimeBuckets(long bucketId) throws IOException
    +  {
    +    synchronized (timeBucketsMeta) {
    +      TreeSet<ImmutableTimeBucketMeta> immutableTimeBucketMetas = Sets.newTreeSet(
    +          Collections.<ImmutableTimeBucketMeta>reverseOrder());
    +
    +      if (timeBucketsMeta.containsRow(bucketId)) {
    +        for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +          immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +        }
    +        return immutableTimeBucketMetas;
    +      }
    +      if (exists(bucketId, META_FILE_NAME)) {
    +        try (DataInputStream dis = getInputStream(bucketId, META_FILE_NAME)) {
    +          //Load meta info of all the time buckets of the bucket identified by bucket id
    +          loadBucketMetaFile(bucketId, dis);
    +          for (Map.Entry<Long, MutableTimeBucketMeta> entry : timeBucketsMeta.row(bucketId).entrySet()) {
    +            immutableTimeBucketMetas.add(entry.getValue().getImmutableTimeBucketMeta());
    +          }
    +          return immutableTimeBucketMetas;
    +        }
    +      }
    +      return immutableTimeBucketMetas;
    +    }
    +  }
    +
    +  /**
    +   * Loads the bucket meta-file.
    +   *
    +   * @param bucketId bucket id
    +   * @param dis      data input stream
    +   * @throws IOException
    +   */
    +  private void loadBucketMetaFile(long bucketId, DataInputStream dis) throws IOException
    +  {
    +    int numberOfEntries = dis.readInt();
    +
    +    for (int i = 0; i < numberOfEntries; i++) {
    +      long timeBucketId = dis.readLong();
    +      long dataSize = dis.readLong();
    +      long lastTransferredWindow = dis.readLong();
    +
    +      MutableTimeBucketMeta tbm = new MutableTimeBucketMeta(bucketId, timeBucketId);
    +
    +      int sizeOfFirstKey = dis.readInt();
    +      byte[] firstKeyBytes = new byte[sizeOfFirstKey];
    +      dis.readFully(firstKeyBytes, 0, firstKeyBytes.length);
    +      tbm.updateTimeBucketMeta(lastTransferredWindow, dataSize, new Slice(firstKeyBytes));
    +
    +      timeBucketsMeta.put(bucketId, timeBucketId, tbm);
    +    }
    +  }
    +
    +  /**
    +   * Saves the updated bucket meta on disk.
    +   *
    +   * @param bucketId bucket id
    +   * @throws IOException
    +   */
    +  void updateBucketMetaFile(long bucketId) throws IOException
    +  {
    +    Map<Long, MutableTimeBucketMeta> timeBuckets;
    +    synchronized (timeBucketsMeta) {
    +      timeBuckets = timeBucketsMeta.row(bucketId);
    +    }
    --- End diff --
    
    synchronized block may have to cover most of this function because the map returned by the row() call is not a separate copy, so if another thread modifies the row while the loop below is iterating, there could be problems. The docs for the row() call say: "Changes to the returned map will update the underlying table, and vice versa."


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56754934
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/TimeSlicedBucketedState.java ---
    @@ -0,0 +1,98 @@
    +/**
    + * 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;
    +
    +import java.util.concurrent.Future;
    +
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * A type of bucketed state where a bucket's data is further divided into time buckets. This requires
    + * time per key to figure out which time bucket a particular key belongs to.
    + * <p/>
    + * The time value eases purging of aged key/value pair.
    + */
    +public interface TimeSlicedBucketedState
    +{
    +  /**
    +   * Sets the value of a key in the bucket identified by bucketId. Time is used to derive which time bucket (within
    +   * the main bucket) a key belongs to.
    +   *
    +   * @param bucketId identifier of the bucket.
    +   * @param time    time associated with the key.
    +   * @param key     key
    +   * @param value   value
    +   */
    +  void put(long bucketId, long time, Slice key, Slice value);
    +
    +  /**
    +   * Returns the value of the key in the bucket identified by bucketId. This will search for the key in all the
    +   * time buckets.</br>
    +   *
    +   * It retrieves the value synchronously that can be expensive.<br/>
    +   * {@link #getAsync(long, Slice)} is recommended for efficient reading the value of a key.
    +   *
    +   *
    +   * @param bucketId identifier of the bucket
    +   * @param key key
    +   * @return value of the key if found; null if the key is not found;
    +   */
    +  Slice getSync(long bucketId, Slice key);
    +
    +
    +  /**
    +   * Returns the value of key in the bucket identified by bucketId. This expects the time value.
    +   * Time is be used to derive the time bucket (within the main bucket) to which the key belonged and only that
    --- End diff --
    
    I think there is a type. "Time is be used to derive the time bucket" should be "Time is used to derive the time bucket"


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56781175
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,243 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return expired slice.
    +      return BucketedState.EXPIRED;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +
    +    synchronized (bucket) {
    +      return bucket.get(key, timeBucket, Bucket.ReadSource.ALL);
    +    }
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return Futures.immediateFuture(BucketedState.EXPIRED);
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    Long purgedTimeBucket;
    +
    +    //tear down all the purged time buckets
    +    while (null != (purgedTimeBucket = purgedTimeBuckets.poll())) {
    +      int purgedTimeBucketIdx = getBucketIdx(purgedTimeBucket);
    +      if (buckets[purgedTimeBucketIdx] != null && buckets[purgedTimeBucketIdx].getBucketId() == purgedTimeBucket) {
    +        buckets[purgedTimeBucketIdx].teardown();
    --- End diff --
    
    If there are asynchronous get calls happening when a bucket is purged they may attempt to retrieve data from a closed reader. I would suggest keeping track of the asynchronous get calls and wait for them to finish before purging a time bucket.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-malhar pull request: APEXMALHAR-1897 added managed ...

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

    https://github.com/apache/incubator-apex-malhar/pull/145#discussion_r56754895
  
    --- Diff: library/src/main/java/com/datatorrent/lib/state/managed/ManagedTimeUnifiedStateImpl.java ---
    @@ -0,0 +1,238 @@
    +/**
    + * 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.DataInputStream;
    +import java.io.DataOutputStream;
    +import java.io.IOException;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.LinkedBlockingQueue;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.LocatedFileStatus;
    +import org.apache.hadoop.fs.RemoteIterator;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.Queues;
    +import com.google.common.util.concurrent.Futures;
    +
    +import com.datatorrent.lib.fileaccess.FileAccess;
    +import com.datatorrent.lib.state.BucketedState;
    +import com.datatorrent.netlet.util.Slice;
    +
    +/**
    + * In this implementation of {@link ManagedState} the buckets in memory are time-buckets.
    + * <p/>
    + *
    + * <b>Difference from {@link ManagedTimeStateImpl}</b>: <br/>
    + * <ol>
    + * <li>The main buckets in {@link ManagedTimeStateImpl} are unique adhoc long ids which the user provides with the
    + * key. In this implementation the main buckets are time buckets. The user provides just the time and the time bucket is
    + * derived from it.
    + * </li>
    + * <br/>
    + *
    + * <li>In regards to the bucket data on disk, in {@link ManagedTimeStateImpl} the buckets are persisted on disk
    + * with each bucket data further grouped into time-buckets: {base_path}/{bucketId}/{time-bucket id}. <br/>
    + * In this implementation operator id is used as bucketId (on disk) and there is just one time-bucket under a
    + * particular operator id:
    + * {base_path}/{operator id}/{time bucket id}.
    + * </li>
    + * <br/>
    + *
    + * <li>In {@link ManagedTimeStateImpl} a bucket belongs to just one partition. Multiple partitions cannot write to
    + * the same bucket. <br/>
    + * In this implementation multiple partitions can be working with the same time-bucket (since time-bucket is derived
    + * from time). This works because on the disk the time-bucket data is segregated under each operator id.
    + * </li>
    + * <br/>
    + *
    + * <li>While {@link ManagedTimeStateImpl} can support dynamic partitioning by pre-allocating buckets this will not
    + * be able to support dynamic partitioning efficiently.
    + * </li>
    +
    + * </ol>
    + */
    +public class ManagedTimeUnifiedStateImpl extends AbstractManagedStateImpl implements BucketedState
    +{
    +  private final transient LinkedBlockingQueue<Long> purgedTimeBuckets = Queues.newLinkedBlockingQueue();
    +
    +  public ManagedTimeUnifiedStateImpl()
    +  {
    +    bucketsFileSystem = new TimeUnifiedBucketsFileSystem();
    +  }
    +
    +  @Override
    +  public int getNumBuckets()
    +  {
    +    return timeBucketAssigner.getNumBuckets();
    +  }
    +
    +  @Override
    +  public void put(long time, Slice key, Slice value)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +
    +    buckets[bucketIdx].put(key, timeBucket, value);
    +
    +  }
    +
    +  @Override
    +  public Slice getSync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return null;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    return buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.ALL);
    +  }
    +
    +  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
    +  @Override
    +  public Future<Slice> getAsync(long time, Slice key)
    +  {
    +    long timeBucket = timeBucketAssigner.getTimeBucketFor(time);
    +    if (timeBucket == -1) {
    +      //time is expired so return null.
    +      return null;
    +    }
    +    int bucketIdx = prepareBucket(timeBucket);
    +    Bucket bucket = buckets[bucketIdx];
    +    synchronized (bucket) {
    +      Slice cachedVal = buckets[bucketIdx].get(key, timeBucket, Bucket.ReadSource.MEMORY);
    +      if (cachedVal != null) {
    +        return Futures.immediateFuture(cachedVal);
    +      }
    +      return readerService.submit(new KeyFetchTask(bucket, key, timeBucket, throwable));
    +    }
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    super.endWindow();
    +    Long purgedTimeBucket;
    +
    +    //tear down all the purged time buckets
    +    while (null != (purgedTimeBucket = purgedTimeBuckets.poll())) {
    +      int purgedTimeBucketIdx = getBucketIdx(purgedTimeBucket);
    +      if (buckets[purgedTimeBucketIdx] != null && buckets[purgedTimeBucketIdx].getBucketId() == purgedTimeBucket) {
    +        buckets[purgedTimeBucketIdx].teardown();
    +        buckets[purgedTimeBucketIdx] = null;
    +      }
    +    }
    +  }
    +
    +  @Override
    +  protected void handleBucketConflict(int bucketIdx, long newBucketId)
    +  {
    +    Preconditions.checkArgument(buckets[bucketIdx].getBucketId() < newBucketId, "new time bucket should have a value"
    +        + " greater than the old time bucket");
    +    //Time buckets are purged periodically so here a bucket conflict is expected and so we just ignore conflicts.
    +    buckets[bucketIdx].teardown();
    +    buckets[bucketIdx] = newBucket(newBucketId);
    +    buckets[bucketIdx].setup(this);
    +  }
    +
    +  @Override
    +  public void purgeTimeBucketsLessThanEqualTo(long timeBucket)
    +  {
    +    purgedTimeBuckets.add(timeBucket);
    +    super.purgeTimeBucketsLessThanEqualTo(timeBucket);
    +  }
    +
    +  /**
    +   * This uses operator id instead of bucket id as the name of parent folder of time-buckets. This is because
    +   * multiple partitions may work on same time-buckets.
    +   */
    +  public static class TimeUnifiedBucketsFileSystem extends BucketsFileSystem
    --- End diff --
    
    I think it can be private. I don't foresee any need of extending this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---