Skip to content

Instantly share code, notes, and snippets.

@keith-turner
Last active December 7, 2022 12:15
Show Gist options
  • Save keith-turner/d57098befc53de04ddd7e4f2684528c8 to your computer and use it in GitHub Desktop.
Save keith-turner/d57098befc53de04ddd7e4f2684528c8 to your computer and use it in GitHub Desktop.
Accumulo compaction drop behind experiment

Accumulo compaction drop behind experiment

This a summary of test run to see if the drop behind settings make a noticable difference for Accumulo compactions. No differences were seen. A test with C code was run and differences were seen. One difference between the C and Accumulo code is the C code is only reading data. Further investigation is needed. Not sure if there is a bug in Hadoop/Accumulo or if there was a problem with the test.

Setup

These test were run using this commint from this branch which is a modified verions of #3083

To generate data for Accumulo to compact the following accumulo-testing command was run. Test were conducted on a laptop with 16G of RAM and a single DN and tserver setup by Uno.

$ ./bin/cingest createtable
$ ./bin/cingest ingest -o test.ci.ingest.client.entries=20000000 -o test.ci.ingest.delete.probability=0.0

Accumulo experiment

Below is an experiment of compacting w/o drop behind and looking and meminfo before and after.

$ accumulo shell -e "config -t ci -s table.compaction.major.input.drop.cache=false"
$ accumulo shell -e "config -t ci -s table.compaction.major.output.drop.cache=false"
$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2281948 kB
SwapCached:       138420 kB
Active(file):     155296 kB
Inactive(file):   227312 kB
$ accumulo shell -e "du -t ci"
             782,260,524 [ci]
$ accumulo shell -e "compact -t ci -w"
2022-11-25T17:02:36,338 [shell.Shell] INFO : Compacting table ...
2022-11-25T17:02:56,860 [shell.Shell] INFO : Compaction of table ci completed for given range
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          3243120 kB
SwapCached:       142680 kB
Active(file):     266360 kB
Inactive(file):  1088056 kB

Below is the same experimentas above except w/ drop behind. The drop behind settings did not seem to clear cache.


$ accumulo shell -e "config -t ci -s table.compaction.major.output.drop.cache=true"
$ accumulo shell -e "config -t ci -s table.compaction.major.input.drop.cache=true"
$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2270704 kB
SwapCached:       142684 kB
Active(file):     226084 kB
Inactive(file):   146692 kB
$ accumulo shell -e "compact -t ci -w"
2022-11-25T17:04:44,771 [shell.Shell] INFO : Compacting table ...
2022-11-25T17:05:05,809 [shell.Shell] INFO : Compaction of table ci completed for given range
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          3197684 kB
SwapCached:       145116 kB
Active(file):     286048 kB
Inactive(file):  1034448 kB

Looking in the tserver logs there is evidence that the hadoop drop behind calls are being made during the compaction above.

2022-11-25T17:04:45,208 [rfile.RFileOperations] DEBUG: Called setDropBehind(TRUE) for stream writing file hdfs://localhost:8020/accumulo/tables/1/t-0000000/A00000op.rf_tmp
2022-11-25T17:04:45,209 [impl.CachableBlockFile] DEBUG: Called setDropBehind(TRUE) for stream reading file hdfs://localhost:8020/accumulo/tables/1/t-0000000/A00000o5.rf
2022-11-25T17:04:45,210 [rfile.RFileOperations] DEBUG: Called setDropBehind(TRUE) for stream writing file hdfs://localhost:8020/accumulo/tables/1/t-0000003/A00000os.rf_tmp
2022-11-25T17:04:45,210 [rfile.RFileOperations] DEBUG: Called setDropBehind(TRUE) for stream writing file hdfs://localhost:8020/accumulo/tables/1/t-0000001/A00000oq.rf_tmp
   .
   .
   .
2022-11-25T17:05:01,110 [rfile.RFileOperations] DEBUG: Called setDropBehind(TRUE) for stream writing file hdfs://localhost:8020/accumulo/tables/1/t-0000006/A00000p7.rf_tmp
2022-11-25T17:05:01,111 [impl.CachableBlockFile] DEBUG: Called setDropBehind(TRUE) for stream reading file hdfs://localhost:8020/accumulo/tables/1/t-0000006/A00000on.rf
2022-11-25T17:05:01,226 [rfile.RFileOperations] DEBUG: Called setDropBehind(TRUE) for stream writing file hdfs://localhost:8020/accumulo/tables/1/t-0000005/A00000p8.rf_tmp
2022-11-25T17:05:01,227 [impl.CachableBlockFile] DEBUG: Called setDropBehind(TRUE) for stream reading file hdfs://localhost:8020/accumulo/tables/1/t-0000005/A00000oo.rf

I tried using strace on the datanode also (wanted to see the DN system calls to fadvise). Not sure what to make of that though, need to try that again.

