pvfs: design and implementation sam lang mathematics and computer science division argonne national...

64
PVFS: Design and Implementation Sam Lang Mathematics and Computer Science Division Argonne National Laboratory [email protected]

Upload: arabella-cross

Post on 29-Dec-2015

214 views

Category:

Documents


0 download

TRANSCRIPT

PVFS: Design and Implementation

Sam Lang

Mathematics and Computer Science Division

Argonne National Laboratory

[email protected]

Argonne National Laboratory PVFS: Design and Implementation 2

Talk Outline

Overview Design

– Handles Client

– Kernel Module

– Client Daemon

– System Interfaces

– State Machines Server

– Concurrency

– Components• Trove– Storage

• Flow• BMI

Argonne National Laboratory PVFS: Design and Implementation 3

PVFS Development History

PVFS version 2 based on experiences from PVFS version 1 Joint effort at Argonne

– Rob Ross

– Rob Latham

– Neill Miller And Clemson University

– Walt Ligon

– Phil Carns

– Brad Settlemeyer And OSC: Pete Wyckoff Additions by Murali Villayannur, Sam Lang

Argonne National Laboratory PVFS: Design and Implementation 4

PVFS Overview

Parallel Virtual File System Distributed yes, but more Parallel Virtual? Byte-regions instead of blocks Provide a well-known (file system) interface to data Allow parallel programs running on large clusters to store and access data

Client

Client

Client

Client

Client

Client

Client

Client

Server

Server

Server

Server

Argonne National Laboratory PVFS: Design and Implementation 5

PVFS Design

Designed for coordinated parallel programs

– Efficient support for MPI-IO via ROMIO driver for PVFS Scalable

– Datafile handles instead of blocks

– Assumes collective programs don’t need to lock for IO

– Consistency model doesn’t require locking for metadata

– Atomic metadata ops and serialize or rollback where necessary

– Avoid single point of contention where possible• E.g. File size

– Configurable file striping

– Non-contiguous IO patterns built-in Usability: Easy to install, support, augment

– small VFS kernel driver forwards work to the userspace client daemon

– Well defined plugable components for disk, network, flow

– Servers can use local file system

– State machine DSL provides a framework for adding operations

Argonne National Laboratory PVFS: Design and Implementation

OrangeFS Software Architecture

6

System InterfaceSystem Interface

State MachinesState Machines

BMIBMI FlowsFlows

Request SchedulerRequest Scheduler

State MachinesState Machines

BMIBMI FlowsFlows TroveTrove

Interconnection NetworkInterconnection Network Storage HWStorage HW

PVFS2 Client (libpvfs2) PVFS2 Server

Progress EngineProgress Engine Progress EngineProgress Engine

Argonne National Laboratory PVFS: Design and Implementation 7

PVFS Handles

A handle is a 64bit integer value refers to some type of object in PVFS Similar to inodes and dentries Scoped to a filesystem

– Only within a file system is it unique Mapping table from handle to server alias

– determines handle locality– limits allocation ranges– Example:

pvfs2-viewdist will show the datafile handles for a file

Handle types:– Metafile: refers to file attributes,

datafile handles array– Datafile: refers to actual data stripe

on disk (or bstream on local file system)– Directory: refers to directory attributes,

directory data– Directory Data: refers to directory’s entries– Symlink: refers to file target

<MetaHandleRanges> Range node202 4-286331155</MetaHandleRanges><DataHandleRanges> Range node202 286331156-572662307 Range node203 572662308-858993459 Range node204 858993460-1145324611 Range node205 1145324612-1431655763 Range node206 1431655764-1717986915 Range node207 1717986916-2004318067 Range node209 2004318068-2290649219 Range node210 2290649220-2576980371 Range node211 2576980372-2863311523 Range node212 2863311524-3149642675 Range node213 3149642676-3435973827 Range node214 3435973828-3722304979 Range node215 3722304980-4008636131 Range node216 4008636132-4294967283</DataHandleRanges>

Argonne National Laboratory PVFS: Design and Implementation

Handles, Metadata, and Data

8

