You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2022/11/14 20:38:47 UTC

[GitHub] [accumulo] dlmarion opened a new issue, #3078: Mark blocks for compacted files as not needed to be cached

dlmarion opened a new issue, #3078:
URL: https://github.com/apache/accumulo/issues/3078

   While looking at #3077 , I found https://issues.apache.org/jira/browse/HBASE-14098. This got me thinking whether or not we would be able to remove blocks that are no longer referenced from the BlockCache for files that were recently compacted.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315885313

   So, looking at https://linux.die.net/man/3/posix_fadvise, the advisory information is attached to the file descriptor. Each process, in this case the DataNode, has it's own file descriptor table. I don't know enough about the file descriptors on Linux to know if a you write a file and close it, then open it later, could it have the same file descriptor (and advisory information)?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion closed issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion closed issue #3078: Mark blocks for compacted files as not needed to be cached
URL: https://github.com/apache/accumulo/issues/3078


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1314366801

   I think there are two things that could be done here (assuming successful compaction):
   
     1. Call setDropBehind(true) for the compaction input files so that the DataNodes can drop the files from their page caches.
     2. Remove (or cause to be removed faster) any cached blocks in the BlockCache for the compaction input files.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315974009

   For the case of reading a file, I wonder if it only drops the caches on the replicas it read from of from all replicas (even then ones it did not read from.
   
   One case where dropping the caches could have an impact is when a file is shared between multiple tablets.  A situation like the following.
   
    * Table A has files F1,F2,F3
    * Table A splits into Tablets B and C
    * Tablet B compacts F1 and F2 into F4
    * Tablet C is running scans against F1 and F2
   
   In the case above maybe the compaction for Tablet B could cause some performance problems for Tablet C.  I don't think this case is something to worry about, I just thought it was interesting.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315330215

   Some good points were raised in the comments in https://issues.apache.org/jira/browse/HBASE-10052 and https://issues.apache.org/jira/browse/HBASE-14098. It looks like they settled on using this feature on input and output files of large compactions only.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315967145

   I didn't look closely at the range accounting. I think you can read a range, so if you do that, then it marks that range as not needed in the page cache. If you read the whole block, then it likely marks the entire file as not needed in the page cache.
   
   then I suppose the smaller the hadoop block size the better it is for the cache in that case.
   
   > I think that depends on what you mean by better. Dropping a larger block from the cache because it's not needed seems like it would be better than leaving it there.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315963932

    >It's in the [BlockSender.close](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java#L480) in the DataNode. So, after it sends the block to the client, it may (depending on the specified options) set POSIX_FADV_DONTNEED on a range of the file descriptor for the file (that is that block).
   
   If it reads the entire block and drops it from cache after reading it, then I suppose the smaller the hadoop block size the better it is for the cache in that case.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315283430

   > I think there are two things that could be done here (assuming successful compaction):
   >
   > 1. Call setDropBehind(true) for the compaction input files so that the DataNodes can drop the files from their page caches.
   > 2. Remove (or cause to be removed faster) any cached blocks in the BlockCache for the compaction input files.
   
   We should not have to worry about compactions causing blocks to be cached. When a file is opened for compaction [here](https://github.com/apache/accumulo/blob/f389518b6b9c4897e017edade95f927846214e6e/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java#L314-L316) no accumulo block caches are passed in.
   
   Would be nice to setDropBehind(true) for compaction input files.  Could also do that for the output file.  The output file is created [here](https://github.com/apache/accumulo/blob/f389518b6b9c4897e017edade95f927846214e6e/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java#L215).  
   
   Seems like we would need to modify the internal reader and writer builders to support the set drop behind option.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315288609

   > We should not have to worry about compactions causing blocks to be cached.
   
   I was more thinking that we could somehow remove (or mark for early removal) cached blocks for files there were recently input files for a successful compaction. I have looked at the cache's and this may not be possible. We may just have to wait for them to be evicted normally.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315293006

   > Could also do that for the output file.
   
   I wasn't suggesting that. I'm not sure how long the FADVISE hint persists. For example, if we created a new file during compaction and called setDropBehind() on it, at what point would the DN hosts cache the files in their page cache?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315942135

   It's in the [BlockSender.close](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java#L480) in the DataNode. So, after it sends the block to the client, it may (depending on the specified options) set POSIX_FADV_DONTNEED on a range of the file descriptor for the file (that is that block).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315880513

   > For example, if we created a new file during compaction and called setDropBehind() on it, at what point would the DN hosts cache the files in their page cache?
   
   Thats a good question.  I was assuming it was scoped to the lifetime of the file output stream, but not sure now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315979727

   > I wonder if it only drops the caches on the replicas it read from
   
   I believe that is the case, I added that in my commit [comment](https://github.com/apache/accumulo/pull/3079/commits/f180381655b45c97e29d51c7418ea9b33359ae9c). In the case of a shared file, they should be different byte offsets of the same file. I guess Accumulo can't take advantage of that because of the RFile structure. I'm not sure that the compaction would cause an issue against a running scan. I'm thinking that those blocks might already be cached at the TabletServer level.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1315930089

   Digging around in the hadoop code I think it creates an output stream per output block in the DN and calls fadvise on its file descriptor.  So it does seem like its tightly scoped to the clients output stream, but not completely sure, got a bit lost, below are some things I was looking at.  Did not look at the input path.
   
   https://github.com/apache/hadoop/blob/branch-3.3.5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java#L928
   
   https://github.com/apache/hadoop/blob/5ca626e3e3ea29c2c90632129a57e73e3a1a420a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/ReplicaOutputStreams.java#L157
   
   https://github.com/apache/hadoop/blob/5ca626e3e3ea29c2c90632129a57e73e3a1a420a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java#L407
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] keith-turner commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
keith-turner commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1317150080

   > I think that depends on what you mean by better. Dropping a larger block from the cache because it's not needed seems like it would be better than leaving it there.
   
   Yeah, better to drop it in either case.  Was thinking that large blocks can push more stuff out of cache than smaller ones.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] ivakegg commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
ivakegg commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1314371758

   In this case we are assuming that a compacted file will not be needed in the near future.  Is that a valid assumption?  I could see some places where that is not true (e.g. accumulo.metadata table).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [accumulo] dlmarion commented on issue #3078: Mark blocks for compacted files as not needed to be cached

Posted by GitBox <gi...@apache.org>.
dlmarion commented on issue #3078:
URL: https://github.com/apache/accumulo/issues/3078#issuecomment-1314382412

   I'm assuming that a tablet read *some time after* a compaction will not read any of the compacted files. If ScanServers are not being used, then that timeframe is likely very small. If ScanServers are being used, then the timeframe is the metadata cache expiration property value. Something also to consider is that compactions performed externally won't have access to the BlockCache, so this might be something internal to the BlockCache.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org