C Experiment

After not seeing anything in the Accumulo experiments. I created a simple C program to see if I could see a difference in /proc/meminfo

#include <stdio.h>
#include <fcntl.h>
#include <stdint.h>
#include <inttypes.h>

int main(int argc, char *argv[])
{

    if(argc != 3) {
	    return -1;
    }

    char b[4096*16];
    FILE *fp = fopen(argv[1],"rb");
    int fd = fileno(fp);
    if(!fp){
	    return -1;
    }

    int64_t totalRead = 0;

    size_t ret_code = fread(b, sizeof *b, 4096*16, fp);
    totalRead +=ret_code;
    while(ret_code > 0) {
      //printf("read %d\n", (int)ret_code);
      ret_code = fread(b, sizeof *b, 4096*16, fp);
      totalRead+=ret_code;
    }

    if(argv[2][0] == 'y'){
      printf("calling fadvise\n");
      posix_fadvise(fd,0,0,POSIX_FADV_DONTNEED);
    }

    printf("read %" PRId64 "\n", totalRead);
    fclose(fp);
}

Below is the output of running the program w/ and w/o fadvise. Do see a difference in cache in /proc/meminfo here.

$ head -c 1000000000 /dev/urandom > test.bin
$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2118404 kB
SwapCached:       138160 kB
Active(file):     108880 kB
Inactive(file):   103240 kB
$ gcc testfp.c -o testfp
$ ./testfp test.bin n
read 1000000000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2947948 kB
SwapCached:       138160 kB
Active(file):     147472 kB
Inactive(file):   903356 kB
$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ ./testfp test.bin y
calling fadvise
read 1000000000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2133972 kB
SwapCached:       138160 kB
Active(file):     125160 kB
Inactive(file):   101976 kB
$ 

HDFS Drop Behind Experiments

Wrote some simple java programs to try reaing and writing directly to HDFS calling setDropBehind.

Read test

Below is a simple java program that reads data an optionally calls setDropBehind.

package org.apache.accumulo.core;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import java.net.URI;

public class ReadDropBehindTest {
    public static void main(String[] args) throws Exception {

        if(args.length != 4) {
            System.err.printf("Usage: %s <hdfs uri> <file> <buffer size> <y|n>\n", ReadDropBehindTest.class.getSimpleName());
            System.exit(1);
        }

        Configuration conf = new Configuration();
        FileSystem fs = FileSystem.get(URI.create(args[0]), conf);

        var is = fs.open(new Path(args[1]));

        if(args[3].equals("y")) {
            is.setDropBehind(true);
        }

        byte[] data = new byte[Integer.parseInt(args[2])];

        int read;
        long total = 0;
        while((read = is.read(data,0, data.length)) !=-1) {
            total+=read;
        }

        is.close();

        System.out.printf("Total read : %,d\n", total);
    }
}

Copy a 1G file to HDFS and call above program with setDropBehind. Cache is only slightly larger after run.

$ hadoop fs -copyFromLocal ~/test.bin /
$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2380992 kB
SwapCached:       141596 kB
Active(file):     279460 kB
Inactive(file):    41668 kB
$ accumulo org.apache.accumulo.core.ReadDropBehindTest hdfs://127.0.0.1:8020 /test.bin 4096 y
Total read : 1,000,000,000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2467160 kB
SwapCached:       141596 kB
Active(file):     290360 kB
Inactive(file):   114976 kB

Do same thing as above except do not call setDropBehind. Cache is much bigger. This shows that setDropBehind works well for reading.

$ echo 3 | sudo tee /proc/sys/vm/drop_caches
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2368676 kB
SwapCached:       141596 kB
Active(file):     280048 kB
Inactive(file):    42672 kB
$ accumulo org.apache.accumulo.core.ReadDropBehindTest hdfs://127.0.0.1:8020 /test.bin 4096 n
Total read : 1,000,000,000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          3436980 kB
SwapCached:       141596 kB
Active(file):     293156 kB
Inactive(file):  1070128 kB

Write test

Simple program that can read data from stdin to a file in HDFS. Can optionally call setDropBehind and hsync.

package org.apache.accumulo.core;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import java.io.BufferedInputStream;
import java.net.URI;

public class WriteDropBehindTest {
    public static void main(String[] args) throws Exception {

        if(args.length != 5) {
            System.err.printf("Usage: %s <hdfs uri> <file> <buffer size> <y|n> <y|n>\n", WriteDropBehindTest.class.getSimpleName());
            System.exit(1);
        }

        Configuration conf = new Configuration();
        FileSystem fs = FileSystem.get(URI.create(args[0]), conf);


        var os = fs.create(new Path(args[1]), true);

        if(args[3].equals("y")) {
            System.out.println("calling etDropBehind(true)");
            os.setDropBehind(true);
        }

        byte[] data = new byte[Integer.parseInt(args[2])];

        BufferedInputStream is = new BufferedInputStream(System.in);

        int read;
        long total = 0;
        while((read = is.read(data,0, data.length)) !=-1) {
            os.write(data,0,read);
            total+=read;
        }

        if(args[4].equals("y")) {
            System.out.println("calling hsync()");
            os.hsync();
        }
        os.close();

        System.out.printf("Total read : %,d\n", total);
    }
}

