[jira] [Created] (HBASE-16624) MVCC DeSerialization bug in the upstream HFileScannerImpl

2016-09-12 Thread deepankar (JIRA)
deepankar created HBASE-16624:
-

 Summary: MVCC DeSerialization bug in the upstream HFileScannerImpl
 Key: HBASE-16624
 URL: https://issues.apache.org/jira/browse/HBASE-16624
 Project: HBase
  Issue Type: Bug
  Components: HFile
Affects Versions: 2.0.0
Reporter: deepankar
Assignee: deepankar
Priority: Blocker


My colleague [~naggarwal] found a bug in the deserialization of mvcc from 
HFile, As a part of the optimization of deserialization of VLong, we read a int 
at once but we forgot to convert it to unsigned one. 

This would cause issues because once we cross the integer threshold in 
sequenceId and a compaction happens we would write MAX_MEMSTORE_TS in the 
trailer as 0 (because we will be reading negative values from the file that got 
flushed with sequenceId > Integer.MAX_VALUE). And once we have MAX_MEMSTORE_TS 
as 0, and there are sequenceId values present alongside with KeyValues the 
regionserver will now start failing to read the compacted file and thus 
corruption. 

Interestingly this would happen only on the tables that don't have  
DataBlockEncoding enabled and unfortunately in our case that turned out to be 
META and a another small table.

Fix is small (~20 chars) and attached



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


[jira] [Created] (HBASE-16630) Fragmentation in long running Bucket Cache

2016-09-13 Thread deepankar (JIRA)
deepankar created HBASE-16630:
-

 Summary: Fragmentation in long running Bucket Cache
 Key: HBASE-16630
 URL: https://issues.apache.org/jira/browse/HBASE-16630
 Project: HBase
  Issue Type: Bug
  Components: BucketCache
Affects Versions: 1.2.3, 1.1.6, 2.0.0, 1.3.1
Reporter: deepankar
Assignee: deepankar


As we are running bucket cache for a long time in our system, we are observing 
cases where some nodes after some time does not fully utilize the bucket cache, 
in some cases it is even worse in the sense they get stuck at a value < 0.25 % 
of the bucket cache (DEFAULT_MEMORY_FACTOR as all our tables are configured 
in-memory for simplicity sake).

We took a heap dump and analyzed what is happening and saw that is classic case 
of fragmentation, current implementation of BucketCache (mainly 
BucketAllocator) relies on the logic that fullyFreeBuckets are available for 
switching/adjusting cache usage between different bucketSizes . But once a 
compaction / bulkload happens and the blocks are evicted from a bucket size , 
these are usually evicted from random places of the buckets of a bucketSize and 
thus locking the number of buckets associated with a bucketSize and in the 
worst case of the fragmentation we have seen some bucketSizes with occupancy 
ratio of <  10 % But they dont have any completelyFreeBuckets to share with the 
other bucketSize. 

Currently the existing eviction logic helps in the cases where cache used is 
more the MEMORY_FACTOR or MULTI_FACTOR and once those evictions are also done, 
the eviction (freeSpace function) will not evict anything and the cache 
utilization will be stuck at that value without any allocations for other 
required sizes.

The fix for this we came up with is simple that we do deFragmentation ( 
compaction) of the bucketSize and thus increasing the occupancy ratio and also 
freeing up the buckets to be fullyFree, this logic itself is not complicated as 
the bucketAllocator takes care of packing the blocks in the buckets, we need 
evict and re-allocate the blocks for all the BucketSizes that dont fit the 
criteria.

I am attaching an initial patch just to give an idea of what we are thinking 
and I'll improve it based on the comments from the community.




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


[jira] [Created] (HBASE-16878) Call Queue length enforced not accounting for the queue handler factor

2016-10-18 Thread deepankar (JIRA)
deepankar created HBASE-16878:
-

 Summary: Call Queue length enforced not accounting for the queue 