Argonne National Laboratory PVFS: Design and Implementation 9

PVFS Handle Reference Tree

foo

Datafile Datafile Datafile

Datafile Datafile Datafile

Metadata Server 1

Metadata Server 2IO Server 1 IO Server 3IO Server 2

bar

bazMetafile

Metafile

Directory

Dir Data

Directory

Argonne National Laboratory PVFS: Design and Implementation 10

The PVFS system interfaces

A set of exported functions that provide interaction with the filesystem Client-side Similar to system calls Two types: Blocking and Non-blocking

– Blocking: PVFS_sys_<op>(); once called, the function does not return until the operation has completed

– Non-blocking: PVFS_isys_<op>(); returns immediately with a unique id to test for completion later

Each system interface call has an associated client state machine Examples:

– PVFS_isys_lookup - map a path to a handle

– PVFS_isys_create - creates a metafile

– PVFS_isys_readdir - for a given directory handle, returns a list of directory entries

– PVFS_isys_read - reads bytes from a file, takes a request structure

Argonne National Laboratory PVFS: Design and Implementation 11

Example PVFS System Interface: Create

PVFS_isys_create

Create metafile handle

Get parent attributes

Create datafile handles

Set datafile handle array

Create directory entry

Metadata Server A

Metadata Server B

IO Server 1

IO Server 2

IO Server N

create(MH)

create(DH)

create(DH)

create(DH)

getattr(PH)

crdirent(MH)

setattr(DHA)

Argonne National Laboratory PVFS: Design and Implementation 12

The Concurrency Model in PVFS

Motivation:

– PVFS (any distributed fs) uses devices extensively

– allow CPU to do work while devices make progress

All PVFS Component APIs follow the same pattern:

– Post a unit of work (also called an operation)• Returns a receipt, the operation id• Operations can be posted to a completion context

– Test for completion of operations• Test for a single operation: pass the operation id• Test for a set of operations: pass all the operation ids• Test on a completion context: specify the context

– Cancel an operation

Argonne National Laboratory PVFS: Design and Implementation 13

PVFS Non-blocking

Within the component, operations advance or make progress

– By queuing the operation and using the test calls to make progress

– By making blocking system calls to the kernel

– By starting a thread (pthread_create, thread pooling, etc.) Immediate Completion: a POST call may return COMPLETE

– Operations may be cached

– Small IO operations may finish within the post timeframe

Argonne National Laboratory PVFS: Design and Implementation 14

State Machines

A Domain Specific Language for PVFS server operations and client interfaces

Abstraction of an operation’s component operations Often done with continuations, switch/case blocks, or callback functions Motivation: easier to implement, debug, and visualize Used by PVFS client to implement entire system interface calls

– Each system interface call (PVFS_isys_*) has an associated state machine that performs operations

Used by PVFS server to implement a server operation Nested state machines do common work of state machines

– pvfs2_client_getattr_sm

– Pvfs2_msgpairarray_sm Individual state actions post jobs to components (bmi, trove)

– The job(s) posted from a state action drive the state machine

Argonne National Laboratory PVFS: Design and Implementation 15

State Machine and Job Diagram

Test Jobs andContinue SMs

State Machines JOB API

State MachinesPost Jobs

Argonne National Laboratory PVFS: Design and Implementation 16

State Machine Diagram: Client Create

Argonne National Laboratory PVFS: Design and Implementation 17

State Machine Diagram: Client Readdir

Argonne National Laboratory PVFS: Design and Implementation 18

State Machine: Message Pair Array

A Message Pair consists of a request and response Often want to send an operation to multiple servers simultaneously Specify server address list, the operation, and jump to the msgpairarray

state machine Once the state machine completes (returns to the parent SM) all the

operations will have completed Operation:

– First, post a receive for each pair

– Next, post a send for each pair

– Enter “complete” state each time a posted operation finishes• Determine which messages complete• Check to see if all are complete

– If there are recoverable errors, loop and repost those messages

Argonne National Laboratory PVFS: Design and Implementation 19

State Machine Diagram: MessagePair Array