Write 1G to HDFS w/o setDropBehind. Entire 1G seems to end up in cache as expected.

$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2362856 kB
SwapCached:       142540 kB
Active(file):     254500 kB
Inactive(file):    88848 kB
$ head -c 1000000000 /dev/urandom | accumulo org.apache.accumulo.core.WriteDropBehindTest hdfs://127.0.0.1:8020 /test2.bin 4096 n n
Total read : 1,000,000,000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          3401876 kB
SwapCached:       142540 kB
Active(file):     280852 kB
Inactive(file):  1106796 kB

Write 1G to HDFS w/ setDropBehind and w/o hsync. Cache ends up w/ 378mb, more than read experiment but less than fill 1G.

$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2364512 kB
SwapCached:       142540 kB
Active(file):     269056 kB
Inactive(file):    77052 kB
$ head -c 1000000000 /dev/urandom | accumulo org.apache.accumulo.core.WriteDropBehindTest hdfs://127.0.0.1:8020 /test2.bin 4096 y n
calling etDropBehind(true)
Total read : 1,000,000,000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2700416 kB
SwapCached:       142540 kB
Active(file):     283200 kB
Inactive(file):   378408 kB

Write 1G to HDFS w/ setDropBehind and w/ hsync. Seems to end up w/ similar cache size as experiment w/o hsync.

$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2390400 kB
SwapCached:       142540 kB
Active(file):     266396 kB
Inactive(file):    76148 kB
$ head -c 1000000000 /dev/urandom | accumulo org.apache.accumulo.core.WriteDropBehindTest hdfs://127.0.0.1:8020 /test2.bin 4096 y y
calling etDropBehind(true)
calling hsync()
Total read : 1,000,000,000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2670752 kB
SwapCached:       142540 kB
Active(file):     279548 kB
Inactive(file):   376364 kB

Next step is plan to write a C program that does similar task and see what cache size looks like. Have only tested reading in C program.

Calling fadvise dont need for write in C code.

Created the following C file with the name test_fa_write.c.

#include <stdio.h>
#include <fcntl.h>
#include <stdint.h>
#include <inttypes.h>

int main(int argc, char *argv[])
{

    if(argc != 3) {
	    return -1;
    }

    char b[4096*16];
    FILE *fp = fopen(argv[1],"wb");
    int fd = fileno(fp);
    if(!fp){
	    return -1;
    }

    int64_t totalRead = 0;

    size_t num_read = fread(b, sizeof *b, 4096*16, stdin);
    totalRead +=num_read;
    while(num_read > 0) {
      //printf("read %d\n", (int)ret_code);
      size_t num_wrote = fwrite(b, sizeof *b, num_read, fp);
      if(num_wrote != num_read) {
	      printf("Write failed\n");
	      return -1;
      }
      num_read = fread(b, sizeof *b, 4096*16, stdin);
      totalRead+=num_read;
    }

    if(argv[2][0] == 'y'){
      printf("calling fadvise\n");
      posix_fadvise(fd,0,0,POSIX_FADV_DONTNEED);
    }

    printf("read and wrote %" PRId64 "\n", totalRead);
    fclose(fp);
}

Ran the following test w/o fadvise which shows that the cache grows as expected.

$ gcc -o test_fa_write test_fa_write.c
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          1982724 kB
SwapCached:            0 kB
Active(file):     443492 kB
Inactive(file):   347148 kB
$ head -c 1000000000 /dev/urandom | ./test_fa_write test.bin n
read and wrote 1000000000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          2977576 kB
SwapCached:            0 kB
Active(file):     452656 kB
Inactive(file):  1341516 kB

Ran the following write experiment which called fadvise. Saw all the data was dropped from cache. This is in contrast w/ experiments w/ HDFS where only some of the data was dropped from cache.

$ echo 3 | sudo tee /proc/sys/vm/drop_caches
3
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          1970676 kB
SwapCached:            0 kB
Active(file):     445604 kB
Inactive(file):   355924 kB
$ head -c 1000000000 /dev/urandom | ./test_fa_write test.bin y
calling fadvise
read and wrote 1000000000
$ cat /proc/meminfo | egrep '\(file\)|Cached'
Cached:          1983140 kB
SwapCached:            0 kB
Active(file):     453292 kB
Inactive(file):   367120 kB
@keith-turner
Copy link
Author

There is discussion around this in Accumulo slack w/ useful info and context. https://the-asf.slack.com/archives/CERNB8NDC/p1669399302264189

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment