Merge pull request #2329 from balena-os/refactor-host-config-module

Refactor host config module
This commit is contained in:
flowzone-app[bot] 2024-07-04 20:24:45 +00:00 committed by GitHub
commit da807e45f6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 1212 additions and 360 deletions

View File

@ -7,6 +7,10 @@ import * as deviceState from '../device-state';
import * as logger from '../logger';
import * as config from '../config';
import * as hostConfig from '../host-config';
import type {
HostConfiguration,
LegacyHostConfiguration,
} from '../host-config/types';
import * as applicationManager from '../compose/application-manager';
import type { CompositionStepAction } from '../compose/composition-steps';
import { generateStep } from '../compose/composition-steps';
@ -445,14 +449,12 @@ export const getHostConfig = async () => {
* Used by:
* - PATCH /v1/device/host-config
*/
export const patchHostConfig = async (
conf: Parameters<typeof hostConfig.patch>[0],
force: boolean,
) => {
// If hostname is an empty string, return first 7 digits of device uuid
if (conf.network?.hostname === '') {
const uuid = await config.get('uuid');
conf.network.hostname = uuid?.slice(0, 7);
export const patchHostConfig = async (conf: unknown, force: boolean) => {
let parsedConf: HostConfiguration | LegacyHostConfiguration;
try {
parsedConf = hostConfig.parse(conf);
} catch (e: unknown) {
throw new BadRequestError((e as Error).message);
}
await hostConfig.patch(conf, force);
await hostConfig.patch(parsedConf, force);
};

View File

@ -6,7 +6,6 @@ import type { AuthorizedRequest } from '../lib/api-keys';
import * as eventTracker from '../event-tracker';
import type * as deviceState from '../device-state';
import * as constants from '../lib/constants';
import { checkInt, checkTruthy } from '../lib/validation';
import log from '../lib/supervisor-console';
import {
@ -16,8 +15,6 @@ import {
} from '../lib/errors';
import type { CompositionStepAction } from '../compose/composition-steps';
const disallowedHostConfigPatchFields = ['local_ip', 'local_port'];
export const router = express.Router();
router.post('/v1/restart', (req: AuthorizedRequest, res, next) => {
@ -176,34 +173,6 @@ router.patch('/v1/device/host-config', async (req, res) => {
// If network does not exist, skip all field validation checks below
throw new Error();
}
const { proxy } = req.body.network;
// Validate proxy fields, if they exist
if (proxy && Object.keys(proxy).length) {
const blacklistedFields = Object.keys(proxy).filter((key) =>
disallowedHostConfigPatchFields.includes(key),
);
if (blacklistedFields.length > 0) {
log.warn(`Invalid proxy field(s): ${blacklistedFields.join(', ')}`);
}
if (
proxy.type &&
!constants.validRedsocksProxyTypes.includes(proxy.type)
) {
log.warn(
`Invalid redsocks proxy type, must be one of ${constants.validRedsocksProxyTypes.join(
', ',
)}`,
);
}
if (proxy.noProxy && !Array.isArray(proxy.noProxy)) {
log.warn('noProxy field must be an array of addresses');
}
}
} catch (e) {
/* noop */
}
@ -218,6 +187,12 @@ router.patch('/v1/device/host-config', async (req, res) => {
if (e instanceof UpdatesLockedError) {
return res.status(423).send(e?.message ?? e);
}
// User input cannot be parsed to type HostConfiguration or LegacyHostConfiguration
if (isBadRequestError(e)) {
return res.status(e.statusCode).send(e.statusMessage);
}
return res.status(503).send((e as Error)?.message ?? e ?? 'Unknown error');
}
});

View File

@ -1,224 +0,0 @@
import { stripIndent } from 'common-tags';
import _ from 'lodash';
import { promises as fs } from 'fs';
import path from 'path';
import * as config from './config';
import * as applicationManager from './compose/application-manager';
import * as dbus from './lib/dbus';
import { isENOENT } from './lib/errors';
import { mkdirp, unlinkAll } from './lib/fs-utils';
import {
writeToBoot,
readFromBoot,
pathOnRoot,
pathOnBoot,
} from './lib/host-utils';
import * as updateLock from './lib/update-lock';
const redsocksHeader = stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
local_ip = 127.0.0.1;
local_port = 12345;
`;
const redsocksFooter = '}\n';
const proxyFields = ['type', 'ip', 'port', 'login', 'password'];
const proxyBasePath = pathOnBoot('system-proxy');
const redsocksConfPath = path.join(proxyBasePath, 'redsocks.conf');
const noProxyPath = path.join(proxyBasePath, 'no_proxy');
interface ProxyConfig {
[key: string]: string | string[] | number;
}
interface HostConfig {
network: {
proxy?: ProxyConfig;
hostname?: string;
};
}
const isAuthField = (field: string): boolean =>
['login', 'password'].includes(field);
const memoizedAuthRegex = _.memoize(
(proxyField: string) => new RegExp(proxyField + '\\s*=\\s*"(.*)"\\s*;'),
);
const memoizedRegex = _.memoize(
// Add beginning-of-line RegExp to prevent local_ip and local_port static fields from being memoized
(proxyField) => new RegExp('^\\s*' + proxyField + '\\s*=\\s*([^;\\s]*)\\s*;'),
);
async function readProxy(): Promise<ProxyConfig | undefined> {
const conf: ProxyConfig = {};
let redsocksConf: string;
try {
redsocksConf = await readFromBoot(redsocksConfPath, 'utf-8');
} catch (e: unknown) {
if (!isENOENT(e)) {
throw e;
}
return;
}
const lines = redsocksConf.split('\n');
for (const line of lines) {
for (const proxyField of proxyFields) {
let match: string[] | null = null;
if (isAuthField(proxyField)) {
match = line.match(memoizedAuthRegex(proxyField));
} else {
match = line.match(memoizedRegex(proxyField));
}
if (match != null) {
conf[proxyField] = match[1];
}
}
}
try {
const noProxy = await readFromBoot(noProxyPath, 'utf-8')
// Prevent empty newline from being reported as a noProxy address
.then((addrs) => addrs.split('\n').filter((addr) => addr !== ''));
if (noProxy.length) {
conf.noProxy = noProxy;
}
} catch (e: unknown) {
if (!isENOENT(e)) {
throw e;
}
}
// Convert port to number per API doc spec
if (conf.port) {
conf.port = parseInt(conf.port as string, 10);
}
return conf;
}
function generateRedsocksConfEntries(conf: ProxyConfig): string {
let val = '';
for (const field of proxyFields) {
let v = conf[field];
if (v != null) {
if (isAuthField(field)) {
// Escape any quotes in the field value
v = `"${v.toString().replace(/"/g, '\\"')}"`;
}
val += `\t${field} = ${v};\n`;
}
}
return val;
}
async function setProxy(maybeConf: ProxyConfig | null): Promise<void> {
if (_.isEmpty(maybeConf)) {
await unlinkAll(redsocksConfPath, noProxyPath);
} else {
// We know that maybeConf is not null due to the _.isEmpty check above,
// but the compiler doesn't
const conf = maybeConf as ProxyConfig;
await mkdirp(proxyBasePath);
if (Array.isArray(conf.noProxy)) {
await writeToBoot(noProxyPath, conf.noProxy.join('\n'));
}
let currentConf: ProxyConfig | undefined;
try {
currentConf = await readProxy();
} catch {
// Noop - current redsocks.conf does not exist
}
// If currentConf is undefined, the currentConf spread will be skipped.
// See: https://www.typescriptlang.org/docs/handbook/release-notes/typescript-4-1.html#conditional-spreads-create-optional-properties
const redsocksConf = `
${redsocksHeader}\n
${generateRedsocksConfEntries({ ...currentConf, ...conf })}
${redsocksFooter}
`;
await writeToBoot(redsocksConfPath, redsocksConf);
}
// restart balena-proxy-config if it is loaded and NOT PartOf redsocks-conf.target
if (
(
await Promise.any([
dbus.servicePartOf('balena-proxy-config'),
dbus.servicePartOf('resin-proxy-config'),
])
).includes('redsocks-conf.target') === false
) {
await Promise.any([
dbus.restartService('balena-proxy-config'),
dbus.restartService('resin-proxy-config'),
]);
}
// restart redsocks if it is loaded and NOT PartOf redsocks-conf.target
if (
(await dbus.servicePartOf('redsocks')).includes('redsocks-conf.target') ===
false
) {
await dbus.restartService('redsocks');
}
}
const hostnamePath = pathOnRoot('/etc/hostname');
async function readHostname() {
const hostnameData = await fs.readFile(hostnamePath, 'utf-8');
return _.trim(hostnameData);
}
async function setHostname(val: string) {
// Changing the hostname on config.json will trigger
// the OS config-json service to restart the necessary services
// so the change gets reflected on containers
await config.set({ hostname: val });
}
export async function get(): Promise<HostConfig> {
return {
network: {
proxy: await readProxy(),
hostname: await readHostname(),
},
};
}
export async function patch(
conf: HostConfig,
force: boolean = false,
): Promise<void> {
const apps = await applicationManager.getCurrentApps();
const appIds = Object.keys(apps).map((strId) => parseInt(strId, 10));
// It's possible for appIds to be an empty array, but patch shouldn't fail
// as it's not dependent on there being any running user applications.
return updateLock.lock(appIds, { force }, async () => {
const promises: Array<Promise<void>> = [];
if (conf != null && conf.network != null) {
if (conf.network.proxy != null) {
promises.push(setProxy(conf.network.proxy));
}
if (conf.network.hostname != null) {
promises.push(setHostname(conf.network.hostname));
}
}
await Promise.all(promises);
});
}