Argonne National Laboratory PVFS: Design and Implementation 20

Designing a State Machine Determine component operations that make up a PVFS operation Define the state machine composed of states (operations) and transitions (results of

operations)– States can be:

• run a state action• jump to a nested state machine• pjmp to run a concurrent state machine

– Transitions map a result of a previous state action to the next state Define enum of transitions, positive valued transition codes Define state actions for each operation

– A function with a well defined signature:static PINT_sm_action fn_name(struct PINT_smcb *smcb, job_status_s *js_p)

– Return values from state action indicate job posted or not• Return SM_ACTION_DEFERRED indicates a job was posted. Wait for job

completion before transitioning.• Return SM_ACTION_COMPLETE indicates no job was posted. Transition to

next state immediately.• Return SM_ACTION_TERMINATE indicates state machine processing is

finished and state machine should shut down– js_p->error_code is set to indicate transition

• Set to pre-defined positive transition code to indicate the transition • A value of 0 indicates transition to the pre-defined success transition• Catch-all pre-defined default transition

Argonne National Laboratory PVFS: Design and Implementation

State Machine / Job Interaction

21

Call State Action Function

JS_P JS_P

Job

NEXT STATE

COMPLETE DEFERRED

PREVIOUSSTATE

Argonne National Laboratory PVFS: Design and Implementation

Flows

A flow is used to pass large amounts of data

– more than fits in a single message

– Generally used on for read or write operations Flow uses a rendezvous

– Keeps data from moving until both nodes are ready• Send request• Receive ack• Start flow• Receive write-ack (if a write operation)

Flows have two endpoints

– Network, Disk file, Memory are endpoints – one must be network

– Data is transferred directly from one endpoint to the other on the node

– Two nodes must participate, so 4 endpoints total• read: From disk to network -> network to memory• write: From memory to network -> network to disk

Flow protocol controls how data is buffered and transferred

– Currently there is only one default flow protocol

22

Argonne National Laboratory PVFS: Design and Implementation 23

Defining the State Machine

%%

machine my_machine_sm{ state foo { run foo_action; BAZAR => bazar; BAZ => baz; default => bar; }

state baz { run baz_action; success => bar; default => foo; }

state bazar { run bazar_action; BAZ => baz; default => bar; }

state bar { run bar_action; default => terminate; }}

%%

enum{ BAZAR = 1, BAZ = 2};

static PINT_sm_action foo_action( struct PINT_smcb *smcb, job_status_ s *js_p){ job_post(...); js_p->error_code = BAZAR; /* ignored */ return SM_ACTION_DEFERRED;}static PINT_sm_action bazar_action( struct PINT_smcb *smcb, jo_status_s *js_p){ js_p->error_code = BAZ; return SM_ACTION_COMPLETE;}

Always need a default transition

Must specify termination state

Argonne National Laboratory PVFS: Design and Implementation 24

Nested State Machines Nested state machines allow common functions to be shared between different file

system operations. They also allow important functions to be run in different ways.– Any state machine can be called as a nested state machine– Transition “return” is used to end a nested state machine– A runtime stack is maintained to help process state machines

• Every running state machine has a “frame” that is used to keep state variables for the state machine

• Frames can be of different types, depending on the state machine• When a nested state machine is called, it has the same frame as its caller,

unless …– The caller pushes a new frame onto the frame stack– The caller can set values in the new frame before running the nested state

machine– When the nested state machine returns, the caller gets its original frame

back, but the pushed frame remains on the stack until it is popped– The caller can retrieve values from the nested frame before popping it– The caller can push multiple frames, but the last one pushed becomes the

frame of the nested state machine• State machines access frames with the following function:void *PINT_sm_frame(smcb, PINT_FRAME_CURRENT);• 0 (PINT_FRAME_CURRENT) is the “base” frame, negative values are frames

pushed by the parent, positive values are frames pushed by the current state machine.

Argonne National Laboratory PVFS: Design and Implementation 25

Jumping to nested State Machines

Parent base frame

Base frame

Frames pushed by state machine

Frames pushed by parent

Argonne National Laboratory PVFS: Design and Implementation 26

Jumping to nested State Machines

%%

machine my_machine_sm{ state foo { run foo_action; BAZAR => bazar; BAZ => baz; default => bar; }

state baz { jump baz_sm; success => bar; default => foo; }

state bazar { run bazar_action; BAZ => baz; default => bar; }

state bar { run bar_action; default => terminate; }}

%%

Jumps to the baz state machine

%%machine baz_sm{ state bop { run bop_action; success => bop; default => quux; }

state quux { run quux_action; default => return; }}%%

Argonne National Laboratory PVFS: Design and Implementation 27

State Machines Implementation

SMCB – State Machine Control Block

– Opaque structure that manages a running state machine

– Manages runtime stack Frames

– Client

struct PINT_client_sm *sm_p

– Server

struct PINT_server_op *s_op

– Message Pair Array

Struct PINT_sm_msgarray_op *mop Last state action:

– Clients

PINT_SET_OP_COMPLETE;

return SM_ACTION_TERMINATE;

– Servers

return (server_state_machine_complete(smcb));

Argonne National Laboratory PVFS: Design and Implementation 28

State Machines Functions

• int PINT_smcb_alloc(struct PINT_smcb **smcb,

int op,

int frame_size,

struct PINT_state_machine_s *(*getmach)(int),

int (*term_fn)(struct PINT_smcb *, job_status_s *),

job_context_id context_id);• PINT_sm_action PINT_state_machine_start(struct PINT_smcb *smcb,

job_status_s *r);• void PINT_smcb_free(struct PINT_smcb *smcb)• int PINT_sm_push_frame(struct PINT_smcb *smcb,

int task_id,

void *frame_p);• void *PINT_sm_pop_frame(struct PINT_smcb *smcb,

int *task_id,

int *error_code,

int *remaining);

Argonne National Laboratory PVFS: Design and Implementation 29

Concurrent State Machines (pjmp)

Create multiple running state machines that execute concurrently.

– Simplifies some coding issues

– May improve performance by allowing overlap of tasks Push one or more new frames

– Each frame is pushed with a task_id

– Task_id identifies which SM will be executed pjmp state starts concurrent state machines

– One new state machine for each new frame

– State action provided to setup the new frames before creating tasks

– Completes when all of the new tasks have completed State following pjmp removes frames, analyzes results

Argonne National Laboratory PVFS: Design and Implementation

Concurrent State Machine Example

30

%%

machine my_machine_sm{ state state_1 { run state_action_1; success => state_2 default => state_3; }

state state_2 { pjmp state_action_2 { /* values here are set up */ /* in state_action_2 */ TASK_1 => state_machine_1; TASK_2 => state_machine_2; default => state_machine_3; } default => state_3; }

state state_3 { /* this state action cleans */ /* up after the pjmp */ run state_action_3; default => terminate; }}

%%

static PINT_sm_action state_action_2 ( struct PINT_smcb *smcb, job_status_s *js_p){ PINT_server_op *sop = (PINT_server_op *) PINT_sm_frame(smcb, PINT_FRAME_CURRENT); for (i = 0; i < sop->req.u.foo.num_servers; i++) { PINT_server_op *new = (PINT_server_op *) malloc(sizeof(PINT_server_op)); new->req.u.foo.blah = sop->req.u.foo.blah; /* determine which state machine to run */ if (i = 0) tag = RETVAL; /* run state_machine_3 */ else if (somevar > someval) tag = TASK_1; /* run state_machine_1 */ else tag = TASK_2; /* run state_machine_2 */ /* push frame */

PINT_push_frame(smcb, tag, new); } return SM_ACTION_DEFERED;}

Argonne National Laboratory PVFS: Design and Implementation

Anatomy of a SYSINT State Machine

Resides in src/client/sysint/sys-<op>.sm

– Includes code for PVFS_sys_<op>• Calls PVFS_isys_<op>• Waits for operation to complete

– Includes code for PVFS_isys_<op>• Creates state machine for the op• Initializes frame

State machine

