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

    https://github.com/apache/storm/pull/1970#discussion_r125626240
  
    --- Diff: 
external/storm-hbase/src/main/java/org/apache/storm/hbase/state/HBaseKeyValueState.java
 ---
    @@ -0,0 +1,419 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.storm.hbase.state;
    +
    +import com.google.common.collect.Maps;
    +import com.google.common.primitives.UnsignedBytes;
    +import org.apache.hadoop.hbase.client.Delete;
    +import org.apache.hadoop.hbase.client.Durability;
    +import org.apache.hadoop.hbase.client.Get;
    +import org.apache.hadoop.hbase.client.Mutation;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.storm.hbase.bolt.mapper.HBaseProjectionCriteria;
    +import org.apache.storm.hbase.common.ColumnList;
    +import org.apache.storm.hbase.common.HBaseClient;
    +import org.apache.storm.state.DefaultStateEncoder;
    +import org.apache.storm.state.DefaultStateSerializer;
    +import org.apache.storm.state.KeyValueState;
    +import org.apache.storm.state.Serializer;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.NavigableMap;
    +import java.util.TreeMap;
    +import java.util.concurrent.ConcurrentNavigableMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +
    +/**
    + * A Hbase based implementation that persists the state in HBase.
    + */
    +public class HBaseKeyValueState<K, V> implements KeyValueState<K, V> {
    +    private static final Logger LOG = 
LoggerFactory.getLogger(HBaseKeyValueState.class);
    +
    +    public static byte[] STATE_QUALIFIER = "s".getBytes();
    +
    +    public static final NavigableMap<byte[], byte[]> 
EMPTY_PENDING_COMMIT_MAP = Maps.unmodifiableNavigableMap(
    +            new TreeMap<byte[], 
byte[]>(UnsignedBytes.lexicographicalComparator()));
    +
    +    private static byte[] COMMIT_TXID_KEY = "commit".getBytes();
    +    private static byte[] PREPARE_TXID_KEY = "prepare".getBytes();
    +
    +    private final byte[] keyNamespace;
    +    private final byte[] prepareNamespace;
    +    private final byte[] txidNamespace;
    +    private final String namespace;
    +    private final byte[] columnFamily;
    +    private final DefaultStateEncoder<K, V> encoder;
    +    private final HBaseClient hBaseClient;
    +
    +    private ConcurrentNavigableMap<byte[], byte[]> pendingPrepare;
    +    private NavigableMap<byte[], byte[]> pendingCommit;
    +
    +    // the key and value of txIds are guaranteed to be converted to UTF-8 
encoded String
    +    private NavigableMap<byte[], byte[]> txIds;
    +
    +    public HBaseKeyValueState(HBaseClient hbaseClient, String 
columnFamily, String namespace) {
    +        this(hbaseClient, columnFamily, namespace, new 
DefaultStateSerializer<K>(),
    +                new DefaultStateSerializer<V>());
    +    }
    +
    +    public HBaseKeyValueState(HBaseClient hBaseClient, String 
columnFamily, String namespace,
    +                              Serializer<K> keySerializer, Serializer<V> 
valueSerializer) {
    +
    +        this.hBaseClient = hBaseClient;
    +        this.columnFamily = columnFamily.getBytes();
    +        this.namespace = namespace;
    +        this.keyNamespace = (namespace + "$key:").getBytes();
    +        this.prepareNamespace = (namespace + "$prepare").getBytes();
    +        this.txidNamespace = (namespace + "$txid").getBytes();
    +        this.encoder = new DefaultStateEncoder<K, V>(keySerializer, 
valueSerializer);
    +        this.pendingPrepare = createPendingPrepareMap();
    +        initTxids();
    +        initPendingCommit();
    +    }
    +
    +    private void initTxids() {
    +        HBaseProjectionCriteria criteria = new HBaseProjectionCriteria();
    +        criteria.addColumnFamily(columnFamily);
    +        Get get = hBaseClient.constructGetRequests(txidNamespace, 
criteria);
    +        try {
    +            Result[] results = 
hBaseClient.batchGet(Collections.singletonList(get));
    +            Result result = results[0];
    +            if (!result.isEmpty()) {
    +                NavigableMap<byte[], byte[]> familyMap = 
result.getFamilyMap(columnFamily);
    +                txIds = new TreeMap<>(familyMap);
    +            } else {
    +                txIds = new 
TreeMap<>(UnsignedBytes.lexicographicalComparator());
    +            }
    +
    +            LOG.debug("initTxids, txIds {}", txIds);
    +        } catch (Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    private void initPendingCommit() {
    +        HBaseProjectionCriteria criteria = new HBaseProjectionCriteria();
    +        criteria.addColumnFamily(columnFamily);
    +        Get get = hBaseClient.constructGetRequests(prepareNamespace, 
criteria);
    +        try {
    +            Result[] results = 
hBaseClient.batchGet(Collections.singletonList(get));
    +            Result result = results[0];
    +            if (!result.isEmpty()) {
    +                LOG.debug("Loading previously prepared commit from {}", 
prepareNamespace);
    +                NavigableMap<byte[], byte[]> familyMap = 
result.getFamilyMap(columnFamily);
    +                pendingCommit = Maps.unmodifiableNavigableMap(familyMap);
    +            } else {
    +                LOG.debug("No previously prepared commits.");
    +                pendingCommit = EMPTY_PENDING_COMMIT_MAP;
    +            }
    +        } catch (Exception e) {
    +            throw new RuntimeException(e);
    +        }
    +    }
    +
    +    @Override
    +    public void put(K key, V value) {
    +        LOG.debug("put key '{}', value '{}'", key, value);
    +        byte[] columnKey = encoder.encodeKey(key);
    +        byte[] columnValue = encoder.encodeValue(value);
    +        pendingPrepare.put(columnKey, columnValue);
    +    }
    +
    +    @Override
    +    public V get(K key) {
    +        LOG.debug("get key '{}'", key);
    +        byte[] columnKey = encoder.encodeKey(key);
    +        byte[] columnValue = null;
    +
    +        if (pendingPrepare.containsKey(columnKey)) {
    +            columnValue = pendingPrepare.get(columnKey);
    +        } else if (pendingCommit.containsKey(columnKey)) {
    +            columnValue = pendingCommit.get(columnKey);
    +        } else {
    +            HBaseProjectionCriteria criteria = new 
HBaseProjectionCriteria();
    +            HBaseProjectionCriteria.ColumnMetaData column = new 
HBaseProjectionCriteria.ColumnMetaData(columnFamily,
    +                    STATE_QUALIFIER);
    +            criteria.addColumn(column);
    +            Get get = 
hBaseClient.constructGetRequests(getRowKeyForStateKey(columnKey), criteria);
    +            try {
    +                Result[] results = 
hBaseClient.batchGet(Collections.singletonList(get));
    +                Result result = results[0];
    +                columnValue = result.getValue(column.getColumnFamily(), 
column.getQualifier());
    +            } catch (Exception e) {
    +                throw new RuntimeException(e);
    +            }
    +        }
    +        V value = null;
    +        if (columnValue != null) {
    +            value = encoder.decodeValue(columnValue);
    +        }
    +        LOG.debug("Value for key '{}' is '{}'", key, value);
    +        return value;
    +    }
    +
    +    @Override
    +    public V get(K key, V defaultValue) {
    +        V val = get(key);
    +        return val != null ? val : defaultValue;
    +    }
    +
    +    @Override
    +    public V delete(K key) {
    +        LOG.debug("delete key '{}'", key);
    +        byte[] columnKey = encoder.encodeKey(key);
    +        V curr = get(key);
    +        pendingPrepare.put(columnKey, encoder.getTombstoneValue());
    +        return curr;
    +    }
    +
    +    @Override
    +    public Iterator<Map.Entry<K, V>> iterator() {
    +        throw new UnsupportedOperationException("Not supported yet.");
    --- End diff --
    
    Ah I missed to replace the line with HBaseKeyValueStateIterator. 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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to