131
src/host-config/index.ts Normal file
View File

@ -0,0 +1,131 @@
import { promises as fs } from 'fs';
import { isRight } from 'fp-ts/lib/Either';
import Reporter from 'io-ts-reporters';
import type { RedsocksConfig, ProxyConfig } from './types';
import { HostConfiguration, LegacyHostConfiguration } from './types';
import { readProxy, setProxy } from './proxy';
import * as config from '../config';
// FIXME: The host-config module shouldn't be importing from compose
import * as applicationManager from '../compose/application-manager';
import { pathOnRoot } from '../lib/host-utils';
import log from '../lib/supervisor-console';
import * as updateLock from '../lib/update-lock';
const hostnamePath = pathOnRoot('/etc/hostname');
async function readHostname() {
const hostnameData = await fs.readFile(hostnamePath, 'utf-8');
return hostnameData.trim();
}
async function setHostname(val: string) {
let hostname = val;
// If hostname is an empty string, return first 7 digits of device uuid
if (!val) {
const uuid = await config.get('uuid');
hostname = uuid?.slice(0, 7) as string;
}
// Changing the hostname on config.json will trigger
// the OS config-json service to restart the necessary services
// so the change gets reflected on containers
await config.set({ hostname });
}
export function parse(
conf: unknown,
): HostConfiguration | LegacyHostConfiguration {
const decoded = HostConfiguration.decode(conf);
if (isRight(decoded)) {
return decoded.right;
} else {
log.warn(
['Malformed host config detected, things may not behave as expected:']
.concat(Reporter.report(decoded))
.join('\n'),
);
// We haven't strictly validated user input since introducing the API,
// endpoint, so accept configs where values may not be of the right type
// but have the right shape for backwards compatibility.
const legacyDecoded = LegacyHostConfiguration.decode(conf);
if (isRight(legacyDecoded)) {
return legacyDecoded.right;
}
}
throw new Error('Could not parse host config input to a valid format');
}
function patchProxy(
currentConf: RedsocksConfig,
inputConf: Partial<{
redsocks: Partial<ProxyConfig>;
}>,
): RedsocksConfig {
const patchedConf: RedsocksConfig = {};
// If input is empty, redsocks config should be removed
if (!inputConf || Object.keys(inputConf).length === 0) {
return patchedConf;
}
if (inputConf.redsocks && Object.keys(inputConf.redsocks).length > 0) {
// This method assumes that currentConf is a full ProxyConfig
// for backwards compatibility, so patchedConf.redsocks should
// also be a full ProxyConfig.
const patched = {
...currentConf.redsocks,
...inputConf.redsocks,
} as Record<string, any>;
patchedConf.redsocks = patched as ProxyConfig;
}
return patchedConf;
}
export async function patch(
conf: HostConfiguration | LegacyHostConfiguration,
force: boolean = false,
): Promise<void> {
const apps = await applicationManager.getCurrentApps();
const appIds = Object.keys(apps).map((strId) => parseInt(strId, 10));
if (conf.network.hostname != null) {
await setHostname(conf.network.hostname);
}
if (conf.network.proxy != null) {
const targetConf = conf.network.proxy;
// It's possible for appIds to be an empty array, but patch shouldn't fail
// as it's not dependent on there being any running user applications.
return updateLock.lock(appIds, { force }, async () => {
const proxyConf = await readProxy();
let currentConf: ProxyConfig | undefined = undefined;
if (proxyConf) {
delete proxyConf.noProxy;
currentConf = proxyConf;
}
// Merge current & target redsocks.conf
const patchedConf = patchProxy(
{
redsocks: currentConf,
},
{
redsocks: targetConf,
},
);
await setProxy(patchedConf, targetConf.noProxy);
});
}
}
export async function get(): Promise<HostConfiguration> {
const proxy = await readProxy();
return {
network: {
hostname: await readHostname(),
// Only return proxy if readProxy is not undefined
...(proxy && { proxy }),
},
};
}

254
src/host-config/proxy.ts Normal file
View File

@ -0,0 +1,254 @@
import { promises as fs } from 'fs';
import path from 'path';
import { isRight } from 'fp-ts/lib/Either';
import Reporter from 'io-ts-reporters';
import type { RedsocksConfig, HostProxyConfig } from './types';
import { ProxyConfig } from './types';
import { pathOnBoot, readFromBoot, writeToBoot } from '../lib/host-utils';
import { unlinkAll, mkdirp } from '../lib/fs-utils';
import { isENOENT } from '../lib/errors';
import log from '../lib/supervisor-console';
import * as dbus from '../lib/dbus';
const proxyBasePath = pathOnBoot('system-proxy');
const noProxyPath = path.join(proxyBasePath, 'no_proxy');
const redsocksConfPath = path.join(proxyBasePath, 'redsocks.conf');
const disallowedProxyFields = ['local_ip', 'local_port'];
const isAuthField = (field: string): boolean =>
['login', 'password'].includes(field);
// ? is a lazy operator, so only the contents up until the first `}(?=\s|$)` is matched.
// (?=\s|$) indicates that `}` must be followed by a whitespace or end of file to match,
// in case there are user fields with brackets such as login or password fields.
const blockRegexFor = (blockLabel: string) =>
new RegExp(`${blockLabel}\\s?{([\\s\\S]+?)}(?=\\s|$)`);
const baseBlock = {
log_debug: 'off',
log_info: 'on',
log: 'stderr',
daemon: 'off',
redirector: 'iptables',
};
export class RedsocksConf {
public static stringify(config: RedsocksConfig): string {
const blocks: string[] = [];
if (config.redsocks && Object.keys(config.redsocks).length > 0) {
blocks.push(RedsocksConf.stringifyBlock('base', baseBlock));
blocks.push(
RedsocksConf.stringifyBlock('redsocks', {
...config.redsocks,
local_ip: '127.0.0.1',
local_port: 12345,
}),
);
}
return blocks.length ? blocks.join('\n') : '';
}
public static parse(rawConf: string): RedsocksConfig {
const conf: RedsocksConfig = {};
rawConf = rawConf.trim();
if (rawConf.length === 0) {
return conf;
}
// Extract contents of `redsocks {...}` using regex
const rawRedsocksBlockMatch = rawConf.match(blockRegexFor('redsocks'));
// No group was captured, indicating malformed config
if (!rawRedsocksBlockMatch) {
log.warn('Invalid redsocks block in redsocks.conf');
return conf;
}
const rawRedsocksBlock = RedsocksConf.parseBlock(
rawRedsocksBlockMatch[1],
disallowedProxyFields,
);
const maybeProxyConfig = ProxyConfig.decode(rawRedsocksBlock);
if (isRight(maybeProxyConfig)) {
conf.redsocks = {
...maybeProxyConfig.right,
};
return conf;
} else {
log.warn(
['Invalid redsocks block in redsocks.conf:']
.concat(Reporter.report(maybeProxyConfig))
.join('\n'),
);
return {};
}
}
private static stringifyBlock(
label: string,
block: Record<string, any>,
): string {
const lines = Object.entries(block).map(([key, value]) => {
if (isAuthField(key)) {
// Add double quotes around login and password fields
value = `${value.startsWith('"') ? '' : '"'}${value}${value.endsWith('"') ? '' : '"'}`;
}
return `\t${key} = ${value};`;
});
return `${label} {\n${lines.join('\n')}\n}\n`;
}
/**
* Given the raw contents of a block redsocks.conf file,
* extract to a key-value object.
*/
private static parseBlock(
rawBlockConf: string,
unsupportedKeys: string[],
): Record<string, string> {
const parsedBlock: Record<string, string> = {};
// Split by newline and optional semicolon
for (const line of rawBlockConf.split(/;?\n/)) {
if (!line.trim().length) {
continue;
}
let [key, value] = line.split(/ *?= *?/).map((s) => s.trim());
// Don't parse unsupported keys
if (key && unsupportedKeys.some((k) => key.match(k))) {
continue;
}
if (key && value) {
if (isAuthField(key)) {
// Remove double quotes from login and password fields for readability
value = value.replace(/"/g, '');
}
parsedBlock[key] = value;
} else {
// Skip malformed lines
log.warn(
`Ignoring malformed redsocks.conf line ${isAuthField(key) ? `"${key}"` : `"${line.trim()}"`} due to missing key, value, or "="`,
);
}
}
return parsedBlock;
}
}
export async function readProxy(): Promise<HostProxyConfig | undefined> {
// Get and parse redsocks.conf
let rawConf: string | undefined;
try {
rawConf = await readFromBoot(redsocksConfPath, 'utf-8');
} catch (e: unknown) {
if (!isENOENT(e)) {
throw e;
}
return undefined;
}
const redsocksConf = RedsocksConf.parse(rawConf);
// Get and parse no_proxy
const noProxy = await readNoProxy();
// Build proxy object
const proxy = {
...redsocksConf.redsocks,
...(noProxy.length && { noProxy }),
};
// Assumes mandatory proxy config fields (type, ip, port) are present,
// even if they very well may not be. It is up to the user to ensure
// that all the necessary fields are present in the redsocks.conf file.
return proxy as HostProxyConfig;
}
export async function setProxy(
conf: RedsocksConfig,
noProxy: Nullable<string[]>,
) {
// Ensure proxy directory exists
await mkdirp(proxyBasePath);
// Set no_proxy
let noProxyChanged = false;
if (noProxy != null) {
noProxyChanged = await setNoProxy(noProxy);
}
// Write to redsocks.conf
const toWrite = RedsocksConf.stringify(conf);
if (toWrite) {
await writeToBoot(redsocksConfPath, toWrite);
}
// If target is empty aside from noProxy and noProxy got patched,
// do not change redsocks.conf to remain backwards compatible
else if (!noProxyChanged) {
await unlinkAll(redsocksConfPath);
}
// Restart services using dbus
await restartProxyServices();
}
async function restartProxyServices() {
// restart balena-proxy-config if it is loaded and NOT PartOf redsocks-conf.target
if (
(
await Promise.any([
dbus.servicePartOf('balena-proxy-config'),
dbus.servicePartOf('resin-proxy-config'),
])
).includes('redsocks-conf.target') === false
) {
await Promise.any([
dbus.restartService('balena-proxy-config'),
dbus.restartService('resin-proxy-config'),
]);
}
// restart redsocks if it is loaded and NOT PartOf redsocks-conf.target
if (
(await dbus.servicePartOf('redsocks')).includes('redsocks-conf.target') ===
false
) {
await dbus.restartService('redsocks');
}
}
async function readNoProxy(): Promise<string[]> {
try {
const noProxy = await readFromBoot(noProxyPath, 'utf-8')
// Prevent empty newline from being reported as a noProxy address
.then((addrs) => addrs.split('\n').filter((addr) => addr !== ''));
if (noProxy.length) {
return noProxy;
} else {
return [];
}
} catch (e: unknown) {
if (!isENOENT(e)) {
throw e;
}
return [];
}
}
async function setNoProxy(list: Nullable<string[]>) {
const current = await readNoProxy();
if (!list || !Array.isArray(list) || !list.length) {
await unlinkAll(noProxyPath);
} else {
await fs.writeFile(noProxyPath, list.join('\n'));
}
// If noProxy has changed, return true
return (
Array.isArray(list) &&
(current.length !== list.length ||
!current.every((addr) => list.includes(addr)))
);
}

69
src/host-config/types.ts Normal file
View File

@ -0,0 +1,69 @@
import * as t from 'io-ts';
import { NumericIdentifier } from '../types';
export const ProxyConfig = t.intersection([
t.type({
type: t.union([
t.literal('socks4'),
t.literal('socks5'),
t.literal('http-connect'),
t.literal('http-relay'),
]),
ip: t.string,
port: NumericIdentifier,
}),
// login & password are optional fields
t.partial({
login: t.string,
password: t.string,
}),
]);
export type ProxyConfig = t.TypeOf<typeof ProxyConfig>;
/**
* The internal object representation of redsocks.conf, obtained
* from RedsocksConf.parse
*/
export const RedsocksConfig = t.partial({
redsocks: ProxyConfig,
});
export type RedsocksConfig = t.TypeOf<typeof RedsocksConfig>;
/**
* An intersection of writeable redsocks.conf configurations, and
* additional noProxy field (which is a config relating to proxy configuration)
*/
export const HostProxyConfig = t.intersection([
ProxyConfig,
t.partial({
noProxy: t.array(t.string),
}),
]);
export type HostProxyConfig = t.TypeOf<typeof HostProxyConfig>;
/**
* A host configuration object which includes redsocks proxy configuration
* and hostname configuration. This is the input type provided by the user
* with host-config PATCH and provided to the user with host-config GET.
*/
export const HostConfiguration = t.type({
network: t.partial({
proxy: HostProxyConfig,
hostname: t.string,
}),
});
export type HostConfiguration = t.TypeOf<typeof HostConfiguration>;
/**
* A user may provide an input which is not a valid HostConfiguration object,
* but we've historically accepted these malformed configurations. This type
* covers the case of a user providing a configuration which is not strictly
* valid but has the correct shape.
*/
export const LegacyHostConfiguration = t.type({
network: t.partial({
proxy: t.record(t.string, t.any),
hostname: t.string,
}),
});
export type LegacyHostConfiguration = t.TypeOf<typeof LegacyHostConfiguration>;

View File

@ -75,7 +75,6 @@ const constants = {
// (this number is used as an upper bound when generating
// a random jitter)
maxApiJitterDelay: 60 * 1000,
validRedsocksProxyTypes: ['socks4', 'socks5', 'http-connect', 'http-relay'],
};
if (process.env.DOCKER_HOST == null) {

View File

@ -1,17 +1,17 @@
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
local_ip = 127.0.0.1;
local_port = 12345;
ip = example.org;
port = 1080;
type = socks5;
login = "foo";
password = "bar";
ip = example.org;
port = 1080;
type = socks5;
login = "foo";
password = "bar";
local_ip = 127.0.0.1;
local_port = 12345;
}

View File

@ -1294,18 +1294,4 @@ describe('patches host config', () => {
await actions.patchHostConfig(conf, true);
expect(hostConfigPatch).to.have.been.calledWith(conf, true);
});
it('patches hostname as first 7 digits of uuid if hostname parameter is empty string', async () => {
const conf = {
network: {
hostname: '',
},
};
const uuid = await config.get('uuid');
await actions.patchHostConfig(conf, true);
expect(hostConfigPatch).to.have.been.calledWith(
{ network: { hostname: uuid?.slice(0, 7) } },
true,
);
});
});