– Usually consists of a sequence of one or more calls to msgpairarray_sm

– Each of these sends a request to a server and receives the response

– Results are analyzed and the op either proceeds or terminates

– Most ops center around an object identified with a PVFS_handle

– First step often to read the metadata for the object with getattr• PVFS_sys_getattr provides a nested state machine for use by other

ops and a FILL macro to set up the request

31

Argonne National Laboratory PVFS: Design and Implementation

Setting Up msgpairarray_sm Call

Initialize msgarray_op Initialize request specific fields Set up msgpairs as needed Push frame

32

PINT_msgpair_init(&sm_p->msgarray_op); msg_p = &sm_p->msgarray_op.msgpair;

object_ref = sm_p->getattr.object_ref;

/* setup the msgpair to do a getattr operation */ PINT_SERVREQ_GETATTR_FILL(msg_p->req, *sm_p->cred_p, object_ref.fs_id, object_ref.handle, sm_p->getattr.req_attrmask, sm_p->hints);

msg_p->fs_id = object_ref.fs_id; msg_p->handle = object_ref.handle; msg_p->retry_flag = PVFS_MSGPAIR_RETRY; msg_p->comp_fn = getattr_object_getattr_comp_fn; ret = PVFS_SID_get_addr(&msg_p->svr_addr, &msg_p->sid_array[0]); PINT_sm_push_frame(smcb, 0, &sm_p->msgarray_op);

Argonne National Laboratory PVFS: Design and Implementation

Automated Request Processing

On the client side, msgpairarray_sm handles many details of request

– Encoding request

– BMI interaction

– Receiving response

– Decoding response

– Passing back error codes Similar facilities are provided on the server Numerous pieces of code must be provided to make this work

– src/proto/pvfs2_req_proto.h• Defines request and response fields• Defines encoding/decoding routines– Macros in src/proto/endecode-funcs.h/.c– Update case statements in src/proto/PINT-le-bytefield.c

• Defines FILL macros• Defines request codes (unique value to identify the request)

33

Argonne National Laboratory PVFS: Design and Implementation

Adding State Machine Client-side

src/client/sysint/client-state-machine.h

– Add struct for syscall specific fields (struct PINT_client_<op>_sm)

– Add entry to the union in struct PINT_client_sm

– Add enumeration in enum PINT_client_post_op

– Add declaration of state machine function (pvfs2_client_<op>_sm)

src/client/sysint/client-state-machine.c

– Add entry in PINT_client_sm_sys_table[]

– Add entry in PINT_client_get_name_str()

include/pvfs2-sysint.h

– Add function prototypes for syscall and isyscall

– Add definition of syscall response struct (struct PVFS_sysresp_<op>_s)

34

Argonne National Laboratory PVFS: Design and Implementation

Adding State Machines Server-Side

Server state machines are initiated by receiving a request

– Server posted unexpected receive

– Message received

– Receive state machine “morphs” into request state machine Requests are fully decoded before the state machine starts

– s_op frame has pointer to both raw request and decoded request

– s_op also has a pointer to the response struct Most state machines start with a jump to “pvfs2_prelude_sm”

– Looks up target object

– Checks permissions

– Invokes request scheduler (delays request if a conflict exists) Most state machines end with a jump to “pvfs2_final_response_sm”

– Encodes response struct

– Sends response

– Frees resources

35

Argonne National Laboratory PVFS: Design and Implementation 36

Trove Storage

Bstream - Stores the stripe of data Trove Handle Allocation

– Handle entries are stored in the dspace db– Special in-memory handle ledger for allocation

Dataspace (dspace) - Stores attributes common to all types of handles– Object info: fs-id, handle, object type– User info (uid/gid) – Permissions– Timestamps

Keyval - Stores key/value pairs specific to handle type– Metadata:

• md: Distribution• dh: Datafile handles array

– Directory:• de: Directory entries handle

– Symlink:• st: Symlink target

– Directory Data:• <filename>: Metafile handle

Argonne National Laboratory PVFS: Design and Implementation 37

Trove Berkeley DB Usage

Main Databases:

