[ 
https://issues.apache.org/jira/browse/STORM-676?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15192042#comment-15192042
 ] 

ASF GitHub Bot commented on STORM-676:
--------------------------------------

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

    https://github.com/apache/storm/pull/1072#discussion_r55930433
  
    --- Diff: 
external/storm-hbase/src/main/java/org/apache/storm/hbase/trident/windowing/HBaseWindowsStore.java
 ---
    @@ -0,0 +1,275 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.trident.windowing;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hbase.client.Delete;
    +import org.apache.hadoop.hbase.client.Get;
    +import org.apache.hadoop.hbase.client.HTable;
    +import org.apache.hadoop.hbase.client.Put;
    +import org.apache.hadoop.hbase.client.Result;
    +import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
    +import org.apache.hadoop.hbase.client.Scan;
    +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
    +import org.apache.storm.trident.windowing.WindowsStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InterruptedIOException;
    +import java.io.UnsupportedEncodingException;
    +import java.nio.ByteBuffer;
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Queue;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +
    +/**
    + * This class stores entries into hbase instance of the given 
configuration.
    + *
    + */
    +public class HBaseWindowsStore implements WindowsStore {
    +    private static final Logger log = 
LoggerFactory.getLogger(HBaseWindowsStore.class);
    +    public static final String UTF_8 = "utf-8";
    +
    +    private final ThreadLocal<HTable> threadLocalHtable;
    +    private Queue<HTable> htables = new ConcurrentLinkedQueue<>();
    +    private final byte[] family;
    +    private final byte[] qualifier;
    +
    +    public HBaseWindowsStore(final Configuration config, final String 
tableName, byte[] family, byte[] qualifier) {
    --- End diff --
    
    this looks like its better have its own config method and take a Map<?,?> 
config object to configure it.


> Storm Trident support for sliding/tumbling windows
> --------------------------------------------------
>
>                 Key: STORM-676
>                 URL: https://issues.apache.org/jira/browse/STORM-676
>             Project: Apache Storm
>          Issue Type: Improvement
>          Components: storm-core
>            Reporter: Sriharsha Chintalapani
>            Assignee: Satish Duggana
>             Fix For: 1.0.0, 2.0.0
>
>         Attachments: StormTrident_windowing_support-676.pdf
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to