View File

@ -18,7 +18,6 @@ import {
BadRequestError,
} from '~/lib/errors';
import log from '~/lib/supervisor-console';
import * as constants from '~/lib/constants';
// All routes that require Authorization are integration tests due to
// the api-key module relying on the database.
@ -805,50 +804,6 @@ describe('device-api/v1', () => {
before(() => stub(actions, 'patchHostConfig'));
after(() => (actions.patchHostConfig as SinonStub).restore());
const validProxyReqs: { [key: string]: number[] | string[] } = {
ip: ['proxy.example.org', 'proxy.foo.org'],
port: [5128, 1080],
type: constants.validRedsocksProxyTypes,
login: ['user', 'user2'],
password: ['foo', 'bar'],
};
it('warns on the supervisor console when provided disallowed proxy fields', async () => {
const invalidProxyReqs: { [key: string]: string | number } = {
// At this time, don't support changing local_ip or local_port
local_ip: '0.0.0.0',
local_port: 12345,
type: 'invalidType',
noProxy: 'not a list of addresses',
};
for (const key of Object.keys(invalidProxyReqs)) {
await request(api)
.patch('/v1/device/host-config')
.set('Authorization', `Bearer ${await apiKeys.getGlobalApiKey()}`)
.send({ network: { proxy: { [key]: invalidProxyReqs[key] } } })
.expect(200)
.then(() => {
if (key === 'type') {
expect(log.warn as SinonStub).to.have.been.calledWith(
`Invalid redsocks proxy type, must be one of ${validProxyReqs.type.join(
', ',
)}`,
);
} else if (key === 'noProxy') {
expect(log.warn as SinonStub).to.have.been.calledWith(
'noProxy field must be an array of addresses',
);
} else {
expect(log.warn as SinonStub).to.have.been.calledWith(
`Invalid proxy field(s): ${key}`,
);
}
});
(log.warn as SinonStub).reset();
}
});
it('warns on console when sent a malformed patch body', async () => {
await request(api)
.patch('/v1/device/host-config')
@ -861,5 +816,41 @@ describe('device-api/v1', () => {
);
});
});
it('responds with 200 if patch successful', async () => {
(actions.patchHostConfig as SinonStub).resolves();
await request(api)
.patch('/v1/device/host-config')
.send({ network: { hostname: 'deadbeef' } })
.set('Authorization', `Bearer ${await apiKeys.getGlobalApiKey()}`)
.expect(200);
});
it('responds with 423 for update lock errors', async () => {
(actions.patchHostConfig as SinonStub).throws(new UpdatesLockedError());
await request(api)
.patch('/v1/device/host-config')
.send({ network: { hostname: 'deadbeef' } })
.set('Authorization', `Bearer ${await apiKeys.getGlobalApiKey()}`)
.expect(423);
});
it('responds with 400 for BadRequestErrors', async () => {
(actions.patchHostConfig as SinonStub).throws(new BadRequestError());
await request(api)
.patch('/v1/device/host-config')
.send({ network: { hostname: 'deadbeef' } })
.set('Authorization', `Bearer ${await apiKeys.getGlobalApiKey()}`)
.expect(400);
});
it('responds with 503 for other errors that occur during patch', async () => {
(actions.patchHostConfig as SinonStub).throws(new Error());
await request(api)
.patch('/v1/device/host-config')
.send({ network: { hostname: 'deadbeef' } })
.set('Authorization', `Bearer ${await apiKeys.getGlobalApiKey()}`)
.expect(503);
});
});
});