– Dataspace:• Key: handle• Value: Common attributes

– Keyval:• Handle+<string>• <string> depends on the type of object:– dh, de, md, st

• Directory entries stored as Handle+<filename>• Special “null” keyval: Handle+NULL stores

Argonne National Laboratory PVFS: Design and Implementation

Server-Side State Machines

IO performed by calling job_trove_xxx

– job_trove_dspace_getattr() – setattr, create, remove, iterate_handles

– job_trove_keyval_read() – write, remove, iterate

– job_trove_bstream_write_list() – read_list

– job_flow() src/server/pvfs2-server.h

– Create op specific fields (struct PINT_<op>_op)

– Add entry to union in struct PINT_server_op

– Add SM function declaration src/server/pvfs2-server-req.c

– Add new entry to PINT_server_req_table[]

38

Argonne National Laboratory PVFS: Design and Implementation

Server Side SM Control

Required definitions

– struct PINT_server_req_params pvfs2_<op>_params

39

static int perm_lookup(PINT_server_op *s_op){ int ret; if (s_op->req->capability.op_mask & PINT_CAP_EXEC) { ret = 0; } else { ret = -PVFS_EACCES; } return ret;}PINT_GET_CREDENTIAL_DEFINE(lookup_path);PINT_GET_OBJECT_REF_DEFINE(lookup_path);

struct PINT_server_req_params pvfs2_lookup_params ={ .string_name = "lookup", .perm = perm_lookup, .sched_policy = PINT_SERVER_REQ_SCHEDULE, .get_credential = PINT_get_credential_lookup_path, .get_object_ref = PINT_get_object_ref_lookup_path, .state_machine = &pvfs2_lookup_sm};

Argonne National Laboratory PVFS: Design and Implementation 40

Server to Server Communication

Motivation

– Remember sys-create?

– System calls often consist of sending requests to many servers

– Much of the communication must be synchronized

– Fewer roundrips

– Rollback no longer necessary

Argonne National Laboratory PVFS: Design and Implementation 41

Server to Server: Create

PVFS_isys_create

Create

Get parent attributes

Create DHs

Set datafile handle array

Create directory entry

Metadata Server A

Metadata Server B

IO Server 1

IO Server 2

IO Server N

create_mfile

create

create

create

getattr

crdirent

Return New Handle

Done Create MH

Argonne National Laboratory PVFS: Design and Implementation 42

The PVFS VFS Client

Breakdown:

– Kernel module: pvfs2.ko

– Userspace client daemon: pvfs2-client

– System interfaces: PVFS_isys_* Clients are stateless

– No point of contention for file locks

– Clients can fail and not hose the system User apps make system calls to the PVFS mountpoint The Linux VFS traps the system calls, and invokes the file operation

callbacks in PVFS kernel module pvfs2-client reads a new operation from the kernel module (the upcall) Invokes system interfaces, which perform operations to servers Pvfs2-client writes the response back to the kmod (the downcall) The response of the system call is returned to the user

Argonne National Laboratory PVFS: Design and Implementation 43

PVFS Linux VFS Interaction (Control)

/pvfs > ls

Userspace Linux Kernel

readdir()

VFS

ext3 nfs

pvfs2.ko

pvfs2_directory_ops.readdir

pvfs2_devreq_ops

RequestQueue

pvfs2-client

/dev/pvfs2-req

poll()

read(readdir_op)

write(readdir_resp)

<readdir_op>

<op ready>

PVFS_isys_readdir

<readdir_resp>

upcall

downcall

.poll

.read

.write

Argonne National Laboratory PVFS: Design and Implementation 44

PVFS Linux VFS Interaction (IO)

> cp /tmp/foo /pvfs/foo

Userspace Linux Kernel

write(fd, userbuf)

VFS

ext3 nfs

pvfs2.ko

pvfs2_devreq_ops

RequestQueue

pvfs2-client

/dev/pvfs2-req

read(io_op)

write(io_resp)

<io_op>

PVFS_isys_io(io_op)

<io_resp>

upcall

downcall

mappedIO pages

copy_from_user(userbuf)