handler factor
 Key: HBASE-16878
 URL: https://issues.apache.org/jira/browse/HBASE-16878
 Project: HBase
  Issue Type: Bug
  Components: rpc
Affects Versions: 0.98.4, 1.0.0
Reporter: deepankar
Assignee: deepankar
Priority: Minor


After HBASE-11355 currently we are not accounting for the handler factor when 
deciding callQueue length, this leading to some pretty large queue sizes if the 
handler factor is high enough.

Patch attached, change is oneline






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


[jira] [Created] (HBASE-17711) Add test for HBASE-16630 fix

2017-02-28 Thread deepankar (JIRA)
deepankar created HBASE-17711:
-

 Summary: Add test for HBASE-16630 fix
 Key: HBASE-17711
 URL: https://issues.apache.org/jira/browse/HBASE-17711
 Project: HBase
  Issue Type: Task
Reporter: deepankar
Assignee: deepankar






--
This message was sent by Atlassian JIRA
(v6.3.15#6346)


[jira] [Created] (HBASE-10645) Fix wrapping of Requests Counts Regionserver level metrics

2014-02-28 Thread deepankar (JIRA)
deepankar created HBASE-10645:
-

 Summary: Fix wrapping of Requests Counts Regionserver level 
metrics 
 Key: HBASE-10645
 URL: https://issues.apache.org/jira/browse/HBASE-10645
 Project: HBase
  Issue Type: Bug
  Components: regionserver
Affects Versions: 0.94.15
Reporter: deepankar
Priority: Trivial


For aggregating the metrics from all of its regions HRegionServer is using int,
while the underlying region level metrics use long. Due to this they get 
wrapped around giving out negative values if the regionserver is not restarted 
for a long time.



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)


[jira] [Created] (HBASE-10811) StoreFile StartKey not included in Bloom Filter Meta

2014-03-23 Thread deepankar (JIRA)
deepankar created HBASE-10811:
-

 Summary: StoreFile StartKey not included in Bloom Filter Meta
 Key: HBASE-10811
 URL: https://issues.apache.org/jira/browse/HBASE-10811
 Project: HBase
  Issue Type: Improvement
  Components: HFile
Reporter: deepankar
Priority: Minor


The StartKey of the StoreFile is not included in the Bloom Filter Meta and this 
information is not leveraged while excluding the store scanners that are to be 
created for a scan(passesGeneralBloomFilter Function in StoreFile Class) , This 
is a small Optimization. Is there are any particular reason for this? or Am I 
missing something here?.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Resolved] (HBASE-10811) StoreFile StartKey not included in Bloom Filter Meta