View File

@ -6,7 +6,7 @@ import type { SinonStub } from 'sinon';
import { stub } from 'sinon';
import * as fs from 'fs/promises';
import * as hostConfig from '~/src/host-config';
import { get, patch } from '~/src/host-config';
import * as config from '~/src/config';
import * as applicationManager from '~/src/compose/application-manager';
import type { InstancedAppState } from '~/src/compose/types';
@ -72,16 +72,18 @@ describe('host-config', () => {
// Stub external dependencies
stub(dbus, 'servicePartOf').resolves([]);
stub(dbus, 'restartService').resolves();
stub(applicationManager, 'getCurrentApps').resolves({});
});
afterEach(async () => {
await tFs.restore();
(dbus.servicePartOf as SinonStub).restore();
(dbus.restartService as SinonStub).restore();
(applicationManager.getCurrentApps as SinonStub).restore();
});
it('reads proxy configs and hostname', async () => {
const { network } = await hostConfig.get();
const { network } = await get();
expect(network).to.have.property('hostname', 'deadbeef');
expect(network).to.have.property('proxy');
expect(network.proxy).to.have.property('ip', 'example.org');
@ -96,40 +98,47 @@ describe('host-config', () => {
});
it('prevents patch if update locks are present', async () => {
stub(applicationManager, 'getCurrentApps').resolves(currentApps);
(applicationManager.getCurrentApps as SinonStub).resolves(currentApps);
try {
await hostConfig.patch({ network: { hostname: 'test' } });
await patch({ network: { proxy: {} } });
expect.fail('Expected hostConfig.patch to throw UpdatesLockedError');
} catch (e: unknown) {
expect(e).to.be.instanceOf(UpdatesLockedError);
}
(applicationManager.getCurrentApps as SinonStub).restore();
});
it('patches if update locks are present but force is specified', async () => {
stub(applicationManager, 'getCurrentApps').resolves(currentApps);
(applicationManager.getCurrentApps as SinonStub).resolves(currentApps);
try {
await hostConfig.patch({ network: { hostname: 'deadreef' } }, true);
expect(await config.get('hostname')).to.equal('deadreef');
await patch({ network: { proxy: {} } }, true);
} catch (e: unknown) {
expect.fail(`Expected hostConfig.patch to not throw, but got ${e}`);
}
expect(await get()).to.deep.equal({ network: { hostname: 'deadbeef' } });
});
(applicationManager.getCurrentApps as SinonStub).restore();
it('patches hostname regardless of update locks', async () => {
(applicationManager.getCurrentApps as SinonStub).resolves(currentApps);
try {
await patch({ network: { hostname: 'test' } });
expect(await config.get('hostname')).to.equal('test');
} catch (e: unknown) {
expect.fail(`Expected hostConfig.patch to not throw, but got ${e}`);
}
});
it('patches hostname', async () => {
await hostConfig.patch({ network: { hostname: 'test' } });
await patch({ network: { hostname: 'test' } });
// /etc/hostname isn't changed until the balena-hostname service
// is restarted by the OS.
expect(await config.get('hostname')).to.equal('test');
});
it('patches proxy', async () => {
await hostConfig.patch({
await patch({
network: {
proxy: {
ip: 'example2.org',
@ -141,7 +150,7 @@ describe('host-config', () => {
},
},
});
const { network } = await hostConfig.get();
const { network } = await get();
expect(network).to.have.property('proxy');
expect(network.proxy).to.have.property('ip', 'example2.org');
expect(network.proxy).to.have.property('port', 1090);
@ -154,9 +163,65 @@ describe('host-config', () => {
]);
});
it('patches proxy fields specified while leaving unspecified fields unchanged', async () => {
await patch({
network: {
proxy: {
ip: 'example2.org',
port: 1090,
},
},
});
const { network } = await get();
expect(network).to.have.property('proxy');
expect(network.proxy).to.have.property('ip', 'example2.org');
expect(network.proxy).to.have.property('port', 1090);
expect(network.proxy).to.have.property('type', 'socks5');
expect(network.proxy).to.have.property('login', 'foo');
expect(network.proxy).to.have.property('password', 'bar');
expect(network.proxy).to.have.deep.property('noProxy', [
'152.10.30.4',
'253.1.1.0/16',
]);
});
it('patches proxy to empty if input is empty', async () => {
await patch({ network: { proxy: {} } });
const { network } = await get();
expect(network).to.not.have.property('proxy');
});
it('keeps current proxy if input is invalid', async () => {
await patch({ network: { proxy: null as any } });
const { network } = await get();
expect(network).to.have.property('proxy');
expect(network.proxy).to.have.property('ip', 'example.org');
expect(network.proxy).to.have.property('port', 1080);
expect(network.proxy).to.have.property('type', 'socks5');
expect(network.proxy).to.have.property('login', 'foo');
expect(network.proxy).to.have.property('password', 'bar');
expect(network.proxy).to.have.deep.property('noProxy', [
'152.10.30.4',
'253.1.1.0/16',
]);
});
it('ignores unsupported fields when patching proxy', async () => {
const rawConf = await fs.readFile(redsocksConf, 'utf-8');
await patch({
network: {
proxy: {
local_ip: '127.0.0.2',
local_port: 12346,
} as any,
},
});
expect(await fs.readFile(redsocksConf, 'utf-8')).to.equal(rawConf);
});
it('skips restarting proxy services when part of redsocks-conf.target', async () => {
(dbus.servicePartOf as SinonStub).resolves(['redsocks-conf.target']);
await hostConfig.patch({
await patch({
network: {
proxy: {
ip: 'example2.org',
@ -169,7 +234,7 @@ describe('host-config', () => {
},
});
expect(dbus.restartService as SinonStub).to.not.have.been.called;
const { network } = await hostConfig.get();
const { network } = await get();
expect(network).to.have.property('proxy');
expect(network.proxy).to.have.property('ip', 'example2.org');
expect(network.proxy).to.have.property('port', 1090);
@ -183,9 +248,9 @@ describe('host-config', () => {
});
it('patches redsocks.conf to be empty if prompted', async () => {
await hostConfig.patch({ network: { proxy: {} } });
const { network } = await hostConfig.get();
expect(network).to.have.property('proxy', undefined);
await patch({ network: { proxy: {} } });
const { network } = await get();
expect(network).to.not.have.property('proxy');
expect(await fs.readdir(proxyBase)).to.not.have.members([
'redsocks.conf',
'no_proxy',
@ -193,23 +258,30 @@ describe('host-config', () => {
});
it('patches no_proxy to be empty if prompted', async () => {
await hostConfig.patch({
await patch({
network: {
proxy: {
noProxy: [],
},
},
});
const { network } = await hostConfig.get();
expect(network).to.have.property('proxy');
const { network } = await get();
// If only noProxy is patched, redsocks.conf should remain unchanged
expect(network).to.have.property('proxy').that.deep.includes({
ip: 'example.org',
port: 1080,
type: 'socks5',
login: 'foo',
password: 'bar',
});
expect(network.proxy).to.not.have.property('noProxy');
expect(await fs.readdir(proxyBase)).to.not.have.members(['no_proxy']);
});
it("doesn't update hostname or proxy when both are empty", async () => {
const { network } = await hostConfig.get();
await hostConfig.patch({ network: {} });
const { network: newNetwork } = await hostConfig.get();
const { network } = await get();
await patch({ network: {} });
const { network: newNetwork } = await get();
expect(network.hostname).to.equal(newNetwork.hostname);
expect(network.proxy).to.deep.equal(newNetwork.proxy);
});

View File

@ -0,0 +1,597 @@
import { expect } from 'chai';
import { stripIndent } from 'common-tags';
import type { SinonStub } from 'sinon';
import * as hostConfig from '~/src/host-config';
import { RedsocksConf } from '~/src/host-config/proxy';
import type { RedsocksConfig, ProxyConfig } from '~/src/host-config/types';
import log from '~/lib/supervisor-console';
describe('RedsocksConf', () => {
describe('stringify', () => {
it('stringifies RedsocksConfig into config string', () => {
const conf: RedsocksConfig = {
redsocks: {
type: 'socks5',
ip: 'example.org',
port: 1080,
login: '"foo"',
password: '"bar"',
},
};
const confStr = RedsocksConf.stringify(conf);
expect(confStr).to.equal(
stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = "foo";
password = "bar";
local_ip = 127.0.0.1;
local_port = 12345;
}
` + '\n',
);
});
it('adds double quotes to auth fields if not exists', () => {
const conf: RedsocksConfig = {
redsocks: {
type: 'socks5',
ip: 'example.org',
port: 1080,
login: 'foo',
password: 'bar',
},
};
const confStr = RedsocksConf.stringify(conf);
expect(confStr).to.equal(
stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = "foo";
password = "bar";
local_ip = 127.0.0.1;
local_port = 12345;
}
` + '\n',
);
});
it('accepts port field of type string', () => {
const conf = {
redsocks: {
type: 'socks5',
ip: 'example.org',
port: '1080',
login: 'foo',
password: 'bar',
},
} as unknown as RedsocksConfig;
const confStr = RedsocksConf.stringify(conf);
expect(confStr).to.equal(
stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = "foo";
password = "bar";
local_ip = 127.0.0.1;
local_port = 12345;
}
` + '\n',
);
});
it('stringifies to empty string when provided empty RedsocksConfig', () => {
const conf: RedsocksConfig = {};
const confStr = RedsocksConf.stringify(conf);
expect(confStr).to.equal('');
});
it('stringifies to empty string when provided empty redsocks block', () => {
const conf: RedsocksConfig = {
redsocks: {} as ProxyConfig,
};
const confStr = RedsocksConf.stringify(conf);
expect(confStr).to.equal('');
});
});
describe('parse', () => {
it('parses config string into RedsocksConfig', () => {
const redsocksConfStr = stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
local_ip = 127.0.0.1;
local_port = 12345;
type = socks5;
ip = example.org;
port = 1080;
login = "foo";
password = "bar";
}
dnstc {
test = test;
}
`;
const conf = RedsocksConf.parse(redsocksConfStr);
expect(conf).to.deep.equal({
redsocks: {
type: 'socks5',
ip: 'example.org',
port: 1080,
login: 'foo',
password: 'bar',
},
});
});
it("parses `redsocks {...}` config block no matter what position it's in or how many newlines surround it", () => {
const redsocksConfStr = stripIndent`
dnsu2t {
test = test;
}
redsocks {
local_ip = 127.0.0.1;
local_port = 12345;
type = http-connect;
ip = {test2}.balenadev.io;
port = 1082;
login = "us}{er";
password = "p{}a}}s{{s";
}
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
`;
const conf = RedsocksConf.parse(redsocksConfStr);
expect(conf).to.deep.equal({
redsocks: {
type: 'http-connect',
ip: '{test2}.balenadev.io',
port: 1082,
login: 'us}{er',
password: 'p{}a}}s{{s',
},
});
const redsocksConfStr2 = stripIndent`
base {
log_debug = off;
log_info = on;
log = stderr;
daemon = off;
redirector = iptables;
}
redsocks {
local_ip = 127.0.0.1;
local_port = 12345;
type = http-connect;
ip = {test2}.balenadev.io;
port = 1082;
login = "us}{er";
password = "p{}a}}s{{s";
}`; // No newlines
const conf2 = RedsocksConf.parse(redsocksConfStr2);
expect(conf2).to.deep.equal({
redsocks: {
type: 'http-connect',
ip: '{test2}.balenadev.io',
port: 1082,
login: 'us}{er',
password: 'p{}a}}s{{s',
},
});
});
it('parses and removes double quotes around auth fields if present', async () => {
const expected = {
redsocks: {
ip: 'example.org',
login: 'user',
password: 'pass',
port: 1080,
type: 'socks5',
},
};
const confStr = stripIndent`
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = user;
password = pass;
}
`;
const conf = RedsocksConf.parse(confStr);
expect(conf).to.deep.equal(expected);
const confStr2 = stripIndent`
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = "user;
password = pass";
}
`;
const conf2 = RedsocksConf.parse(confStr2);
expect(conf2).to.deep.equal(expected);
const confStr3 = stripIndent`
redsocks {
type = socks5;
ip = example.org;
port = 1080;
login = "user";
password = "pass";
}
`;
const conf3 = RedsocksConf.parse(confStr3);
expect(conf3).to.deep.equal(expected);
});
it('parses to empty redsocks config with warnings while any values are invalid', () => {
const redsocksConfStr = stripIndent`
redsocks {
local_ip = 123;
local_port = foo;
type = socks6;
ip = 456;
port = bar;
login = user;
password = pass;
invalid_field = invalid_value;
}
`;
(log.warn as SinonStub).resetHistory();
const conf = RedsocksConf.parse(redsocksConfStr);
expect((log.warn as SinonStub).lastCall.args[0]).to.equal(
'Invalid redsocks block in redsocks.conf:\n' +
'Expecting NumericIdentifier at 0.port but instead got: "bar" (must be be an positive integer)\n' +
'Expecting one of:\n' +
' "socks4"\n' +
' "socks5"\n' +
' "http-connect"\n' +
' "http-relay"\n' +
'at 0.type but instead got: "socks6"',
);
(log.warn as SinonStub).resetHistory();
expect(conf).to.deep.equal({});
});
it('parses to empty config with warnings while some key-value pairs are malformed', () => {
// Malformed key-value pairs are pairs that are missing a key, value, or "="
const redsocksConfStr = stripIndent`
base {
log_debug off;
log_info = on
= stderr;
daemon = ;
redirector = iptables;
}
redsocks {
local_ip 127.0.0.1;
local_port = 12345
= socks5;
ip = ;
= 1080;
login =;
password = "bar";
}
`;
(log.warn as SinonStub).resetHistory();
const conf = RedsocksConf.parse(redsocksConfStr);
expect(
(log.warn as SinonStub).getCalls().map((call) => call.firstArg),
).to.deep.equal([
'Ignoring malformed redsocks.conf line "= socks5" due to missing key, value, or "="',
'Ignoring malformed redsocks.conf line "ip =" due to missing key, value, or "="',
'Ignoring malformed redsocks.conf line "= 1080" due to missing key, value, or "="',
'Ignoring malformed redsocks.conf line "login" due to missing key, value, or "="',
'Invalid redsocks block in redsocks.conf:\n' +
'Expecting string at 0.ip but instead got: undefined\n' +
'Expecting number at 0.port.0 but instead got: undefined\n' +
'Expecting string at 0.port.1 but instead got: undefined\n' +
'Expecting one of:\n' +
' "socks4"\n' +
' "socks5"\n' +
' "http-connect"\n' +
' "http-relay"\n' +
'at 0.type but instead got: undefined',
]);
(log.warn as SinonStub).resetHistory();
expect(conf).to.deep.equal({});
});
it('parses to empty config with warnings when a block is empty', () => {
const redsocksConfStr = stripIndent`
base {
}
redsocks {
}
`;
(log.warn as SinonStub).resetHistory();
const conf = RedsocksConf.parse(redsocksConfStr);
expect(
(log.warn as SinonStub).getCalls().map((call) => call.firstArg),
).to.deep.equal([
'Invalid redsocks block in redsocks.conf:\n' +
'Expecting string at 0.ip but instead got: undefined\n' +
'Expecting number at 0.port.0 but instead got: undefined\n' +
'Expecting string at 0.port.1 but instead got: undefined\n' +
'Expecting one of:\n' +
' "socks4"\n' +
' "socks5"\n' +
' "http-connect"\n' +
' "http-relay"\n' +
'at 0.type but instead got: undefined',
]);
(log.warn as SinonStub).resetHistory();
expect(conf).to.deep.equal({});
});
});
});
describe('src/host-config', () => {
describe('parse', () => {
it('parses valid HostConfiguration', () => {
const conf = {
network: {
proxy: {
type: 'socks4',
ip: 'balena.io',
port: 1079,
login: '"baz"',
password: '"foo"',
noProxy: ['8.8.8.8'],
},
hostname: 'balena',
},
};
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
type: 'socks4',
ip: 'balena.io',
port: 1079,
login: '"baz"',
password: '"foo"',
noProxy: ['8.8.8.8'],
},
hostname: 'balena',
},
});
});
it('parses valid HostConfiguration with only hostname', () => {
const conf = {
network: {
hostname: 'balena2',
},
};
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
hostname: 'balena2',
},
});
});
it('parses valid HostConfiguration with only proxy', () => {
const conf = {
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: 1081,
login: '"foo"',
password: '"bar"',
noProxy: ['3.3.3.3'],
},
},
};
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: 1081,
login: '"foo"',
password: '"bar"',
noProxy: ['3.3.3.3'],
},
},
});
});
it('parses valid HostConfiguration with only noProxy', () => {
const conf = {
network: {
proxy: {
noProxy: ['1.1.1.1', '2.2.2.2'],
},
},
};
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
noProxy: ['1.1.1.1', '2.2.2.2'],
},
},
});
});
it('parses HostConfiguration where auth fields are missing double quotes', () => {
const conf = {
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: 1081,
login: 'foo',
password: 'bar',
noProxy: ['3.3.3.3'],
},
},
};
(log.warn as SinonStub).resetHistory();
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: 1081,
login: 'foo',
password: 'bar',
noProxy: ['3.3.3.3'],
},
},
});
// Should not warn about missing double quotes
expect(log.warn as SinonStub).to.not.have.been.called;
});
it('parses HostConfiguration where port is a string', () => {
const conf = {
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: '1081',
login: '"foo"',
password: '"bar"',
noProxy: ['3.3.3.3'],
},
},
};
(log.warn as SinonStub).resetHistory();
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
type: 'http-connect',
ip: 'test.balena.io',
port: 1081,
login: '"foo"',
password: '"bar"',
noProxy: ['3.3.3.3'],
},
},
});
// Should not warn about port being a string
expect(log.warn as SinonStub).to.not.have.been.called;
});
// Allow invalid fields through for backwards compatibility
it('parses input with invalid proxy as LegacyHostConfiguration with console warnings', () => {
const conf = {
network: {
proxy: {
type: 'socks6',
ip: 123,
port: 'abc',
login: 'user',
password: 'pass',
noProxy: true,
},
},
};
(log.warn as SinonStub).resetHistory();
expect(hostConfig.parse(conf)).to.deep.equal({
network: {
proxy: {
type: 'socks6',
ip: 123,
port: 'abc',
login: 'user',
password: 'pass',
noProxy: true,
},
},
});
expect((log.warn as SinonStub).lastCall.args[0]).to.equal(
'Malformed host config detected, things may not behave as expected:\n' +
'Expecting string at network.proxy.0.0.ip but instead got: 123\n' +
'Expecting NumericIdentifier at network.proxy.0.0.port but instead got: "abc" (must be be an positive integer)\n' +
'Expecting one of:\n' +
' "socks4"\n' +
' "socks5"\n' +
' "http-connect"\n' +
' "http-relay"\n' +
'at network.proxy.0.0.type but instead got: "socks6"\n' +
'Expecting Array<string> at network.proxy.1.noProxy but instead got: true',
);
(log.warn as SinonStub).resetHistory();
});
it('throws error for HostConfiguration without network key', () => {
expect(() => hostConfig.parse({})).to.throw(
'Could not parse host config input to a valid format',
);
});
it('throws error for HostConfiguration with invalid network key', () => {
const conf = {
network: 123,
};
expect(() => hostConfig.parse(conf)).to.throw(
'Could not parse host config input to a valid format',
);
});
it('throws error for HostConfiguration with invalid hostname', () => {
const conf = {
network: {
hostname: 123,
},
};
expect(() => hostConfig.parse(conf)).to.throw(
'Could not parse host config input to a valid format',
);
});
});
});