pvfs2_file_ops.write

get_user_pages()IO buffers

Argonne National Laboratory PVFS: Design and Implementation 45

PVFS Server

The server daemon waits for unexpected messages Essentially RPC mechanism: each unexpected message is an operation Operations invoke state machines (one-to-one) State machines asynchronously drive operations through to completion Response is returned

Trove

BMI

Flow

State MachineRequestScheduler

Accept Unexpected Operations

Argonne National Laboratory PVFS: Design and Implementation 46

Request Scheduler

Server operations operate on a single handle Some metadata operations modify handle attributes Could just use fine-grained locks at the TROVE layer

– Uncoordinated

– May still cause contention Instead, serialize operations that need to be FIFO ordering Allows for grouping of operations, sorting

Argonne National Laboratory PVFS: Design and Implementation 47

PVFS Component APIs

Trove - manages disk operations, both metadata and IO (server only) BMI - manages network operations Job - manages jobs for state machines Device - manages device operations (client only) Flow - provides network to disk interchange with managed buffers

Argonne National Laboratory PVFS: Design and Implementation 48

PVFS Server Component Interactions

BMI API Trove IO API

Flow API

Trove MD API

JOB POST API

JOBTESTAPI

State Machines

Argonne National Laboratory PVFS: Design and Implementation 49

Device API

PVFS Client Component Interactions

BMI API

JOBTESTAPI

JOB POST API

/dev/pvfs2

State Machines

Argonne National Laboratory PVFS: Design and Implementation 50

PVFS Server Threads

Main Server Thread

– Starts/Continues state machines

– Tests for completion of Jobs BMI Thread

– Tests for completion of BMI operations

– Moves completed BMI operations to the BMI completion array Trove Thread

– Tests for completion of trove operations

– Moves completed trove operations to the job completion array DBPF Thread

– Service metadata operations AIO/Alt-AIO Threads

Argonne National Laboratory PVFS: Design and Implementation 51

PVFS Server Thread Interaction

Server Thread

DBPF Thread

Trove CompletionThread

BMI CompletionThread

JobCompletion

Queue

Trove

Flow

AIOCOMPLETE

POSTTEST

FLOW POST

Functions

BMI

ServerJob

FlowTrove

Components

Argonne National Laboratory PVFS: Design and Implementation 52

PVFS Server Thread Interaction

TroveCompletion

Thread

JobCompletion

Queue

FlowTrove

DBPF Thread

AIO

BMI

ServerJob

FlowTrove

Components

BMICompletion

Thread

COMPLETEFLOW POST

Functions

POSTTEST

MetadataQueue

Argonne National Laboratory PVFS: Design and Implementation 53

Trove DBPF Method

On a post, operations are queued Separate DBPF thread services metadata operations IO is serviced through asynchronous vectored APIs Bytes streams (stripes) are stored as local files

– Much easier to manage than blocks directly

– No good interfaces for high performance IO, more on AIO later

– Cache open file descriptors Berkeley DB Advantages:

– Fast and easy to use

– Suits our needs

– Transactions with group commit Berkeley DB Caveats:

– Requires “tuning”

– Interface isn’t terribly expressive

– Don’t use DB_RECNUM

– DB_MULTIPLE_KEY is your friend

– Comparison function anecdote

Argonne National Laboratory PVFS: Design and Implementation 54

Trove: Sync Coalescing

Op queue SyncCompletion

QueueLow WatermarkHigh Watermark

Service Sync

Argonne National Laboratory PVFS: Design and Implementation 55

Trove: Sync Coalescing

Keyval modify operations with MetaSync enabled require separate DB calls:

– db->put: write the keyval

– db->sync: sync the DB to disk Keyval storage format changes

– Previous format: one berkeley db per file• Required caching open DBs• Used local fs to map handle to DB• Key: <string>• Individual keyval DBs were small

– New format: one berkeley db for file system• No open DB caching• Key: Handle+<string>• Large keyval DB: syncing becomes an issue

db->sync isn’t keyval specific Idea: coalesce keyval_db->put operations

