blob: 871b3a0b38772bd7b00a11e45084f3f51f7f1069 [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge">
<meta name="viewport" content="width=device-width, initial-scale=1">
<link rel="shortcut icon" href="/favicon.ico" type="image/x-icon">
<link rel="icon" href="/favicon.ico" type="image/x-icon">
<title>OCI/Squashfs Runtime</title>
<!-- Bootstrap core CSS -->
<link href="/assets/css/bootstrap.min.css" rel="stylesheet">
<!-- Bootstrap theme -->
<link href="/assets/css/bootstrap-theme.min.css" rel="stylesheet">
<!-- Custom styles for this template -->
<link rel="stylesheet" href="http://fortawesome.github.io/Font-Awesome/assets/font-awesome/css/font-awesome.css">
<link href="/css/style.css" rel="stylesheet">
<link href="/assets/css/owl.theme.css" rel="stylesheet">
<link href="/assets/css/owl.carousel.css" rel="stylesheet">
<script type="text/javascript" src="/assets/js/jquery.min.js"></script>
<script type="text/javascript" src="/assets/js/bootstrap.min.js"></script>
<script type="text/javascript" src="/assets/js/owl.carousel.min.js"></script>
<script type="text/javascript" src="/assets/js/storm.js"></script>
<!-- Just for debugging purposes. Don't actually copy these 2 lines! -->
<!--[if lt IE 9]><script src="../../assets/js/ie8-responsive-file-warning.js"></script><![endif]-->
<!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
<!--[if lt IE 9]>
<script src="https://oss.maxcdn.com/html5shiv/3.7.2/html5shiv.min.js"></script>
<script src="https://oss.maxcdn.com/respond/1.4.2/respond.min.js"></script>
<![endif]-->
</head>
<body>
<header>
<div class="container-fluid">
<div class="row">
<div class="col-md-5">
<a href="/index.html"><img src="/images/logo.png" class="logo" /></a>
</div>
<div class="col-md-5">
<h1>Version: 2.3.0</h1>
</div>
<div class="col-md-2">
<a href="/downloads.html" class="btn-std btn-block btn-download">Download</a>
</div>
</div>
</div>
</header>
<!--Header End-->
<!--Navigation Begin-->
<div class="navbar" role="banner">
<div class="container-fluid">
<div class="navbar-header">
<button class="navbar-toggle" type="button" data-toggle="collapse" data-target=".bs-navbar-collapse">
<span class="icon-bar"></span>
<span class="icon-bar"></span>
<span class="icon-bar"></span>
</button>
</div>
<nav class="collapse navbar-collapse bs-navbar-collapse" role="navigation">
<ul class="nav navbar-nav">
<li><a href="/index.html" id="home">Home</a></li>
<li><a href="/getting-help.html" id="getting-help">Getting Help</a></li>
<li><a href="/about/integrates.html" id="project-info">Project Information</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="documentation">Documentation <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/releases/2.3.0/index.html">2.3.0</a></li>
<li><a href="/releases/2.2.0/index.html">2.2.0</a></li>
<li><a href="/releases/2.1.0/index.html">2.1.0</a></li>
<li><a href="/releases/2.0.0/index.html">2.0.0</a></li>
<li><a href="/releases/1.2.3/index.html">1.2.3</a></li>
</ul>
</li>
<li><a href="/talksAndVideos.html">Talks and Slideshows</a></li>
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown" id="contribute">Community <b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="/contribute/Contributing-to-Storm.html">Contributing</a></li>
<li><a href="/contribute/People.html">People</a></li>
<li><a href="/contribute/BYLAWS.html">ByLaws</a></li>
</ul>
</li>
<li><a href="/2021/09/27/storm230-released.html" id="news">News</a></li>
</ul>
</nav>
</div>
</div>
<div class="container-fluid">
<h1 class="page-title">OCI/Squashfs Runtime</h1>
<div class="row">
<div class="col-md-12">
<!-- Documentation -->
<p class="post-meta"></p>
<div class="documentation-content"><h1 id="oci-squashfs-runtime-for-workers-running-in-containers">OCI/Squashfs Runtime for Workers Running in Containers</h1>
<p>OCI/Squashfs is a container runtime that allows topologies to run inside docker containers. However, unlike the existing
Docker runtime, the images are fetched from HDFS rather than from the Docker registry or requiring images to be pre-loaded
into Docker on each node. Docker does not need to be installed on the nodes in order for this runtime to work.</p>
<p>Note: This has only been tested on RHEL7.</p>
<h2 id="motivation">Motivation</h2>
<h4 id="docker-runtime-drawbacks">Docker runtime drawbacks</h4>
<p>Using the current Docker runtime (see <a href="Docker-support.md#Docker-Support">Docker-support.md</a> ) has some drawbacks:</p>
<h5 id="docker-daemons-dependency">Docker Daemons Dependency</h5>
<p>The Docker daemons <code>dockerd</code> and <code>containerd</code> must be running on the system in order for the Docker runtime to function.
And these daemons can get out of sync which could cause nontrivial issues to the containers.</p>
<h5 id="docker-registry-issues-at-scale">Docker Registry Issues at Scale</h5>
<p>Using the Docker runtime on a large scale Storm cluster can overwhelm the Docker registry. In practice this requires
admins to pre-load a Docker image on all the cluster nodes in a controlled fashion before a large job requesting
the image can run.</p>
<h5 id="image-costs-in-time-and-space">Image Costs in Time and Space</h5>
<p>Docker stores each image layer as a tar.gz archive. In order to use the layer, the compressed archive must be unpacked
into the node&#39;s filesystem. This can consume significant disk space, especially when the reliable image store location
capacity is relatively small. In addition, unpacking an image layer takes time, especially when the layer is large or
contains thousands of files. This additional time for unpacking delays container launch beyond the time needed to transfer
the layer data over the network.</p>
<h4 id="oci-squashfs-runtime-advantages">OCI/Squashfs Runtime advantages</h4>
<p>The OCI/Squashfs runtime avoids the drawback listed above in the following ways.</p>
<h5 id="no-docker-dependencies-on-the-node">No Docker dependencies on The Node</h5>
<p>Docker does not need to be installed on each node, nor is there a dependency on a daemon or service that needs to be started
by an admin before containers can be launched. All that is required to be present on each node is an OCI-compatible runtime like
<code>runc</code>.</p>
<h5 id="leverages-distributed-file-sytems-for-scale">Leverages Distributed File Sytems For Scale</h5>
<p>Image can be fetched via HDFS or other distributed file systems instead of the Docker registry. This prevents a large cluster from
overwhelming a Docker registry when a big topology causes all of the nodes to request an image at once. This also allows large clusters
to run topologies more dynamically, as images would not need to be pre-loaded by admins on each node to prevent a large Docker registry
image request storm.</p>
<h5 id="smaller-faster-images-on-the-node">Smaller, Faster images on The Node</h5>
<p>The new runtime handles layer localization directly, so layer formats other than tar archive can be supported. For example, each image layer
can be converted to squashfs images as part of copying the layers to HDFS. squashfs is a file system optimized for running directly on a
compressed image. With squashfs layers the layer data can remain compressed on the node saving disk space. Container launch after layer
localization is also faster, as the layers no longer need to be unpacked into a directory to become usable.</p>
<h2 id="prerequisite">Prerequisite</h2>
<p>First you need to use the<code>docker-to-squash.py</code> script to download docker images and configs, convert layers to squashfs files and put them to a directory in HDFS, for example</p>
<div class="highlight"><pre><code class="language-bash" data-lang="bash">python docker-to-squash.py pull-build-push-update <span class="nt">--hdfs-root</span> hdfs://hostname:port/containers <span class="se">\</span>
docker.xxx.com:4443/hadoop-user-images/storm/rhel7:20201202-232133,storm/rhel7:dev_current <span class="nt">--log</span> DEBUG <span class="nt">--bootstrap</span>
</code></pre></div>
<p>With this command, all the layers belonging to this image will be converted to squashfs files and be placed under <code>./layers</code> directory
under the directory specified by <code>--hdfs-root</code>;
the manifest of this image will be placed under <code>./manifests</code> directory with the name as the sha256 value of the manifest content;
the config of this image will be placed under <code>./config</code> directory with the name as the sha256 value of the config content;
the mapping from the image tag to the sha256 value of the manifest will be written to the &quot;./image-tag-to-manifest-file&quot;.</p>
<p>Note that <code>--hdfs-root</code> can be any directory on HDFS, as long as it matches with the <code>storm.oci.image.hdfs.toplevel.dir</code> config. </p>
<h5 id="example">Example</h5>
<p>For example, the directory structure is like this:</p>
<div class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nt">-bash-4</span>.2<span class="nv">$ </span>hdfs dfs <span class="nt">-ls</span> /containers/<span class="k">*</span>
Found 1 items
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 7877 2020-12-04 14:29 /containers/config/ef1ff2c7167a1a6cd01e106f51b84a4d400611ba971c53cbc28de7919515ca4e
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 160 2020-12-04 14:30 /containers/image-tag-to-hash
Found 7 items
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 84697088 2020-12-04 14:28 /containers/layers/152ee1d2cccea9dfe6393d2bdf9d077b67616b2b417b25eb74fc5ffaadcb96f5.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 545267712 2020-12-04 14:28 /containers/layers/18ee671016a1bf3ecab07395d93c2cbecd352d59c497a1551e2074d64e1098d9.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 12906496 2020-10-06 15:24 /containers/layers/1b73e9433ecca0a6bb152bd7525f2b7c233484d51c24f8a6ba483d5cfd3035dc.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 4096 2020-12-04 14:29 /containers/layers/344224962010c03c9ca1f11a9bff0dfcc296ac46d0a55e4ff30a0ad13b9817af.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 26091520 2020-10-06 15:22 /containers/layers/3692c3483ef6516fba685b316448e8aaf0fc10bb66818116edc8e5e6800076c7.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 4096 2020-12-04 14:29 /containers/layers/8710a3d72f75b45c48ab6b9b67eb6d77caea3dac91a0c30e0831f591cba4887e.sqsh
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 121122816 2020-10-06 15:23 /containers/layers/ea067172a7138f035d89a5c378db6d66c1581d98b0497b21f256e04c3d2b5303.sqsh
Found 1 items
<span class="nt">-r--r--r--</span> 3 hdfsqa hadoop 1793 2020-12-04 14:29 /containers/manifests/26fd443859325d5911f3be5c5e231dddca88ee0d526456c0c92dd794148d8585
</code></pre></div>
<p>The <code>image-tag-to-manifest-file</code>:
<code>bash
-bash-4.2$ hdfs dfs -cat /containers/image-tag-to-hash
storm/rhel7:dev_current:26fd443859325d5911f3be5c5e231dddca88ee0d526456c0c92dd794148d8585#docker.xxx.com:4443/hadoop-user-images/storm/rhel7:20201202-232133
</code></p>
<p>The manifest file <code>26fd443859325d5911f3be5c5e231dddca88ee0d526456c0c92dd794148d8585</code>:
<code>json
{
&quot;schemaVersion&quot;: 2,
&quot;mediaType&quot;: &quot;application/vnd.docker.distribution.manifest.v2+json&quot;,
&quot;config&quot;: {
&quot;mediaType&quot;: &quot;application/vnd.docker.container.image.v1+json&quot;,
&quot;size&quot;: 7877,
&quot;digest&quot;: &quot;sha256:ef1ff2c7167a1a6cd01e106f51b84a4d400611ba971c53cbc28de7919515ca4e&quot;
},
&quot;layers&quot;: [
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 26858854,
&quot;digest&quot;: &quot;sha256:3692c3483ef6516fba685b316448e8aaf0fc10bb66818116edc8e5e6800076c7&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 123300113,
&quot;digest&quot;: &quot;sha256:ea067172a7138f035d89a5c378db6d66c1581d98b0497b21f256e04c3d2b5303&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 12927624,
&quot;digest&quot;: &quot;sha256:1b73e9433ecca0a6bb152bd7525f2b7c233484d51c24f8a6ba483d5cfd3035dc&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 567401434,
&quot;digest&quot;: &quot;sha256:18ee671016a1bf3ecab07395d93c2cbecd352d59c497a1551e2074d64e1098d9&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 85748864,
&quot;digest&quot;: &quot;sha256:152ee1d2cccea9dfe6393d2bdf9d077b67616b2b417b25eb74fc5ffaadcb96f5&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 186,
&quot;digest&quot;: &quot;sha256:344224962010c03c9ca1f11a9bff0dfcc296ac46d0a55e4ff30a0ad13b9817af&quot;
},
{
&quot;mediaType&quot;: &quot;application/vnd.docker.image.rootfs.diff.tar.gzip&quot;,
&quot;size&quot;: 156,
&quot;digest&quot;: &quot;sha256:8710a3d72f75b45c48ab6b9b67eb6d77caea3dac91a0c30e0831f591cba4887e&quot;
}
]
}
</code></p>
<p>And the config file <code>ef1ff2c7167a1a6cd01e106f51b84a4d400611ba971c53cbc28de7919515ca4e</code> (some of the content is omitted):
<code>json
{
&quot;architecture&quot;: &quot;amd64&quot;,
&quot;config&quot;: {
&quot;Hostname&quot;: &quot;&quot;,
&quot;Domainname&quot;: &quot;&quot;,
&quot;User&quot;: &quot;root&quot;,
&quot;AttachStdin&quot;: false,
&quot;AttachStdout&quot;: false,
&quot;AttachStderr&quot;: false,
&quot;Tty&quot;: false,
&quot;OpenStdin&quot;: false,
&quot;StdinOnce&quot;: false,
&quot;Env&quot;: [
&quot;X_SCLS=rh-git218&quot;,
&quot;LD_LIBRARY_PATH=/opt/rh/httpd24/root/usr/lib64&quot;,
&quot;PATH=/opt/rh/rh-git218/root/usr/bin:/home/y/bin64:/home/y/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/home/y/share/yjava_jdk/java/bin&quot;,
&quot;PERL5LIB=/opt/rh/rh-git218/root/usr/share/perl5/vendor_perl&quot;,
&quot;LANG=en_US.UTF-8&quot;,
&quot;LANGUAGE=en_US:en&quot;,
&quot;LC_ALL=en_US.UTF-8&quot;,
&quot;JAVA_HOME=/home/y/share/yjava_jdk/java&quot;
],
&quot;Cmd&quot;: [
&quot;/bin/bash&quot;
],
&quot;Image&quot;: &quot;sha256:6977cd0735c96d14248e834f775373e40230c134b70f10163c05ce6c6c8873ca&quot;,
&quot;Volumes&quot;: null,
&quot;WorkingDir&quot;: &quot;&quot;,
&quot;Entrypoint&quot;: null,
&quot;OnBuild&quot;: null,
&quot;Labels&quot;: {
&quot;name&quot;: &quot;xxxxx&quot;
}
},
&quot;container&quot;: &quot;344ff1084dea3e0501a0d426e52c43cd589d6b29f33ab0915b7be8906b9aec41&quot;,
&quot;container_config&quot;: {
&quot;Hostname&quot;: &quot;344ff1084dea&quot;,
&quot;Domainname&quot;: &quot;&quot;,
&quot;User&quot;: &quot;root&quot;,
&quot;AttachStdin&quot;: false,
&quot;AttachStdout&quot;: false,
&quot;AttachStderr&quot;: false,
&quot;Tty&quot;: false,
&quot;OpenStdin&quot;: false,
&quot;StdinOnce&quot;: false,
&quot;Env&quot;: [
&quot;X_SCLS=rh-git218&quot;,
&quot;LD_LIBRARY_PATH=/opt/rh/httpd24/root/usr/lib64&quot;,
&quot;PATH=/opt/rh/rh-git218/root/usr/bin:/home/y/bin64:/home/y/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/home/y/share/yjava_jdk/java/bin&quot;,
&quot;PERL5LIB=/opt/rh/rh-git218/root/usr/share/perl5/vendor_perl&quot;,
&quot;LANG=en_US.UTF-8&quot;,
&quot;LANGUAGE=en_US:en&quot;,
&quot;LC_ALL=en_US.UTF-8&quot;,
&quot;JAVA_HOME=/home/y/share/yjava_jdk/java&quot;
],
&quot;Cmd&quot;: [
&quot;/bin/sh&quot;,
&quot;-c&quot;
],
&quot;Image&quot;: &quot;sha256:6977cd0735c96d14248e834f775373e40230c134b70f10163c05ce6c6c8873ca&quot;,
&quot;Volumes&quot;: null,
&quot;WorkingDir&quot;: &quot;&quot;,
&quot;Entrypoint&quot;: null,
&quot;OnBuild&quot;: null,
&quot;Labels&quot;: {
&quot;name&quot;: &quot;xxxxx&quot;
}
},
&quot;created&quot;: &quot;2020-12-02T23:25:47.354704574Z&quot;,
&quot;docker_version&quot;: &quot;19.03.8&quot;,
&quot;history&quot;: [
{
&quot;created&quot;: &quot;2020-02-18T21:43:36.934503462Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh&quot;
},
{
&quot;created&quot;: &quot;2020-02-18T21:45:05.729764427Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh&quot;
},
{
&quot;created&quot;: &quot;2020-02-18T21:46:36.638896031Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh&quot;
},
{
&quot;created&quot;: &quot;2020-12-02T23:21:54.595662813Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c #(nop) USER root&quot;,
&quot;empty_layer&quot;: true
},
{
&quot;created&quot;: &quot;2020-12-02T23:25:45.822235539Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c /opt/python/bin/pip3.6 install --no-cache-dir numpy scipy pandas requests setuptools scikit-learn matplotlib&quot;
},
{
&quot;created&quot;: &quot;2020-12-02T23:25:46.708884538Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c #(nop) ENV JAVA_HOME=/home/y/share/yjava_jdk/java&quot;,
&quot;empty_layer&quot;: true
},
{
&quot;created&quot;: &quot;2020-12-02T23:25:46.770226108Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c #(nop) ENV PATH=/opt/rh/rh-git218/root/usr/bin:/home/y/bin64:/home/y/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/home/y/share/yjava_jdk/java/bin&quot;,
&quot;empty_layer&quot;: true
},
{
&quot;created&quot;: &quot;2020-12-02T23:25:46.837263533Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c #(nop) COPY file:33283617fbd796b25e53eaf4d26012eea1f610ff9acc0706f11281e86be440dc in /etc/krb5.conf &quot;
},
{
&quot;created&quot;: &quot;2020-12-02T23:25:47.237515768Z&quot;,
&quot;created_by&quot;: &quot;/bin/sh -c echo &#39;7.7.4&#39; \u003e /etc/hadoop-dockerfile-version&quot;
}
],
&quot;os&quot;: &quot;linux&quot;,
&quot;rootfs&quot;: {
&quot;type&quot;: &quot;layers&quot;,
&quot;diff_ids&quot;: [
&quot;sha256:9f627fdb0292afbe5e2eb96edc1b3a5d3a8f468e3acf1d29f1509509285c7341&quot;,
&quot;sha256:83d2667f9458eaf719588a96bb63f2520bd377d29d52f6dbd4ff13c819c08037&quot;,
&quot;sha256:fcba5f49eef4f3d77d3e73e499a1a4e1914b3f20d903625d27c0aa3ab82f41a3&quot;,
&quot;sha256:3bd4567d0726f5d6560b548bc0c0400e868f6a27067887a36edd7e8ceafff96c&quot;,
&quot;sha256:ad56900a1f10e6ef96f17c7e8019384540ab1b34ccce6bda06675473b08d787e&quot;,
&quot;sha256:ac0a645609f957ab9c4a8a62f8646e99f09a74ada54ed2eaca204c6e183c9ae8&quot;,
&quot;sha256:9bf10102fc145156f4081c2cacdbadab5816dce4f88eb02881ab739239d316e6&quot;
]
}
}
</code></p>
<p>Note: To use the <code>docker-to-squash.py</code>, you need to install <a href="https://github.com/containers/skopeo">skopeo</a>, <a href="https://stedolan.github.io/jq/">jq</a> and squashfs-tools.</p>
<h2 id="configurations">Configurations</h2>
<p>Then you need to set up storm with the following configs:</p>
<table><thead>
<tr>
<th>Setting</th>
<th>Description</th>
</tr>
</thead><tbody>
<tr>
<td><code>storm.resource.isolation.plugin.enable</code></td>
<td>set to <code>true</code> to enable isolation plugin. <code>storm.resource.isolation.plugin</code> determines which plugin to use. If this is set to <code>false</code>, <code>org.apache.storm.container.DefaultResourceIsolationManager</code> will be used.</td>
</tr>
<tr>
<td><code>storm.resource.isolation.plugin</code></td>
<td>set to <code>&quot;org.apache.storm.container.oci.RuncLibContainerManager&quot;</code> to enable OCI/Squash runtime support</td>
</tr>
<tr>
<td><code>storm.oci.allowed.images</code></td>
<td>An allowlist of docker images that can be used. Users can only choose a docker image from the list.</td>
</tr>
<tr>
<td><code>storm.oci.image</code></td>
<td>The default docker image to be used if user doesn&#39;t specify which image to use. And it must belong to the <code>storm.oci.allowed.images</code></td>
</tr>
<tr>
<td><code>topology.oci.image</code></td>
<td>Topologies can specify which image to use. It must belong to the <code>storm.oci.allowed.images</code></td>
</tr>
<tr>
<td><code>storm.oci.cgroup.root</code></td>
<td>The root path of cgroup for docker to use. On RHEL7, it should be &quot;/sys/fs/cgroup&quot;.</td>
</tr>
<tr>
<td><code>storm.oci.cgroup.parent</code></td>
<td>--cgroup-parent config for docker command. It must follow the constraints of docker commands. The path will be made as absolute path if it&#39;s a relative path because we saw some weird bugs ((the cgroup memory directory disappears after a while) when a relative path is used.</td>
</tr>
<tr>
<td><code>storm.oci.readonly.bindmounts</code></td>
<td>A list of read only bind mounted directories.</td>
</tr>
<tr>
<td><code>storm.oci.readwrite.bindmounts</code></td>
<td>A list of read-write bind mounted directories.</td>
</tr>
<tr>
<td><code>storm.oci.nscd.dir</code></td>
<td>The directory of nscd (name service cache daemon), e.g. &quot;/var/run/nscd/&quot;. nscd must be running so that profiling can work properly.</td>
</tr>
<tr>
<td><code>storm.oci.seccomp.profile</code></td>
<td>Specify the seccomp Json file to be used as a seccomp filter</td>
</tr>
<tr>
<td><code>supervisor.worker.launcher</code></td>
<td>Full path to the worker-launcher executable.</td>
</tr>
<tr>
<td><code>storm.oci.image.hdfs.toplevel.dir</code></td>
<td>The HDFS location under which the oci image manifests, layers and configs directories exist.</td>
</tr>
<tr>
<td><code>storm.oci.image.tag.to.manifest.plugin</code></td>
<td>The plugin to be used to get the image-tag to manifest mappings.</td>
</tr>
<tr>
<td><code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.hdfs.hash.file</code></td>
<td>The hdfs location of the image-tag to manifest mapping file. If <code>org.apache.storm.container.oci.LocalOrHdfsImageTagToManifestPlugin</code> is used as <code>storm.oci.image.tag.to.manifest.plugin</code>, either <code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.hdfs.hash.file</code> or <code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.local.hash.file</code> needs to be configured.</td>
</tr>
<tr>
<td><code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.local.hash.file</code></td>
<td>The local file system location where the image-tag to manifest mapping file exists. If <code>org.apache.storm.container.oci.LocalOrHdfsImageTagToManifestPlugin</code> is used as <code>storm.oci.image.tag.to.manifest.plugin</code>, either <code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.hdfs.hash.file</code> or <code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.local.hash.file</code> needs to be configured.</td>
</tr>
<tr>
<td><code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.cache.refresh.interval.secs</code></td>
<td>The interval in seconds between refreshing the image-tag to manifest mapping cache, used by <code>org.apache.storm.container.oci.LocalOrHdfsImageTagToManifestPlugin</code>.</td>
</tr>
<tr>
<td><code>storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.num.manifests.to.cache</code></td>
<td>The number of manifests to cache, used by <code>org.apache.storm.container.oci.LocalOrHdfsImageTagToManifestPlugin</code>.</td>
</tr>
<tr>
<td><code>storm.oci.manifest.to.resources.plugin</code></td>
<td>The plugin to be used to get oci resource according to the manifest.</td>
</tr>
<tr>
<td><code>storm.oci.resources.localizer</code></td>
<td>The plugin to use for oci resources localization.</td>
</tr>
<tr>
<td><code>storm.oci.resources.local.dir</code></td>
<td>The local directory for localized oci resources.</td>
</tr>
</tbody></table>
<p>For example,
```bash
storm.resource.isolation.plugin: &quot;org.apache.storm.container.oci.RuncLibContainerManager&quot;</p>
<p>storm.oci.allowed.images:
- &quot;storm/rhel7:dev_current&quot;
- &quot;storm/rhel7:dev_previous&quot;
- &quot;storm/rhel7:dev_test&quot;
storm.oci.image: &quot;storm/rhel7:dev_current&quot;</p>
<p>storm.oci.cgroup.parent: &quot;/storm&quot;
storm.oci.cgroup.root: &quot;/sys/fs/cgroup&quot;
storm.oci.image.hdfs.toplevel.dir: &quot;hdfs://host:port/containers/&quot;
storm.oci.image.tag.to.manifest.plugin: &quot;org.apache.storm.container.oci.LocalOrHdfsImageTagToManifestPlugin&quot;
storm.oci.local.or.hdfs.image.tag.to.manifest.plugin.hdfs.hash.file: &quot;hdfs://host:port/containers/image-tag-to-hash&quot;
storm.oci.manifest.to.resources.plugin: &quot;org.apache.storm.container.oci.HdfsManifestToResourcesPlugin&quot;
storm.oci.readonly.bindmounts:
- &quot;/home/y/lib64/storm&quot;
- &quot;/etc/krb5.conf&quot;</p>
<p>storm.oci.resources.localizer: &quot;org.apache.storm.container.oci.HdfsOciResourcesLocalizer&quot;
storm.oci.seccomp.profile: &quot;/home/y/conf/storm/seccomp.json&quot;
```</p>
<p>To use built-in plugins from <code>external/storm-hdfs-oci</code>, you need to build <code>external/storm-hdfs-oci</code> and copy <code>storm-hdfs-oci.jar</code> and its dependencies to the <code>extlib-daemon</code> directory.</p>
<p>Additionally, if you want to access to secure hdfs, you also need to set the following configs.<br>
<code>
storm.hdfs.login.keytab
storm.hdfs.login.principal
</code></p>
<p>For example,
<code>
storm.hdfs.login.keytab: /etc/keytab
storm.hdfs.login.principal: primary/instance@REALM
</code></p>
<h2 id="implementation">Implementation</h2>
<h5 id="launch-a-container">Launch a container</h5>
<p>The supervisor calls RuncLibContainerManager to launch the container and the worker inside the container. It will first call the <code>storm.oci.image.tag.to.manifest.plugin</code>
to fetch the mapping of image tag to manifest. Then it calls <code>storm.oci.manifest.to.resources.plugin</code> to get the list of resources to be downloaded and invokes
<code>storm.oci.resources.localizer</code> to download the config of the image and the layers of the image to a local directory. It then composes a <code>oci-config.json</code> (see example in Appendix) and
invokes worker-launcher to launch the container.</p>
<p>The worker-launcher parses the <code>oci-config.json</code> file and do some necessary initialization and set up. It then creates /run/worker-launcher/layers/xxx/mnt directories
and associate them with loopback devices, for example:</p>
<div class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nt">-bash-4</span>.2<span class="nv">$ </span><span class="nb">cat</span> /proc/mounts
...
/dev/loop3 /run/worker-launcher/layers/f7452c2657900c53da1a4f7e430485a267b89c7717466ee61ffefba85f690226/mnt squashfs ro,relatime 0 0
/dev/loop4 /run/worker-launcher/layers/8156da43228752c7364b71dabba6aef6bd1cc081e9ea59cf92ea0f79fd8a50b6/mnt squashfs ro,relatime 0 0
/dev/loop5 /run/worker-launcher/layers/c7c9b1d6df043edf307c49d75c7d2bc3df72f8dcaf7d17b733c97022387902e6/mnt squashfs ro,relatime 0 0
/dev/loop6 /run/worker-launcher/layers/f0d08d5707855b02def8ac622a6c60203b380e31c6c237e5b691f5856594a3e7/mnt squashfs ro,relatime 0 0
/dev/loop11 /run/worker-launcher/layers/34b0bc9c446a9be565fb50b04db1e9d1c1c4d14a22a885a7aba6981748b6635e/mnt squashfs ro,relatime 0 0
/dev/loop12 /run/worker-launcher/layers/0ba001c025aa172a7d630914c75c1772228606f622e2c9d46a8fedf10774623e/mnt squashfs ro,relatime 0 0
/dev/loop13 /run/worker-launcher/layers/a5e4e615565081e04eaf4c5ab5b20d37de271db704fc781c7b1e07c5dcdf96e5/mnt squashfs ro,relatime 0 0
...
</code></pre></div>
<p>Then it mounts the layers, for example:
<code>bash
-bash-4.2$ mount
...
/home/y/var/storm/supervisor/oci-resources/layers/3692c3483ef6516fba685b316448e8aaf0fc10bb66818116edc8e5e6800076c7.sqsh on /run/worker-launcher/layers/f7452c2657900c53da1a4f7e430485a267b89c7717466ee61ffefba85f690226/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/ea067172a7138f035d89a5c378db6d66c1581d98b0497b21f256e04c3d2b5303.sqsh on /run/worker-launcher/layers/8156da43228752c7364b71dabba6aef6bd1cc081e9ea59cf92ea0f79fd8a50b6/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/1b73e9433ecca0a6bb152bd7525f2b7c233484d51c24f8a6ba483d5cfd3035dc.sqsh on /run/worker-launcher/layers/c7c9b1d6df043edf307c49d75c7d2bc3df72f8dcaf7d17b733c97022387902e6/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/18ee671016a1bf3ecab07395d93c2cbecd352d59c497a1551e2074d64e1098d9.sqsh on /run/worker-launcher/layers/f0d08d5707855b02def8ac622a6c60203b380e31c6c237e5b691f5856594a3e7/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/152ee1d2cccea9dfe6393d2bdf9d077b67616b2b417b25eb74fc5ffaadcb96f5.sqsh on /run/worker-launcher/layers/34b0bc9c446a9be565fb50b04db1e9d1c1c4d14a22a885a7aba6981748b6635e/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/344224962010c03c9ca1f11a9bff0dfcc296ac46d0a55e4ff30a0ad13b9817af.sqsh on /run/worker-launcher/layers/0ba001c025aa172a7d630914c75c1772228606f622e2c9d46a8fedf10774623e/mnt type squashfs (ro,relatime)
/home/y/var/storm/supervisor/oci-resources/layers/8710a3d72f75b45c48ab6b9b67eb6d77caea3dac91a0c30e0831f591cba4887e.sqsh on /run/worker-launcher/layers/a5e4e615565081e04eaf4c5ab5b20d37de271db704fc781c7b1e07c5dcdf96e5/mnt type squashfs (ro,relatime)
...
</code></p>
<p>It creates the rootfs and mount the overlay filesystem (with lowerdir,upperdir,workdir) for the worker with the command
<code>bash
mount -t overlay overlay -o lowerdir=/lower1:/lower2:/lower3,upperdir=/upper,workdir=/work /merged
</code></p>
<div class="highlight"><pre><code class="language-bash" data-lang="bash"><span class="nt">-bash-4</span>.2<span class="nv">$ </span>mount
...
overlay on /run/worker-launcher/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/rootfs <span class="nb">type </span>overlay <span class="o">(</span>rw,relatime,lowerdir<span class="o">=</span>/run/worker-launcher/layers/a5e4e615565081e04eaf4c5ab5b20d37de271db704fc781c7b1e07c5dcdf96e5/mnt:/run/worker-launcher/layers/0ba001c025aa172a7d630914c75c1772228606f622e2c9d46a8fedf10774623e/mnt:/run/worker-launcher/layers/34b0bc9c446a9be565fb50b04db1e9d1c1c4d14a22a885a7aba6981748b6635e/mnt:/run/worker-launcher/layers/f0d08d5707855b02def8ac622a6c60203b380e31c6c237e5b691f5856594a3e7/mnt:/run/worker-launcher/layers/c7c9b1d6df043edf307c49d75c7d2bc3df72f8dcaf7d17b733c97022387902e6/mnt:/run/worker-launcher/layers/8156da43228752c7364b71dabba6aef6bd1cc081e9ea59cf92ea0f79fd8a50b6/mnt:/run/worker-launcher/layers/f7452c2657900c53da1a4f7e430485a267b89c7717466ee61ffefba85f690226/mnt,upperdir<span class="o">=</span>/run/worker-launcher/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/upper,workdir<span class="o">=</span>/run/worker-launcher/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/work<span class="o">)</span>
...
</code></pre></div>
<p>It then produce a <code>config.json</code> (see example at Appendix) under <code>/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb</code> directory and launch the container with
the command
<code>bash
/usr/bin/runc run -d \
--pid-file /home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/artifacts/container-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb.pid \
-b /home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb \
6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb
</code></p>
<h5 id="kill-a-container">Kill a container</h5>
<p>To kill a container, <code>RuncLibContainerManager</code> sends the <code>SIGTERM</code> or <code>SIGKILL</code> signal to the container process. It then invokes worker-launcher to to umount the mounts and clean up the directories.
The worker-launcher will invoke <code>runc delete container-id</code> to delete the container at the end.</p>
<h2 id="profile-the-processes-inside-the-container">Profile the processes inside the container</h2>
<p>If you have sudo permission, you can also run <code>sudo nsenter --target &lt;container-pid&gt; --pid --mount --setuid &lt;uid&gt; --setgid &lt;gid&gt;</code> to enter the container.
Then you can run <code>jstack</code>, <code>jmap</code> etc inside the container. <code>&lt;container-pid&gt;</code> is the pid of the container process on the host.
<code>&lt;container-pid&gt;</code> can be obtained by running <code>runc list</code> command.</p>
<h2 id="seccomp-security-profiles">Seccomp security profiles</h2>
<p>You can set <code>storm.oci.seccomp.profile</code> to restrict the actions available within the container. If it&#39;s not set, the container runs without
restrictions. You can use <code>conf/seccomp.json.example</code> provided or you can specify our own <code>seccomp.json</code> file. </p>
<h2 id="appendix">Appendix</h2>
<h5 id="example-oci-config-json-file">Example oci-config.json file</h5>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"version"</span><span class="p">:</span><span class="w"> </span><span class="s2">"0.1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"username"</span><span class="p">:</span><span class="w"> </span><span class="s2">"username1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"containerId"</span><span class="p">:</span><span class="w"> </span><span class="s2">"6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pidFile"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/artifacts/container-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb.pid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"containerScriptPath"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/storm-worker-script.sh"</span><span class="p">,</span><span class="w">
</span><span class="s2">"layers"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/3692c3483ef6516fba685b316448e8aaf0fc10bb66818116edc8e5e6800076c7.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/ea067172a7138f035d89a5c378db6d66c1581d98b0497b21f256e04c3d2b5303.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/1b73e9433ecca0a6bb152bd7525f2b7c233484d51c24f8a6ba483d5cfd3035dc.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/18ee671016a1bf3ecab07395d93c2cbecd352d59c497a1551e2074d64e1098d9.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/152ee1d2cccea9dfe6393d2bdf9d077b67616b2b417b25eb74fc5ffaadcb96f5.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/344224962010c03c9ca1f11a9bff0dfcc296ac46d0a55e4ff30a0ad13b9817af.sqsh"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"mediaType"</span><span class="p">:</span><span class="w"> </span><span class="s2">"application/vnd.squashfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/oci-resources/layers/8710a3d72f75b45c48ab6b9b67eb6d77caea3dac91a0c30e0831f591cba4887e.sqsh"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"reapLayerKeepCount"</span><span class="p">:</span><span class="w"> </span><span class="mi">100</span><span class="p">,</span><span class="w">
</span><span class="s2">"ociRuntimeConfig"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"mounts"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/lib64/storm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/lib64/storm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/krb5.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/krb5.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/resolv.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/resolv.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hostname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hostname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hosts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hosts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/var/run/nscd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/var/run/nscd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys/fs/cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys/fs/cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-artifacts/wc1-2-1608581491/6703"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-artifacts/wc1-2-1608581491/6703"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-users/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-users/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/shared_by_topology"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/shared_by_topology"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/tmp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/tmp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"process"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"cwd"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"env"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"X_SCLS=rh-git218"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LD_LIBRARY_PATH=/opt/rh/httpd24/root/usr/lib64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"PATH=/opt/rh/rh-git218/root/usr/bin:/home/y/bin64:/home/y/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/home/y/share/yjava_jdk/java/bin"</span><span class="p">,</span><span class="w">
</span><span class="s2">"PERL5LIB=/opt/rh/rh-git218/root/usr/share/perl5/vendor_perl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LANG=en_US.UTF-8"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LANGUAGE=en_US:en"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LC_ALL=en_US.UTF-8"</span><span class="p">,</span><span class="w">
</span><span class="s2">"JAVA_HOME=/home/y/share/yjava_jdk/java"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LD_LIBRARY_PATH=/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/resources/Linux-amd64:/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/resources:/home/y/lib64:/usr/local/lib64:/usr/lib64:/lib64:"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"bash"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/storm-worker-script.sh"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"linux"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"cgroupsPath"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/storm/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"resources"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"cpu"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"quota"</span><span class="p">:</span><span class="w"> </span><span class="mi">140000</span><span class="p">,</span><span class="w">
</span><span class="s2">"period"</span><span class="p">:</span><span class="w"> </span><span class="mi">100000</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"seccomp"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"defaultAction"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ERRNO"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syscalls"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"accept"</span><span class="p">,</span><span class="w">
</span><span class="s2">"accept4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"access"</span><span class="p">,</span><span class="w">
</span><span class="s2">"alarm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"alarm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"brk"</span><span class="p">,</span><span class="w">
</span><span class="s2">"capget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"capset"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chmod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_getres"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_nanosleep"</span><span class="p">,</span><span class="w">
</span><span class="s2">"close"</span><span class="p">,</span><span class="w">
</span><span class="s2">"connect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"copy_file_range"</span><span class="p">,</span><span class="w">
</span><span class="s2">"creat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_create1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_ctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_ctl_old"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_pwait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_wait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_wait_old"</span><span class="p">,</span><span class="w">
</span><span class="s2">"eventfd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"eventfd2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"execve"</span><span class="p">,</span><span class="w">
</span><span class="s2">"execveat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"exit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"exit_group"</span><span class="p">,</span><span class="w">
</span><span class="s2">"faccessat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fadvise64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fadvise64_64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fallocate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fanotify_mark"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchmod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchmodat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchownat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fcntl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fcntl64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fdatasync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fgetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"flistxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"flock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fork"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fremovexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fsetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatfs64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fsync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ftruncate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ftruncate64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"futex"</span><span class="p">,</span><span class="w">
</span><span class="s2">"futimesat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getcpu"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getcwd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getdents"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getdents64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getegid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getegid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"geteuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"geteuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgroups"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgroups32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getitimer"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpeername"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpgrp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getppid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpriority"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrandom"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"get_robust_list"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrusage"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsockname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsockopt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"get_thread_area"</span><span class="p">,</span><span class="w">
</span><span class="s2">"gettid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"gettimeofday"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_add_watch"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_init"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_init1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_rm_watch"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_cancel"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_destroy"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_getevents"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioprio_get"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioprio_set"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_setup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_submit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ipc"</span><span class="p">,</span><span class="w">
</span><span class="s2">"kill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lchown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lchown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lgetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"link"</span><span class="p">,</span><span class="w">
</span><span class="s2">"linkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"listen"</span><span class="p">,</span><span class="w">
</span><span class="s2">"listxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"llistxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"_llseek"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lremovexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lseek"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lsetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lstat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lstat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"madvise"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"memfd_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mincore"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mkdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mkdirat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mknod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mknodat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlock2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlockall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mmap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mmap2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mprotect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_getsetattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_notify"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_open"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_timedreceive"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_timedsend"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_unlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mremap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgrcv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgsnd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munlock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munlockall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munmap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nanosleep"</span><span class="p">,</span><span class="w">
</span><span class="s2">"newfstatat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"_newselect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"open"</span><span class="p">,</span><span class="w">
</span><span class="s2">"openat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pause"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pipe"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pipe2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"poll"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ppoll"</span><span class="p">,</span><span class="w">
</span><span class="s2">"prctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pread64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"preadv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"prlimit64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pselect6"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pwrite64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pwritev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"read"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readahead"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvfrom"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvmmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"remap_file_pages"</span><span class="p">,</span><span class="w">
</span><span class="s2">"removexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rename"</span><span class="p">,</span><span class="w">
</span><span class="s2">"renameat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"renameat2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"restart_syscall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rmdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigaction"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigpending"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigprocmask"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigqueueinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigreturn"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigsuspend"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigtimedwait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_tgsigqueueinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getaffinity"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getparam"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_get_priority_max"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_get_priority_min"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getscheduler"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_rr_get_interval"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setaffinity"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setparam"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setscheduler"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_yield"</span><span class="p">,</span><span class="w">
</span><span class="s2">"seccomp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"select"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semop"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semtimedop"</span><span class="p">,</span><span class="w">
</span><span class="s2">"send"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendfile"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendfile64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendmmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendto"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgroups"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgroups32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setitimer"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setpgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setpriority"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setregid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setregid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setreuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setreuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_robust_list"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setsid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setsockopt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_thread_area"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_tid_address"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmdt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shutdown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sigaltstack"</span><span class="p">,</span><span class="w">
</span><span class="s2">"signalfd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"signalfd4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sigreturn"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socket"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socketcall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socketpair"</span><span class="p">,</span><span class="w">
</span><span class="s2">"splice"</span><span class="p">,</span><span class="w">
</span><span class="s2">"stat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"stat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"statfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"statfs64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"symlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"symlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sync_file_range"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syncfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sysinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syslog"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tee"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tgkill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_delete"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_settime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_getoverrun"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_settime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"times"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tkill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"truncate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"truncate64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ugetrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"umask"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utimensat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utimes"</span><span class="p">,</span><span class="w">
</span><span class="s2">"vfork"</span><span class="p">,</span><span class="w">
</span><span class="s2">"vmsplice"</span><span class="p">,</span><span class="w">
</span><span class="s2">"wait4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"waitid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"waitpid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"write"</span><span class="p">,</span><span class="w">
</span><span class="s2">"writev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mount"</span><span class="p">,</span><span class="w">
</span><span class="s2">"umount2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"reboot"</span><span class="p">,</span><span class="w">
</span><span class="s2">"name_to_handle_at"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unshare"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">8</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">4294967295</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"arch_prctl"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"modify_ldt"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"clone"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">2080505856</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_MASKED_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h5 id="example-config-json-file">Example config.json file</h5>
<div class="highlight"><pre><code class="language-json" data-lang="json"><span class="p">{</span><span class="w">
</span><span class="s2">"ociVersion"</span><span class="p">:</span><span class="w"> </span><span class="s2">"1.0.0"</span><span class="p">,</span><span class="w">
</span><span class="s2">"hostname"</span><span class="p">:</span><span class="w"> </span><span class="s2">"hostname1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"root"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"path"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/run/worker-launcher/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/rootfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readonly"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"process"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"bash"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/storm-worker-script.sh"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"cwd"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"env"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"X_SCLS=rh-git218"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LD_LIBRARY_PATH=/opt/rh/httpd24/root/usr/lib64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"PATH=/opt/rh/rh-git218/root/usr/bin:/home/y/bin64:/home/y/bin:/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/home/y/share/yjava_jdk/java/bin"</span><span class="p">,</span><span class="w">
</span><span class="s2">"PERL5LIB=/opt/rh/rh-git218/root/usr/share/perl5/vendor_perl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LANG=en_US.UTF-8"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LANGUAGE=en_US:en"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LC_ALL=en_US.UTF-8"</span><span class="p">,</span><span class="w">
</span><span class="s2">"JAVA_HOME=/home/y/share/yjava_jdk/java"</span><span class="p">,</span><span class="w">
</span><span class="s2">"LD_LIBRARY_PATH=/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/resources/Linux-amd64:/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/resources:/home/y/lib64:/usr/local/lib64:/usr/lib64:/lib64:"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"noNewPrivileges"</span><span class="p">:</span><span class="w"> </span><span class="kc">true</span><span class="p">,</span><span class="w">
</span><span class="s2">"user"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"uid"</span><span class="p">:</span><span class="w"> </span><span class="mi">31315</span><span class="p">,</span><span class="w">
</span><span class="s2">"gid"</span><span class="p">:</span><span class="w"> </span><span class="mi">100</span><span class="p">,</span><span class="w">
</span><span class="s2">"additionalGids"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="mi">5548</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"mounts"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"proc"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/proc"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"proc"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"tmpfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/dev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"tmpfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"strictatime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mode=755"</span><span class="p">,</span><span class="w">
</span><span class="s2">"size=65536k"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"devpts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/dev/pts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"devpts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"noexec"</span><span class="p">,</span><span class="w">
</span><span class="s2">"newinstance"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ptmxmode=0666"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mode=0620"</span><span class="p">,</span><span class="w">
</span><span class="s2">"gid=5"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"shm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/dev/shm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"tmpfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"noexec"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nodev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mode=1777"</span><span class="p">,</span><span class="w">
</span><span class="s2">"size=65536k"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"mqueue"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/dev/mqueue"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"mqueue"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"noexec"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nodev"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"sysfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"sysfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"noexec"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nodev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ro"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys/fs/cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"nosuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"noexec"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nodev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"relatime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ro"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/lib64/storm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/lib64/storm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/krb5.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/krb5.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/resolv.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/resolv.conf"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hostname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hostname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hosts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/etc/hosts"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/var/run/nscd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/var/run/nscd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys/fs/cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/sys/fs/cgroup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"ro"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-artifacts/wc1-2-1608581491/6703"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-artifacts/wc1-2-1608581491/6703"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-users/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers-users/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/shared_by_topology"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/supervisor/stormdist/wc1-2-1608581491/shared_by_topology"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"destination"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/tmp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"source"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/home/y/var/storm/workers/1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb/tmp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"options"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"rw"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rprivate"</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"linux"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"cgroupsPath"</span><span class="p">:</span><span class="w"> </span><span class="s2">"/storm/6703-1a23ca4b-6062-4d08-8ac3-b09e7d35e7cb"</span><span class="p">,</span><span class="w">
</span><span class="s2">"resources"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"devices"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"access"</span><span class="p">:</span><span class="w"> </span><span class="s2">"rwm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"allow"</span><span class="p">:</span><span class="w"> </span><span class="kc">false</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"cpu"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"quota"</span><span class="p">:</span><span class="w"> </span><span class="mi">140000</span><span class="p">,</span><span class="w">
</span><span class="s2">"period"</span><span class="p">:</span><span class="w"> </span><span class="mi">100000</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="s2">"namespaces"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"pid"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"ipc"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"uts"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"type"</span><span class="p">:</span><span class="w"> </span><span class="s2">"mount"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"maskedPaths"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"/proc/kcore"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/latency_stats"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/timer_list"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/timer_stats"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/sched_debug"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/scsi"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/sys/firmware"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"readonlyPaths"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"/proc/asound"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/bus"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/fs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/irq"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/sys"</span><span class="p">,</span><span class="w">
</span><span class="s2">"/proc/sysrq-trigger"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"seccomp"</span><span class="p">:</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="s2">"defaultAction"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ERRNO"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syscalls"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"accept"</span><span class="p">,</span><span class="w">
</span><span class="s2">"accept4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"access"</span><span class="p">,</span><span class="w">
</span><span class="s2">"alarm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"alarm"</span><span class="p">,</span><span class="w">
</span><span class="s2">"bind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"brk"</span><span class="p">,</span><span class="w">
</span><span class="s2">"capget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"capset"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chmod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"chown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_getres"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"clock_nanosleep"</span><span class="p">,</span><span class="w">
</span><span class="s2">"close"</span><span class="p">,</span><span class="w">
</span><span class="s2">"connect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"copy_file_range"</span><span class="p">,</span><span class="w">
</span><span class="s2">"creat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"dup3"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_create1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_ctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_ctl_old"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_pwait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_wait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"epoll_wait_old"</span><span class="p">,</span><span class="w">
</span><span class="s2">"eventfd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"eventfd2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"execve"</span><span class="p">,</span><span class="w">
</span><span class="s2">"execveat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"exit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"exit_group"</span><span class="p">,</span><span class="w">
</span><span class="s2">"faccessat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fadvise64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fadvise64_64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fallocate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fanotify_mark"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchmod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchmodat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fchownat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fcntl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fcntl64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fdatasync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fgetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"flistxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"flock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fork"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fremovexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fsetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fstatfs64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"fsync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ftruncate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ftruncate64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"futex"</span><span class="p">,</span><span class="w">
</span><span class="s2">"futimesat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getcpu"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getcwd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getdents"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getdents64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getegid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getegid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"geteuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"geteuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgroups"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getgroups32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getitimer"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpeername"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpgrp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getppid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getpriority"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrandom"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getresuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"get_robust_list"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getrusage"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsockname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getsockopt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"get_thread_area"</span><span class="p">,</span><span class="w">
</span><span class="s2">"gettid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"gettimeofday"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"getxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_add_watch"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_init"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_init1"</span><span class="p">,</span><span class="w">
</span><span class="s2">"inotify_rm_watch"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_cancel"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_destroy"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_getevents"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioprio_get"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ioprio_set"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_setup"</span><span class="p">,</span><span class="w">
</span><span class="s2">"io_submit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ipc"</span><span class="p">,</span><span class="w">
</span><span class="s2">"kill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lchown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lchown32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lgetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"link"</span><span class="p">,</span><span class="w">
</span><span class="s2">"linkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"listen"</span><span class="p">,</span><span class="w">
</span><span class="s2">"listxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"llistxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"_llseek"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lremovexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lseek"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lsetxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lstat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"lstat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"madvise"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mbind"</span><span class="p">,</span><span class="w">
</span><span class="s2">"memfd_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mincore"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mkdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mkdirat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mknod"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mknodat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlock2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mlockall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mmap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mmap2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mprotect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_getsetattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_notify"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_open"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_timedreceive"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_timedsend"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mq_unlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mremap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgrcv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msgsnd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"msync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munlock"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munlockall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"munmap"</span><span class="p">,</span><span class="w">
</span><span class="s2">"nanosleep"</span><span class="p">,</span><span class="w">
</span><span class="s2">"newfstatat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"_newselect"</span><span class="p">,</span><span class="w">
</span><span class="s2">"open"</span><span class="p">,</span><span class="w">
</span><span class="s2">"openat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pause"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pipe"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pipe2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"poll"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ppoll"</span><span class="p">,</span><span class="w">
</span><span class="s2">"prctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pread64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"preadv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"prlimit64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pselect6"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pwrite64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"pwritev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"read"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readahead"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"readv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recv"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvfrom"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvmmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"recvmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"remap_file_pages"</span><span class="p">,</span><span class="w">
</span><span class="s2">"removexattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rename"</span><span class="p">,</span><span class="w">
</span><span class="s2">"renameat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"renameat2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"restart_syscall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rmdir"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigaction"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigpending"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigprocmask"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigqueueinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigreturn"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigsuspend"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_sigtimedwait"</span><span class="p">,</span><span class="w">
</span><span class="s2">"rt_tgsigqueueinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getaffinity"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getparam"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_get_priority_max"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_get_priority_min"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_getscheduler"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_rr_get_interval"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setaffinity"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setparam"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_setscheduler"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sched_yield"</span><span class="p">,</span><span class="w">
</span><span class="s2">"seccomp"</span><span class="p">,</span><span class="w">
</span><span class="s2">"select"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semop"</span><span class="p">,</span><span class="w">
</span><span class="s2">"semtimedop"</span><span class="p">,</span><span class="w">
</span><span class="s2">"send"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendfile"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendfile64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendmmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendmsg"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sendto"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setfsuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgroups"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setgroups32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setitimer"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setpgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setpriority"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setregid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setregid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresgid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresgid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setresuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setreuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setreuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_robust_list"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setsid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setsockopt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_thread_area"</span><span class="p">,</span><span class="w">
</span><span class="s2">"set_tid_address"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setuid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setuid32"</span><span class="p">,</span><span class="w">
</span><span class="s2">"setxattr"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmctl"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmdt"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shmget"</span><span class="p">,</span><span class="w">
</span><span class="s2">"shutdown"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sigaltstack"</span><span class="p">,</span><span class="w">
</span><span class="s2">"signalfd"</span><span class="p">,</span><span class="w">
</span><span class="s2">"signalfd4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sigreturn"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socket"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socketcall"</span><span class="p">,</span><span class="w">
</span><span class="s2">"socketpair"</span><span class="p">,</span><span class="w">
</span><span class="s2">"splice"</span><span class="p">,</span><span class="w">
</span><span class="s2">"stat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"stat64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"statfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"statfs64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"symlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"symlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sync"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sync_file_range"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syncfs"</span><span class="p">,</span><span class="w">
</span><span class="s2">"sysinfo"</span><span class="p">,</span><span class="w">
</span><span class="s2">"syslog"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tee"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tgkill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"time"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_delete"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_create"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timerfd_settime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_getoverrun"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_gettime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"timer_settime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"times"</span><span class="p">,</span><span class="w">
</span><span class="s2">"tkill"</span><span class="p">,</span><span class="w">
</span><span class="s2">"truncate"</span><span class="p">,</span><span class="w">
</span><span class="s2">"truncate64"</span><span class="p">,</span><span class="w">
</span><span class="s2">"ugetrlimit"</span><span class="p">,</span><span class="w">
</span><span class="s2">"umask"</span><span class="p">,</span><span class="w">
</span><span class="s2">"uname"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unlink"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unlinkat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utime"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utimensat"</span><span class="p">,</span><span class="w">
</span><span class="s2">"utimes"</span><span class="p">,</span><span class="w">
</span><span class="s2">"vfork"</span><span class="p">,</span><span class="w">
</span><span class="s2">"vmsplice"</span><span class="p">,</span><span class="w">
</span><span class="s2">"wait4"</span><span class="p">,</span><span class="w">
</span><span class="s2">"waitid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"waitpid"</span><span class="p">,</span><span class="w">
</span><span class="s2">"write"</span><span class="p">,</span><span class="w">
</span><span class="s2">"writev"</span><span class="p">,</span><span class="w">
</span><span class="s2">"mount"</span><span class="p">,</span><span class="w">
</span><span class="s2">"umount2"</span><span class="p">,</span><span class="w">
</span><span class="s2">"reboot"</span><span class="p">,</span><span class="w">
</span><span class="s2">"name_to_handle_at"</span><span class="p">,</span><span class="w">
</span><span class="s2">"unshare"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">8</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"personality"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">4294967295</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"arch_prctl"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"modify_ldt"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"names"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="s2">"clone"</span><span class="w">
</span><span class="p">],</span><span class="w">
</span><span class="s2">"action"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_ACT_ALLOW"</span><span class="p">,</span><span class="w">
</span><span class="s2">"args"</span><span class="p">:</span><span class="w"> </span><span class="p">[</span><span class="w">
</span><span class="p">{</span><span class="w">
</span><span class="s2">"index"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"value"</span><span class="p">:</span><span class="w"> </span><span class="mi">2080505856</span><span class="p">,</span><span class="w">
</span><span class="s2">"valueTwo"</span><span class="p">:</span><span class="w"> </span><span class="mi">0</span><span class="p">,</span><span class="w">
</span><span class="s2">"op"</span><span class="p">:</span><span class="w"> </span><span class="s2">"SCMP_CMP_MASKED_EQ"</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">]</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="p">}</span><span class="w">
</span></code></pre></div>
<h2 id="cgroup-metrics">CGroup Metrics</h2>
<p>Runc internally uses cgroups to control resources for containers. The CGroup Metrics described at <a href="cgroups_in_storm.md#CGroup-Metrics">cgroups_in_storm.md</a> still apply except CGroupCpuGuarantee. To get CGroup cpu guarantee, use CGroupCpuGuaranteeByCfsQuota instead.</p>
</div>
</div>
</div>
</div>
<footer>
<div class="container-fluid">
<div class="row">
<div class="col-md-3">
<div class="footer-widget">
<h5>Meetups</h5>
<ul class="latest-news">
<li><a href="http://www.meetup.com/Apache-Storm-Apache-Kafka/">Apache Storm & Apache Kafka</a> <span class="small">(Sunnyvale, CA)</span></li>
<li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Apache Storm & Kafka Users</a> <span class="small">(Seattle, WA)</span></li>
<li><a href="http://www.meetup.com/New-York-City-Storm-User-Group/">NYC Storm User Group</a> <span class="small">(New York, NY)</span></li>
<li><a href="http://www.meetup.com/Bay-Area-Stream-Processing">Bay Area Stream Processing</a> <span class="small">(Emeryville, CA)</span></li>
<li><a href="http://www.meetup.com/Boston-Storm-Users/">Boston Realtime Data</a> <span class="small">(Boston, MA)</span></li>
<li><a href="http://www.meetup.com/storm-london">London Storm User Group</a> <span class="small">(London, UK)</span></li>
<!-- <li><a href="http://www.meetup.com/Apache-Storm-Kafka-Users/">Seatle, WA</a> <span class="small">(27 Jun 2015)</span></li> -->
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>About Apache Storm</h5>
<p>Apache Storm integrates with any queueing system and any database system. Apache Storm's spout abstraction makes it easy to integrate a new queuing system. Likewise, integrating Apache Storm with database systems is easy.</p>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>First Look</h5>
<ul class="footer-list">
<li><a href="/releases/current/Rationale.html">Rationale</a></li>
<li><a href="/releases/current/Tutorial.html">Tutorial</a></li>
<li><a href="/releases/current/Setting-up-development-environment.html">Setting up development environment</a></li>
<li><a href="/releases/current/Creating-a-new-Storm-project.html">Creating a new Apache Storm project</a></li>
</ul>
</div>
</div>
<div class="col-md-3">
<div class="footer-widget">
<h5>Documentation</h5>
<ul class="footer-list">
<li><a href="/releases/current/index.html">Index</a></li>
<li><a href="/releases/current/javadocs/index.html">Javadoc</a></li>
<li><a href="/releases/current/FAQ.html">FAQ</a></li>
</ul>
</div>
</div>
</div>
<hr/>
<div class="row">
<div class="col-md-12">
<p align="center">Copyright © 2019 <a href="http://www.apache.org">Apache Software Foundation</a>. All Rights Reserved.
<br>Apache Storm, Apache, the Apache feather logo, and the Apache Storm project logos are trademarks of The Apache Software Foundation.
<br>All other marks mentioned may be trademarks or registered trademarks of their respective owners.</p>
</div>
</div>
</div>
</footer>
<!--Footer End-->
<!-- Scroll to top -->
<span class="totop"><a href="#"><i class="fa fa-angle-up"></i></a></span>
</body>
</html>