2014-03-23 Thread deepankar (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-10811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

deepankar resolved HBASE-10811.
---

Resolution: Duplicate

> StoreFile StartKey not included in Bloom Filter Meta
> 
>
> Key: HBASE-10811
> URL: https://issues.apache.org/jira/browse/HBASE-10811
> Project: HBase
>  Issue Type: Improvement
>  Components: HFile
>Reporter: deepankar
>Priority: Minor
>
> The StartKey of the StoreFile is not included in the Bloom Filter Meta and 
> this information is not leveraged while excluding the store scanners that are 
> to be created for a scan(passesGeneralBloomFilter Function in StoreFile 
> Class) , This is a small Optimization. Is there are any particular reason for 
> this? or Am I missing something here?.



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (HBASE-11335) Fix the TABLE_DIR param in TableSnapshotInputFormat

2014-06-11 Thread deepankar (JIRA)
deepankar created HBASE-11335:
-

 Summary: Fix the TABLE_DIR param in TableSnapshotInputFormat
 Key: HBASE-11335
 URL: https://issues.apache.org/jira/browse/HBASE-11335
 Project: HBase
  Issue Type: Bug
  Components: mapreduce, snapshots
Affects Versions: 0.98.3, 0.96.2
Reporter: deepankar


In class *TableSnapshotInputFormat* or *TableSnapshotInputFormatImpl*
in the function 
{code}
public static void setInput(Job job, String snapshotName, Path restoreDir) 
throws IOException {
{code}
we are setting restoreDir (temporary root) to tableDir
{code}
conf.set(TABLE_DIR_KEY, restoreDir.toString());
{code}

The above parameter is used to get the InputSplits, especially for 
calculating favorable hosts in the function
{code}
Path tableDir = new Path(conf.get(TABLE_DIR_KEY));

List hosts = getBestLocations(conf,
  HRegion.computeHDFSBlocksDistribution(conf, htd, hri, tableDir));
{code}

This will lead to returning a empty *HDFSBlocksDistribution*, as there is 
will be no directory with name as the region name from hri in the restored
root directory, which will lead to scheduling of non local tasks.

The change is simple in the sense, is to call the 
{code}FSUtils.getTableDir(rootDir, tableDesc.getTableName()) {code}
in the getSplits function

more discussion in the comments below 

https://issues.apache.org/jira/browse/HBASE-8369?focusedCommentId=14012085&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14012085




--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (HBASE-11484) Provide a way in TableSnapshotInputFormat, not to restore the regions to a path for running MR every time, rather reuse a already restored path

2014-07-09 Thread deepankar (JIRA)
deepankar created HBASE-11484:
-

 Summary: Provide a way in TableSnapshotInputFormat, not to restore 
the regions to a path for running MR every time, rather reuse a already 
restored path
 Key: HBASE-11484
 URL: https://issues.apache.org/jira/browse/HBASE-11484
 Project: HBase
  Issue Type: New Feature
  Components: mapreduce
Reporter: deepankar
Priority: Minor


We are trying to back a Hive Table by the Map Reduce over snapshots  and we 
don't want to restore the snapshot to a restoreDir every time we want to 
execute a query. It would be nice if there is boolean in the function 
* TableSnapshotInputFormat.setInput * and exposed outside in the
* TableMapReduceUtil.initTableSnapshotMapperJob *, with this boolean
it will check whether the snapshot and the restore dir are in sync, rather than 
restoring again. 

Is this Idea looks Ok to you guys or you have any other suggestions, I will put 
up a patch for this if this idea is ok for guys



--
This message was sent by Atlassian JIRA
(v6.2#6252)


[jira] [Created] (HBASE-15063) Bug(s) in MultiByteBuf

2016-01-01 Thread deepankar (JIRA)
deepankar created HBASE-15063:
-

 Summary: Bug(s) in MultiByteBuf
 Key: HBASE-15063
 URL: https://issues.apache.org/jira/browse/HBASE-15063
 Project: HBase
  Issue Type: Bug
  Components: io, Performance
Affects Versions: 2.0.0
Reporter: deepankar
Priority: Blocker


In MutliByteBuff there are couple of inconsistencies, one is in the toBytes 
function where the offset for copying in the initial element is not being reset 
with respect to the element
{code}
 public byte[] toBytes(int offset, int length) {
 byte[] output = new byte[length];
 int itemIndex = getItemIndex(offset);
 ByteBuffer item = this.items[itemIndex];
 // the offset has to be reset here to the items offset 
 // should be offset = offset - itemBeingPos[itemIndex]
 int toRead = item.limit() - offset;
 int destinationOffset = 0;
.  .   .   .
{code}
Since there is already an existing function get to copy to an byte array it is 
better we reuse the function here, I attached a patch with a corresponding unit 
test. (HBASE-.patch)

Another inconsistency I noticed is that there is lack of some consistency in 
using the position marker of the bytebuffers passed, In the constructor we 
noting the beginning offsets of each bytebuffer in the {{itemBeginPos}} array 
we are using these position markers in most of the absolute index functions 
such as {{get(index)}}, {{put(index, byte)}}, {{toBytes}}, 
{{get(int,byte[],int,int)}} to find the current item to access. There are two 
problems with this
# The array {{itemBeginPos}} is not being updated whenever there are some 
writes to internal bytebuffers (remember {{itemBeginPos}} depends on the 
{{bytebuffer.position()}} of the internal bytebuffers and the position marker 
is changed whenever some writes happen to bytebuffers)
# Also the position marker is not being used in any of the index functions for 
example here in the get function
   {code}
 @Override
  public byte get(int index) {
int itemIndex = getItemIndex(index);
return ByteBufferUtils.toByte(this.items[itemIndex], index - 
this.itemBeginPos[itemIndex]);
  } 
   {code}
where the the index I think should be {{index - this.itemBeginPos\[itemIndex\] 
+ items\[itemIndex\].position()}} because we are using the position marker to 
calculate the offsets.

There are two solutions I think for this 
# The simple solution I feel for this should be probably ignoring the position 
marker while calculating the {{itemBeginPos}} array which will unify the 
semantics 
# Not use this array at all and iterate over bytebuffers every time for the 
{{getItemIndex}} function and also use the position marker before calling the 
{{ByteBufferUtils}} functions

I can put up a patch for the second part if we decide which way to go.




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


[jira] [Created] (HBASE-15064) Possbile Bug in MultiByteBuffer

2016-01-01 Thread deepankar (JIRA)
deepankar created HBASE-15064:
-

 Summary: Possbile Bug in MultiByteBuffer
 Key: HBASE-15064
 URL: https://issues.apache.org/jira/browse/HBASE-15064
 Project: HBase
  Issue Type: Bug
  Components: io
Affects Versions: 2.0.0
Reporter: deepankar
Priority: Critical


While running the newer patches on our production system, I saw this error come 
couple of times 
{noformat}
ipc.RpcServer: Unexpected throwable object 
2016-01-01 16:42:56,090 ERROR 
[B.defaultRpcServer.handler=20,queue=20,port=60020] ipc.RpcServer: Unexpected 
throwable object 
java.nio.BufferUnderflowException
at java.nio.Buffer.nextGetIndex(Buffer.java:500)
at java.nio.DirectByteBuffer.get(DirectByteBuffer.java:249)
at org.apache.hadoop.hbase.nio.MultiByteBuff.get(MultiByteBuff.java:494)
at 
org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder$1.decode(FastDiffDeltaEncoder.java:402)
 
at 
org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder$1.decodeNext(FastDiffDeltaEncoder.java:517)
 
at 
org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder$BufferedEncodedSeeker.next(BufferedDataBlockEncoder.java:815)
at 
org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:138)
{noformat}

Looking at the get code 
{code}
if (this.curItem.remaining() == 0) {
  if (items.length - 1 == this.curItemIndex) {
// means cur item is the last one and we wont be able to read a long. 
Throw exception
throw new BufferUnderflowException();
  }
  this.curItemIndex++;
  this.curItem = this.items[this.curItemIndex];
}
return this.curItem.get();
{code}

Can the new currentItem have zero elements (position == limit), does it make 
sense to change the {{if}} to {{while}} ? {{while (this.curItem.remaining() == 
0)}}. This logic is repeated may make sense abstract to a new function if we 
plan to change to  {{if}} to {{while}}



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


[jira] [Created] (HBASE-15101) Leak References to StoreFile.Reader after HBASE-13082

2016-01-13 Thread deepankar (JIRA)
deepankar created HBASE-15101:
-

 Summary: Leak References to StoreFile.Reader after HBASE-13082
 Key: HBASE-15101
 URL: https://issues.apache.org/jira/browse/HBASE-15101
 Project: HBase
  Issue Type: Bug
  Components: HFile, io
Affects Versions: 2.0.0
Reporter: deepankar


We observed this production that after a region server dies there are huge 
number of hfiles in that region for the region server running the version with 
HBASE-13082, In the doc it is given that it is expected to happen, but we found 
a one place where scanners are not being closed. If the scanners are not closed 
their references are not decremented and that is leading to the issue of huge 
number of store files not being finalized

All I was able to find is in the selectScannersFrom, where we discard some of 
the scanners and we are not closing them. I am attaching a patch for that.

Also to avoid these issues should the files that are done be logged and 
finalized (moved to archive) as a part of region close operation. This will 
solve any leaks that can happen and does not cause any dire consequences?











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


[jira] [Created] (HBASE-15361) Remove unnecessary or Document constraints on BucketCache possible bucket sizes

2016-02-29 Thread deepankar (JIRA)
deepankar created HBASE-15361:
-

 Summary: Remove unnecessary or Document  constraints on 
BucketCache possible bucket sizes
 Key: HBASE-15361
 URL: https://issues.apache.org/jira/browse/HBASE-15361
 Project: HBase
  Issue Type: Sub-task
  Components: BucketCache
Reporter: deepankar
Priority: Minor


When we were trying to tune the bucket sizes {{hbase.bucketcache.bucket.sizes}} 
according to our workload, we encountered an issue due to the way offset is 
stored in the bucket entry. We divide the offset into integer base and byte 
value and it assumes that all bucket offsets  will be a multiple of 256 (left 
shifting by 8). See the code below
{code}

long offset() { // Java has no unsigned numbers
  long o = ((long) offsetBase) & 0x;
  o += (((long) (offset1)) & 0xFF) << 32;
  return o << 8;
}

private void setOffset(long value) {
  assert (value & 0xFF) == 0;
  value >>= 8;
  offsetBase = (int) value;
  offset1 = (byte) (value >> 32);
}
{code}

This was there to save 3 bytes per BucketEntry instead of using long and when 
there are no other fields in the Bucket Entry, but now there are lot of fields 
in the bucket entry , This not documented so we could either document the 
constraint that it should be a strict 256 bytes multiple of just go away with 
this constraint.  




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


[jira] [Created] (HBASE-15362) Compression Algorithm does not respect config params from hbase-site

2016-02-29 Thread deepankar (JIRA)
deepankar created HBASE-15362:
-

 Summary: Compression Algorithm does not respect config params from 
hbase-site
 Key: HBASE-15362
 URL: https://issues.apache.org/jira/browse/HBASE-15362
 Project: HBase
  Issue Type: Bug
Reporter: deepankar
Assignee: deepankar
Priority: Trivial


Compression creates conf using new Configuration() and this leads to it not 
respecting the confs set in hbase-site, fixing it is trivial using 
HBaseConfiguration.create()



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


[jira] [Created] (HBASE-15437) Response size calculated in RPCServer for warning tooLarge responses does count CellScanner payload

2016-03-09 Thread deepankar (JIRA)
deepankar created HBASE-15437:
-

 Summary: Response size calculated in RPCServer for warning 
tooLarge responses does count CellScanner payload
 Key: HBASE-15437
 URL: https://issues.apache.org/jira/browse/HBASE-15437
 Project: HBase
  Issue Type: Bug
  Components: IPC/RPC
Reporter: deepankar


After HBASE-13158 where we respond back to RPCs with cells in the payload , the 
protobuf response will just have the count the cells to read from payload, but 
there are set of features where we log warn in RPCServer whenever the response 
is tooLarge, but this size now is not considering the sizes of the cells in the 
PayloadCellScanner. Code form RPCServer
{code}
  long responseSize = result.getSerializedSize();
  // log any RPC responses that are slower than the configured warn
  // response time or larger than configured warning size
  boolean tooSlow = (processingTime > warnResponseTime && warnResponseTime 
> -1);
  boolean tooLarge = (responseSize > warnResponseSize && warnResponseSize > 
-1);
  if (tooSlow || tooLarge) {
// when tagging, we let TooLarge trump TooSmall to keep output simple
// note that large responses will often also be slow.
logResponse(new Object[]{param},
md.getName(), md.getName() + "(" + param.getClass().getName() + ")",
(tooLarge ? "TooLarge" : "TooSlow"),
status.getClient(), startTime, processingTime, qTime,
responseSize);
  }
{code}

Should this feature be not supported any more or should we add a method to 
CellScanner or a new interface which returns the serialized size (but this 
might not include the compression codecs which might be used during response ?) 
Any other Idea this could be fixed ?



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


[jira] [Reopened] (HBASE-15064) BufferUnderflowException after last Cell fetched from an HFile Block served from L2 offheap cache

2016-03-19 Thread deepankar (JIRA)

 [ 
https://issues.apache.org/jira/browse/HBASE-15064?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

deepankar reopened HBASE-15064:
---

> BufferUnderflowException after last Cell fetched from an HFile Block served 
> from L2 offheap cache
> -
>
> Key: HBASE-15064
> URL: https://issues.apache.org/jira/browse/HBASE-15064
> Project: HBase
>  Issue Type: Bug
>  Components: io
>Affects Versions: 2.0.0
>Reporter: deepankar
>Assignee: Anoop Sam John
>Priority: Critical
> Fix For: 2.0.0
>
> Attachments: HBASE-15064.patch
>
>
> While running the newer patches on our production system, I saw this error 
> come couple of times 
> {noformat}
> ipc.RpcServer: Unexpected throwable object 
> 2016-01-01 16:42:56,090 ERROR 
> [B.defaultRpcServer.handler=20,queue=20,port=60020] ipc.RpcServer: Unexpected 
> throwable object 
> java.nio.BufferUnderflowException
> at java.nio.Buffer.nextGetIndex(Buffer.java:500)
> at java.nio.DirectByteBuffer.get(DirectByteBuffer.java:249)
> at org.apache.hadoop.hbase.nio.MultiByteBuff.get(MultiByteBuff.java:494)
> at 
> org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder$1.decode(FastDiffDeltaEncoder.java:402)
>  
> at 
> org.apache.hadoop.hbase.io.encoding.FastDiffDeltaEncoder$1.decodeNext(FastDiffDeltaEncoder.java:517)
>  
> at 
> org.apache.hadoop.hbase.io.encoding.BufferedDataBlockEncoder$BufferedEncodedSeeker.next(BufferedDataBlockEncoder.java:815)
> at 
> org.apache.hadoop.hbase.regionserver.StoreFileScanner.next(StoreFileScanner.java:138)
> {noformat}
> Looking at the get code 
> {code}
> if (this.curItem.remaining() == 0) {
>   if (items.length - 1 == this.curItemIndex) {
> // means cur item is the last one and we wont be able to read a long. 
> Throw exception
> throw new BufferUnderflowException();
>   }
>   this.curItemIndex++;
>   this.curItem = this.items[this.curItemIndex];
> }
> return this.curItem.get();
> {code}
> Can the new currentItem have zero elements (position == limit), does it make 
> sense to change the {{if}} to {{while}} ? {{while (this.curItem.remaining() 
> == 0)}}. This logic is repeated may make sense abstract to a new function if 
> we plan to change to  {{if}} to {{while}}



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


[jira] [Created] (HBASE-15525) Fix OutOfMemory that could occur when using BoundedByteBufferPool during RPC bursts

2016-03-23 Thread deepankar (JIRA)
deepankar created HBASE-15525:
-

 Summary: Fix OutOfMemory that could occur when using 
BoundedByteBufferPool during RPC bursts
 Key: HBASE-15525
 URL: https://issues.apache.org/jira/browse/HBASE-15525
 Project: HBase
  Issue Type: Bug
  Components: IPC/RPC
Reporter: deepankar


After HBASE-13819 the system some times run out of direct memory whenever there 
is some network congestion or some client side issues.
This was because of pending RPCs in the RPCServer$Connection.responseQueue and 
since all the responses in this queue hold a buffer for cellblock from 
BoundedByteBufferPool this could takeup a lot of memory if the 
BoundedByteBufferPool's moving average settles down towards a higher value 

See the discussion here 
https://issues.apache.org/jira/browse/HBASE-13819?focusedCommentId=15207822&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15207822




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