Argonne National Laboratory PVFS: Design and Implementation 56

Trove: IO implementations

Asynchronous IO

– Provides libc calls that post vectored, positioned IO operations to disk

– POSIX standard

– Notification is configurable: callbacks, signals, none

– Linux implementation performs badly• Thread pool, but one thread per file• Creates a separate thread for callback notification• Alternatives (bull opensource, etc.) not stable or ubiquitous

Phil’s Alternative AIO

– Create a new thread for each lio_listio call

– Do IO with blocking pread/pwrite calls

– Call callback in same thread

Argonne National Laboratory PVFS: Design and Implementation 57

Flow

Provides a mechanism to saturate network and disk (server) Or network and memory copies (client) Only one flow method used: MultiQueue For each IO operation, flow MQ creates 8 fixed-length buffers Trove and BMI operations are posted in fixed-length segments

TroveBMI FLOW

Argonne National Laboratory PVFS: Design and Implementation 58

Flow Multiqueue Server Read

A8A2A1 B8B2B1

Trove Read A1 BMI Send A1 Trove Read B1 BMI Send B1

Trove Read A2 BMI Send A2 Trove Read B2 BMI Send B2

Trove Read A8 BMI Send A8 Trove Read B8 BMI Send B8

…Datafile:

time

Argonne National Laboratory PVFS: Design and Implementation 59

Flow Multiqueue Server Write

… A8A2A1 B8B2B1

Trove Write A1

BMI Recv A2 Trove Write A2

BMI Recv A3

BMI Recv A1 BMI Recv A4

BMI Recv A5

Trove Write A3

BMI Recv A6

Trove Write A4

Trove Write A5

BMI Recv B2

BMI Recv A7

BMI Recv A8

BMI Recv B1

Trove Write A6

Trove Write A7

Trove Write A8

BMI Recv B3

Trove Write B1

Datafile:

time

Argonne National Laboratory PVFS: Design and Implementation 60

BMI Component

BMI API

IB MXGM TCP

API for high-bandwidth, low-latency networks Different message types

– Unexpected (small, buffered)

– Expected (large, pre-posted receives) Follows same POST/TEST model

– BMI_send_post

– BMI_recv_post

– BMI_test_unexpected

– BMI_test Message Tags

– Matching

– Allows out-of-order delivery

Memory registration

– BMI_memalloc

Argonne National Laboratory PVFS: Design and Implementation

OrangeFS V2.9 and V3 Changes V2.9 – Distributed Directories

– Originally, all dirents stored on same server as directory metadata

– Now directories have an ordered set of objects holding dirents• Object stored on different servers• Extensible hashing used to locate which object has a dirent

V2.9 – Capabilities

– Originally all requests had an userid and groupid

– Now requests have a capability which is generated by a server• Encodes access rights• Signed by creating server

– Credentials used when capabilities are created V3 – Handles

– Originally 64 bit handles, statically allocated to each server

– Now 128 bit OID can be held by any server, multiple copies may exist

– 128 bit SID identifies which servers have a copy of the object V3 - Msgpairarray_sm

– Now single unified code for both simple messages and flow rendezvous

61

Argonne National Laboratory PVFS: Design and Implementation

Distributed Directories

62

DirEnt1

File1DirEnt2DirEnt3DirEnt4DirEnt5DirEnt6

DirEnt1DirEnt5

DirEnt3

DirEnt2

DirEnt6

DirEnt4

Server0

Server1

Server2

Server3

Extensible Hashing

State Management based on Giga+Improves access times for directories with a very large number of entries

Argonne National Laboratory PVFS: Design and Implementation

Capabilities

63

App

Credential

Capability

User

Verify

CredentialServer

Proxy Cert

Authenticate

Handle Metadata ServerAccess Rights Data

DataServerData

Argonne National Laboratory PVFS: Design and Implementation 64

PVFS Handle Reference Tree

Directory Dir Data

Metafile

foo

bar

baz

Metafile

Metadata Server 1

Directory

Metadata Server 2IO Server 1 IO Server 3IO Server 2

Dir Data

dh

dh