From 15d5988e692c182892a118115fd7025048e06c29 Mon Sep 17 00:00:00 2001 From: Ryan Liptak Date: Sun, 16 Jan 2022 20:11:08 -0800 Subject: Add `process.EnvMap`, a platform-independent environment variable map EnvMap provides the same API as the previously used BufMap (besides `putMove` and `getPtr`), so usage sites of `getEnvMap` can usually remain unchanged. For non-Windows, EnvMap is a wrapper around BufMap. On Windows, it uses a new EnvMapWindows to handle some Windows-specific behavior: - Lookups use Unicode-aware case insensitivity (but `get` cannot return an error because EnvMapWindows has an internal buffer to use for lookup conversions) - Canonical names are returned when iterating the EnvMap Fixes #10561, closes #4603 --- lib/std/process.zig | 403 ++++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 391 insertions(+), 12 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/process.zig b/lib/std/process.zig index c0f11b22ce..f5d14cf6da 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -2,7 +2,6 @@ const std = @import("std.zig"); const builtin = @import("builtin"); const os = std.os; const fs = std.fs; -const BufMap = std.BufMap; const mem = std.mem; const math = std.math; const Allocator = mem.Allocator; @@ -53,9 +52,385 @@ test "getCwdAlloc" { testing.allocator.free(cwd); } -/// Caller owns resulting `BufMap`. -pub fn getEnvMap(allocator: Allocator) !BufMap { - var result = BufMap.init(allocator); +/// EnvMap for Windows that handles Unicode-aware case insensitivity for lookups, while also +/// providing the canonical environment variable names when iterating. +/// +/// Allows for zero-allocation lookups (even though it needs to do UTF-8 -> UTF-16 -> uppercase +/// conversions) by allocating a buffer large enough to fit the largest environment variable +/// name, and using that when doing lookups (i.e. anything that overflows the buffer can be treated +/// as the environment variable not being found). +pub const EnvMapWindows = struct { + allocator: Allocator, + /// Keys are UTF-16le stored as []const u8 + uppercased_map: std.StringHashMapUnmanaged(EnvValue), + /// Buffer for converting to uppercased UTF-16 on key lookups + /// Must call `reallocUppercaseBuf` before doing any lookups after a `put` call. + uppercase_buf_utf16: []u16 = &[_]u16{}, + max_name_utf16_length: usize = 0, + + pub const EnvValue = struct { + value: []const u8, + canonical_name: []const u8, + }; + + const Self = @This(); + + /// Deinitialize with `deinit`. + pub fn init(allocator: Allocator) Self { + return .{ + .allocator = allocator, + .uppercased_map = std.StringHashMapUnmanaged(EnvValue){}, + }; + } + + pub fn deinit(self: *Self) void { + var it = self.uppercased_map.iterator(); + while (it.next()) |entry| { + self.allocator.free(entry.key_ptr.*); + self.allocator.free(entry.value_ptr.value); + self.allocator.free(entry.value_ptr.canonical_name); + } + self.uppercased_map.deinit(self.allocator); + self.allocator.free(self.uppercase_buf_utf16); + } + + /// Increases the size of the uppercase buffer if the maximum name size has increased. + /// Must be called before any `get` calls after any number of `put` calls. + pub fn reallocUppercaseBuf(self: *Self) !void { + if (self.max_name_utf16_length > self.uppercase_buf_utf16.len) { + self.uppercase_buf_utf16 = try self.allocator.realloc(self.uppercase_buf_utf16, self.max_name_utf16_length); + } + } + + /// Converts `src` to uppercase using `RtlUpcaseUnicodeString` and puts the result in `dest`. + /// Returns the length of the converted UTF-16 string. `dest.len` must be >= `src.len`. + /// + /// Note: As of now, RtlUpcaseUnicodeString does not seem to handle codepoints above 0x10000 + /// (i.e. those that require a surrogate pair), so this function will always return a length + /// equal to `src.len`. However, if RtlUpcaseUnicodeString is updated to handle codepoints above + /// 0x10000, this property would still hold unless there are lowercase <-> uppercase conversions + /// that cross over the boundary between codepoints >= 0x10000 and < 0x10000. + /// TODO: Is it feasible that Unicode lowercase <-> uppercase conversions could cross that boundary? + fn uppercaseName(dest: []u16, src: []const u16) u16 { + assert(dest.len >= src.len); + + const dest_bytes = @intCast(u16, dest.len * 2); + var dest_string = os.windows.UNICODE_STRING{ + .Length = dest_bytes, + .MaximumLength = dest_bytes, + .Buffer = @intToPtr([*]u16, @ptrToInt(dest.ptr)), + }; + const src_bytes = @intCast(u16, src.len * 2); + const src_string = os.windows.UNICODE_STRING{ + .Length = src_bytes, + .MaximumLength = src_bytes, + .Buffer = @intToPtr([*]u16, @ptrToInt(src.ptr)), + }; + const rc = os.windows.ntdll.RtlUpcaseUnicodeString(&dest_string, &src_string, os.windows.FALSE); + switch (rc) { + .SUCCESS => return dest_string.Length / 2, + else => unreachable, // we are not allocating, so no errors should be possible + } + } + + /// Note: Does not realloc the uppercase buf to allow for calling put for many variables and + /// only allocating the uppercase buf afterwards. + pub fn putUtf8(self: *Self, name: []const u8, value: []const u8) !void { + const uppercased_len = len: { + const name_uppercased_utf16 = uppercased: { + var name_utf16_buf = try std.ArrayListAligned(u8, @alignOf(u16)).initCapacity(self.allocator, name.len); + errdefer name_utf16_buf.deinit(); + + var uppercased_len = try std.unicode.utf8ToUtf16LeWriter(name_utf16_buf.writer(), name); + assert(uppercased_len == name_utf16_buf.items.len); + + break :uppercased name_utf16_buf.toOwnedSlice(); + }; + errdefer self.allocator.free(name_uppercased_utf16); + + const name_canonical = try self.allocator.dupe(u8, name); + errdefer self.allocator.free(name_canonical); + + const value_dupe = try self.allocator.dupe(u8, value); + errdefer self.allocator.free(value_dupe); + + const get_or_put = try self.uppercased_map.getOrPut(self.allocator, name_uppercased_utf16); + if (get_or_put.found_existing) { + // note: this is only safe from UAF because the errdefer that frees this value above + // no longer has a possibility of being triggered after this point + self.allocator.free(name_uppercased_utf16); + self.allocator.free(get_or_put.value_ptr.value); + self.allocator.free(get_or_put.value_ptr.canonical_name); + } else { + get_or_put.key_ptr.* = name_uppercased_utf16; + } + get_or_put.value_ptr.value = value_dupe; + get_or_put.value_ptr.canonical_name = name_canonical; + + break :len name_uppercased_utf16.len; + }; + + // The buffer for case conversion for key lookups will need to be as big as the largest + // key stored in the hash map. + self.max_name_utf16_length = @maximum(self.max_name_utf16_length, uppercased_len); + } + + /// Asserts that the name does not already exist in the map. + /// Note: Does not realloc the uppercase buf to allow for calling put for many variables and + /// only allocating the uppercase buf afterwards. + pub fn putUtf16NoClobber(self: *Self, name_utf16: []const u16, value_utf16: []const u16) !void { + const uppercased_len = len: { + const name_canonical = try std.unicode.utf16leToUtf8Alloc(self.allocator, name_utf16); + errdefer self.allocator.free(name_canonical); + + const value = try std.unicode.utf16leToUtf8Alloc(self.allocator, value_utf16); + errdefer self.allocator.free(value); + + const name_uppercased_utf16 = try self.allocator.alloc(u16, name_utf16.len); + errdefer self.allocator.free(name_uppercased_utf16); + + const uppercased_len = uppercaseName(name_uppercased_utf16, name_utf16); + assert(uppercased_len == name_uppercased_utf16.len); + + try self.uppercased_map.putNoClobber(self.allocator, std.mem.sliceAsBytes(name_uppercased_utf16), EnvValue{ + .value = value, + .canonical_name = name_canonical, + }); + break :len name_uppercased_utf16.len; + }; + + // The buffer for case conversion for key lookups will need to be as big as the largest + // key stored in the hash map. + self.max_name_utf16_length = @maximum(self.max_name_utf16_length, uppercased_len); + } + + /// Attempts to convert a UTF-8 name into a uppercased UTF-16le name for a lookup. If the + /// name cannot be converted, this function will return `null`. + fn utf8ToUppercasedUtf16(self: Self, name: []const u8) ?[]u16 { + const name_utf16: []u16 = to_utf16: { + var utf16_buf_stream = std.io.fixedBufferStream(std.mem.sliceAsBytes(self.uppercase_buf_utf16)); + _ = std.unicode.utf8ToUtf16LeWriter(utf16_buf_stream.writer(), name) catch |err| switch (err) { + // If the buffer isn't large enough, we can treat that as 'env var not found', as we + // know anything too large for the buffer can't be found in the map. + error.NoSpaceLeft => return null, + // Anything with invalid UTF-8 will also not be found in the map, so treat that as + // 'env var not found' too + error.InvalidUtf8 => return null, + }; + break :to_utf16 std.mem.bytesAsSlice(u16, utf16_buf_stream.getWritten()); + }; + + // uppercase in place + const uppercased_len = uppercaseName(name_utf16, name_utf16); + assert(uppercased_len == name_utf16.len); + + return name_utf16; + } + + /// Returns true if an entry was found and deleted, false otherwise. + pub fn remove(self: *Self, name: []const u8) bool { + const name_utf16 = self.utf8ToUppercasedUtf16(name) orelse return false; + const kv = self.uppercased_map.fetchRemove(std.mem.sliceAsBytes(name_utf16)) orelse return false; + self.allocator.free(kv.key); + self.allocator.free(kv.value.value); + self.allocator.free(kv.value.canonical_name); + return true; + } + + pub fn get(self: Self, name: []const u8) ?EnvValue { + const name_utf16 = self.utf8ToUppercasedUtf16(name) orelse return null; + return self.uppercased_map.get(std.mem.sliceAsBytes(name_utf16)); + } + + pub fn count(self: Self) EnvMap.Size { + return self.uppercased_map.count(); + } + + pub fn iterator(self: *const Self) Iterator { + return .{ + .env_map = self, + .uppercased_map_iterator = self.uppercased_map.iterator(), + }; + } + + pub const Iterator = struct { + env_map: *const Self, + uppercased_map_iterator: std.StringHashMapUnmanaged(EnvValue).Iterator, + + pub fn next(it: *Iterator) ?EnvMap.Entry { + if (it.uppercased_map_iterator.next()) |uppercased_entry| { + return EnvMap.Entry{ + .name = uppercased_entry.value_ptr.canonical_name, + .value = uppercased_entry.value_ptr.value, + }; + } else { + return null; + } + } + }; +}; + +test "EnvMapWindows" { + if (builtin.os.tag != .windows) return error.SkipZigTest; + + var env_map = EnvMapWindows.init(testing.allocator); + defer env_map.deinit(); + + // both put methods + try env_map.putUtf16NoClobber(std.unicode.utf8ToUtf16LeStringLiteral("Path"), std.unicode.utf8ToUtf16LeStringLiteral("something")); + try env_map.putUtf8("КИРИЛЛИЦА", "something else"); + try env_map.reallocUppercaseBuf(); + + try testing.expectEqual(@as(EnvMap.Size, 2), env_map.count()); + + // unicode-aware case-insensitive lookups + try testing.expectEqualStrings("something", env_map.get("PATH").?.value); + try testing.expectEqualStrings("something else", env_map.get("кириллица").?.value); + try testing.expect(env_map.get("missing") == null); + + // canonical names when iterating + var it = env_map.iterator(); + var count: EnvMap.Size = 0; + while (it.next()) |entry| { + const is_an_expected_name = std.mem.eql(u8, "Path", entry.name) or std.mem.eql(u8, "КИРИЛЛИЦА", entry.name); + try testing.expect(is_an_expected_name); + count += 1; + } + try testing.expectEqual(@as(EnvMap.Size, 2), count); +} + +pub const EnvMap = struct { + storage: StorageType, + + pub const StorageType = switch (builtin.os.tag) { + .windows => EnvMapWindows, + else => std.BufMap, + }; + + /// Matches what BufMap uses for its internal HashMap Size + pub const Size = u32; + + const Self = @This(); + + /// Deinitialize with `deinit`. + pub fn init(allocator: Allocator) Self { + return Self{ .storage = StorageType.init(allocator) }; + } + + pub fn deinit(self: *Self) void { + self.storage.deinit(); + } + + pub fn get(self: Self, name: []const u8) ?[]const u8 { + switch (builtin.os.tag) { + .windows => { + if (self.storage.get(name)) |entry| { + return entry.value; + } else { + return null; + } + }, + else => return self.storage.get(name), + } + } + + pub fn count(self: Self) Size { + return self.storage.count(); + } + + pub fn iterator(self: *const Self) Iterator { + return .{ .storage_iterator = self.storage.iterator() }; + } + + pub fn put(self: *Self, name: []const u8, value: []const u8) !void { + switch (builtin.os.tag) { + .windows => { + try self.storage.putUtf8(name, value); + try self.storage.reallocUppercaseBuf(); + }, + else => return self.storage.put(name, value), + } + } + + pub fn remove(self: *Self, name: []const u8) void { + _ = self.storage.remove(name); + } + + pub const Entry = struct { + name: []const u8, + value: []const u8, + }; + + pub const Iterator = struct { + storage_iterator: switch (builtin.os.tag) { + .windows => EnvMapWindows.Iterator, + else => std.BufMap.BufMapHashMap.Iterator, + }, + + pub fn next(it: *Iterator) ?Entry { + switch (builtin.os.tag) { + .windows => return it.storage_iterator.next(), + else => { + if (it.storage_iterator.next()) |entry| { + return Entry{ + .name = entry.key_ptr.*, + .value = entry.value_ptr.*, + }; + } else { + return null; + } + }, + } + } + }; +}; + +test "EnvMap" { + var env = EnvMap.init(testing.allocator); + defer env.deinit(); + + try env.put("SOMETHING_NEW", "hello"); + try testing.expectEqualStrings("hello", env.get("SOMETHING_NEW").?); + try testing.expectEqual(@as(EnvMap.Size, 1), env.count()); + + // overwrite + try env.put("SOMETHING_NEW", "something"); + try testing.expectEqualStrings("something", env.get("SOMETHING_NEW").?); + try testing.expectEqual(@as(EnvMap.Size, 1), env.count()); + + // a new longer name to test the Windows-specific conversion buffer + try env.put("SOMETHING_NEW_AND_LONGER", "1"); + try testing.expectEqualStrings("1", env.get("SOMETHING_NEW_AND_LONGER").?); + try testing.expectEqual(@as(EnvMap.Size, 2), env.count()); + + // case insensitivity on Windows only + if (builtin.os.tag == .windows) { + try testing.expectEqualStrings("1", env.get("something_New_aNd_LONGER").?); + } else { + try testing.expect(null == env.get("something_New_aNd_LONGER")); + } + + var it = env.iterator(); + var count: EnvMap.Size = 0; + while (it.next()) |entry| { + const is_an_expected_name = std.mem.eql(u8, "SOMETHING_NEW", entry.name) or std.mem.eql(u8, "SOMETHING_NEW_AND_LONGER", entry.name); + try testing.expect(is_an_expected_name); + count += 1; + } + try testing.expectEqual(@as(EnvMap.Size, 2), count); + + env.remove("SOMETHING_NEW"); + try testing.expect(env.get("SOMETHING_NEW") == null); + + try testing.expectEqual(@as(EnvMap.Size, 1), env.count()); +} + +/// Returns a snapshot of the environment variables of the current process. +/// Any modifications to the resulting EnvMap will not be not reflected in the environment, and +/// likewise, any future modifications to the environment will not be reflected in the EnvMap. +/// Caller owns resulting `EnvMap` and should call its `deinit` fn when done. +pub fn getEnvMap(allocator: Allocator) !EnvMap { + var result = EnvMap.init(allocator); errdefer result.deinit(); if (builtin.os.tag == .windows) { @@ -65,23 +440,27 @@ pub fn getEnvMap(allocator: Allocator) !BufMap { while (ptr[i] != 0) { const key_start = i; + // There are some special environment variables that start with =, + // so we need a special case to not treat = as a key/value separator + // if it's the first character. + // https://devblogs.microsoft.com/oldnewthing/20100506-00/?p=14133 + if (ptr[key_start] == '=') i += 1; + while (ptr[i] != 0 and ptr[i] != '=') : (i += 1) {} const key_w = ptr[key_start..i]; - const key = try std.unicode.utf16leToUtf8Alloc(allocator, key_w); - errdefer allocator.free(key); if (ptr[i] == '=') i += 1; const value_start = i; while (ptr[i] != 0) : (i += 1) {} const value_w = ptr[value_start..i]; - const value = try std.unicode.utf16leToUtf8Alloc(allocator, value_w); - errdefer allocator.free(value); - i += 1; // skip over null byte + try result.storage.putUtf16NoClobber(key_w, value_w); - try result.putMove(key, value); + i += 1; // skip over null byte } + + try result.storage.reallocUppercaseBuf(); return result; } else if (builtin.os.tag == .wasi and !builtin.link_libc) { var environ_count: usize = undefined; @@ -140,8 +519,8 @@ pub fn getEnvMap(allocator: Allocator) !BufMap { } } -test "os.getEnvMap" { - var env = try getEnvMap(std.testing.allocator); +test "getEnvMap" { + var env = try getEnvMap(testing.allocator); defer env.deinit(); } -- cgit v1.2.3 From 9e89000ffc92fd881ccb59d2571debe003a2f7b1 Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Sun, 6 Feb 2022 23:52:08 -0700 Subject: Update usages of `process.getEnvMap` and change BufMap -> EnvMap where applicable # Conflicts: # lib/std/build/RunStep.zig --- doc/docgen.zig | 4 ++-- lib/std/build.zig | 8 ++++---- lib/std/build/RunStep.zig | 25 +++++++------------------ lib/std/child_process.zig | 26 +++++++++++++------------- lib/std/process.zig | 2 +- 5 files changed, 27 insertions(+), 38 deletions(-) (limited to 'lib/std/process.zig') diff --git a/doc/docgen.zig b/doc/docgen.zig index 3dd58a012d..a101b96be7 100644 --- a/doc/docgen.zig +++ b/doc/docgen.zig @@ -1708,7 +1708,7 @@ fn genHtml( } } -fn exec(allocator: Allocator, env_map: *std.BufMap, args: []const []const u8) !ChildProcess.ExecResult { +fn exec(allocator: Allocator, env_map: *process.EnvMap, args: []const []const u8) !ChildProcess.ExecResult { const result = try ChildProcess.exec(.{ .allocator = allocator, .argv = args, @@ -1732,7 +1732,7 @@ fn exec(allocator: Allocator, env_map: *std.BufMap, args: []const []const u8) !C return result; } -fn getBuiltinCode(allocator: Allocator, env_map: *std.BufMap, zig_exe: []const u8) ![]const u8 { +fn getBuiltinCode(allocator: Allocator, env_map: *process.EnvMap, zig_exe: []const u8) ![]const u8 { const result = try exec(allocator, env_map, &[_][]const u8{ zig_exe, "build-obj", "--show-builtin" }); return result.stdout; } diff --git a/lib/std/build.zig b/lib/std/build.zig index b0200a928b..0f196cc9e9 100644 --- a/lib/std/build.zig +++ b/lib/std/build.zig @@ -12,7 +12,7 @@ const StringHashMap = std.StringHashMap; const Allocator = mem.Allocator; const process = std.process; const BufSet = std.BufSet; -const BufMap = std.BufMap; +const EnvMap = std.process.EnvMap; const fmt_lib = std.fmt; const File = std.fs.File; const CrossTarget = std.zig.CrossTarget; @@ -48,7 +48,7 @@ pub const Builder = struct { invalid_user_input: bool, zig_exe: []const u8, default_step: *Step, - env_map: *BufMap, + env_map: *EnvMap, top_level_steps: ArrayList(*TopLevelStep), install_prefix: []const u8, dest_dir: ?[]const u8, @@ -167,7 +167,7 @@ pub const Builder = struct { cache_root: []const u8, global_cache_root: []const u8, ) !*Builder { - const env_map = try allocator.create(BufMap); + const env_map = try allocator.create(EnvMap); env_map.* = try process.getEnvMap(allocator); const host = try NativeTargetInfo.detect(allocator, .{}); @@ -963,7 +963,7 @@ pub const Builder = struct { warn("\n", .{}); } - pub fn spawnChildEnvMap(self: *Builder, cwd: ?[]const u8, env_map: *const BufMap, argv: []const []const u8) !void { + pub fn spawnChildEnvMap(self: *Builder, cwd: ?[]const u8, env_map: *const EnvMap, argv: []const []const u8) !void { if (self.verbose) { printCmd(cwd, argv); } diff --git a/lib/std/build/RunStep.zig b/lib/std/build/RunStep.zig index e00fe3deb6..bc6ccd7b69 100644 --- a/lib/std/build/RunStep.zig +++ b/lib/std/build/RunStep.zig @@ -9,7 +9,7 @@ const fs = std.fs; const mem = std.mem; const process = std.process; const ArrayList = std.ArrayList; -const BufMap = std.BufMap; +const EnvMap = process.EnvMap; const Allocator = mem.Allocator; const ExecError = build.Builder.ExecError; @@ -29,7 +29,7 @@ argv: ArrayList(Arg), cwd: ?[]const u8, /// Override this field to modify the environment, or use setEnvironmentVariable -env_map: ?*BufMap, +env_map: ?*EnvMap, stdout_action: StdIoAction = .inherit, stderr_action: StdIoAction = .inherit, @@ -91,8 +91,8 @@ pub fn addArgs(self: *RunStep, args: []const []const u8) void { } pub fn clearEnvironment(self: *RunStep) void { - const new_env_map = self.builder.allocator.create(BufMap) catch unreachable; - new_env_map.* = BufMap.init(self.builder.allocator); + const new_env_map = self.builder.allocator.create(EnvMap) catch unreachable; + new_env_map.* = EnvMap.init(self.builder.allocator); self.env_map = new_env_map; } @@ -100,18 +100,7 @@ pub fn addPathDir(self: *RunStep, search_path: []const u8) void { const env_map = self.getEnvMap(); var key: []const u8 = undefined; - var prev_path: ?[]const u8 = undefined; - if (builtin.os.tag == .windows) { - key = "Path"; - prev_path = env_map.get(key); - if (prev_path == null) { - key = "PATH"; - prev_path = env_map.get(key); - } - } else { - key = "PATH"; - prev_path = env_map.get(key); - } + var prev_path = env_map.get("PATH"); if (prev_path) |pp| { const new_path = self.builder.fmt("{s}" ++ [1]u8{fs.path.delimiter} ++ "{s}", .{ pp, search_path }); @@ -121,9 +110,9 @@ pub fn addPathDir(self: *RunStep, search_path: []const u8) void { } } -pub fn getEnvMap(self: *RunStep) *BufMap { +pub fn getEnvMap(self: *RunStep) *EnvMap { return self.env_map orelse { - const env_map = self.builder.allocator.create(BufMap) catch unreachable; + const env_map = self.builder.allocator.create(EnvMap) catch unreachable; env_map.* = process.getEnvMap(self.builder.allocator) catch unreachable; self.env_map = env_map; return env_map; diff --git a/lib/std/child_process.zig b/lib/std/child_process.zig index 5f01ed01dd..48f0776465 100644 --- a/lib/std/child_process.zig +++ b/lib/std/child_process.zig @@ -12,7 +12,7 @@ const linux = os.linux; const mem = std.mem; const math = std.math; const debug = std.debug; -const BufMap = std.BufMap; +const EnvMap = process.EnvMap; const Os = std.builtin.Os; const TailQueue = std.TailQueue; const maxInt = std.math.maxInt; @@ -34,7 +34,7 @@ pub const ChildProcess = struct { argv: []const []const u8, /// Leave as null to use the current env map using the supplied allocator. - env_map: ?*const BufMap, + env_map: ?*const EnvMap, stdin_behavior: StdIo, stdout_behavior: StdIo, @@ -375,7 +375,7 @@ pub const ChildProcess = struct { argv: []const []const u8, cwd: ?[]const u8 = null, cwd_dir: ?fs.Dir = null, - env_map: ?*const BufMap = null, + env_map: ?*const EnvMap = null, max_output_bytes: usize = 50 * 1024, expand_arg0: Arg0Expand = .no_expand, }) !ExecResult { @@ -1237,7 +1237,7 @@ fn readIntFd(fd: i32) !ErrInt { } /// Caller must free result. -pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const BufMap) ![]u16 { +pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const EnvMap) ![]u16 { // count bytes needed const max_chars_needed = x: { var max_chars_needed: usize = 4; // 4 for the final 4 null bytes @@ -1245,7 +1245,7 @@ pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const BufMap) ! while (it.next()) |pair| { // +1 for '=' // +1 for null byte - max_chars_needed += pair.key_ptr.len + pair.value_ptr.len + 2; + max_chars_needed += pair.name.len + pair.value.len + 2; } break :x max_chars_needed; }; @@ -1255,10 +1255,10 @@ pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const BufMap) ! var it = env_map.iterator(); var i: usize = 0; while (it.next()) |pair| { - i += try unicode.utf8ToUtf16Le(result[i..], pair.key_ptr.*); + i += try unicode.utf8ToUtf16Le(result[i..], pair.name); result[i] = '='; i += 1; - i += try unicode.utf8ToUtf16Le(result[i..], pair.value_ptr.*); + i += try unicode.utf8ToUtf16Le(result[i..], pair.value); result[i] = 0; i += 1; } @@ -1273,17 +1273,17 @@ pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const BufMap) ! return allocator.shrink(result, i); } -pub fn createNullDelimitedEnvMap(arena: mem.Allocator, env_map: *const std.BufMap) ![:null]?[*:0]u8 { +pub fn createNullDelimitedEnvMap(arena: mem.Allocator, env_map: *const EnvMap) ![:null]?[*:0]u8 { const envp_count = env_map.count(); const envp_buf = try arena.allocSentinel(?[*:0]u8, envp_count, null); { var it = env_map.iterator(); var i: usize = 0; while (it.next()) |pair| : (i += 1) { - const env_buf = try arena.allocSentinel(u8, pair.key_ptr.len + pair.value_ptr.len + 1, 0); - mem.copy(u8, env_buf, pair.key_ptr.*); - env_buf[pair.key_ptr.len] = '='; - mem.copy(u8, env_buf[pair.key_ptr.len + 1 ..], pair.value_ptr.*); + const env_buf = try arena.allocSentinel(u8, pair.name.len + pair.value.len + 1, 0); + mem.copy(u8, env_buf, pair.name); + env_buf[pair.name.len] = '='; + mem.copy(u8, env_buf[pair.name.len + 1 ..], pair.value); envp_buf[i] = env_buf.ptr; } assert(i == envp_count); @@ -1294,7 +1294,7 @@ pub fn createNullDelimitedEnvMap(arena: mem.Allocator, env_map: *const std.BufMa test "createNullDelimitedEnvMap" { const testing = std.testing; const allocator = testing.allocator; - var envmap = BufMap.init(allocator); + var envmap = EnvMap.init(allocator); defer envmap.deinit(); try envmap.put("HOME", "/home/ifreund"); diff --git a/lib/std/process.zig b/lib/std/process.zig index f5d14cf6da..66a82c4b1d 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -1364,7 +1364,7 @@ pub fn execv(allocator: mem.Allocator, argv: []const []const u8) ExecvError { pub fn execve( allocator: mem.Allocator, argv: []const []const u8, - env_map: ?*const std.BufMap, + env_map: ?*const EnvMap, ) ExecvError { if (!can_execv) @compileError("The target OS does not support execv"); -- cgit v1.2.3 From b2b48fbf2cf84744e5d154b7d4f62b298f725f2b Mon Sep 17 00:00:00 2001 From: Ryan Liptak Date: Thu, 20 Jan 2022 07:19:16 -0800 Subject: Set EnvMap.Size to BufMap.BufMapHashMap.Size Now that BufMap.BufMapHashMap is pub, we can just get Size directly --- lib/std/process.zig | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/process.zig b/lib/std/process.zig index 66a82c4b1d..d7f6570e8b 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -307,8 +307,7 @@ pub const EnvMap = struct { else => std.BufMap, }; - /// Matches what BufMap uses for its internal HashMap Size - pub const Size = u32; + pub const Size = std.BufMap.BufMapHashMap.Size; const Self = @This(); -- cgit v1.2.3 From e70cb04f89005d08d837b062e8c10a3a1b406a6c Mon Sep 17 00:00:00 2001 From: Ryan Liptak Date: Thu, 3 Feb 2022 22:56:06 -0800 Subject: EnvMapWindows: Fix putUtf8 not uppercasing keys --- lib/std/process.zig | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/process.zig b/lib/std/process.zig index d7f6570e8b..712c8e9317 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -141,8 +141,13 @@ pub const EnvMapWindows = struct { var name_utf16_buf = try std.ArrayListAligned(u8, @alignOf(u16)).initCapacity(self.allocator, name.len); errdefer name_utf16_buf.deinit(); - var uppercased_len = try std.unicode.utf8ToUtf16LeWriter(name_utf16_buf.writer(), name); - assert(uppercased_len == name_utf16_buf.items.len); + const bytes_written = try std.unicode.utf8ToUtf16LeWriter(name_utf16_buf.writer(), name); + var name_utf16 = name_utf16_buf.items[0..bytes_written]; + + // uppercase in place + var name_uppercased_utf16 = std.mem.bytesAsSlice(u16, name_utf16); + const uppercased_len = uppercaseName(name_uppercased_utf16, name_uppercased_utf16); + assert(uppercased_len == name_uppercased_utf16.len); break :uppercased name_utf16_buf.toOwnedSlice(); }; @@ -278,7 +283,7 @@ test "EnvMapWindows" { // both put methods try env_map.putUtf16NoClobber(std.unicode.utf8ToUtf16LeStringLiteral("Path"), std.unicode.utf8ToUtf16LeStringLiteral("something")); - try env_map.putUtf8("КИРИЛЛИЦА", "something else"); + try env_map.putUtf8("КИРиллИЦА", "something else"); try env_map.reallocUppercaseBuf(); try testing.expectEqual(@as(EnvMap.Size, 2), env_map.count()); @@ -292,7 +297,7 @@ test "EnvMapWindows" { var it = env_map.iterator(); var count: EnvMap.Size = 0; while (it.next()) |entry| { - const is_an_expected_name = std.mem.eql(u8, "Path", entry.name) or std.mem.eql(u8, "КИРИЛЛИЦА", entry.name); + const is_an_expected_name = std.mem.eql(u8, "Path", entry.name) or std.mem.eql(u8, "КИРиллИЦА", entry.name); try testing.expect(is_an_expected_name); count += 1; } -- cgit v1.2.3 From 69f0a5587d0db07546e91968b21135fdef856136 Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Fri, 4 Feb 2022 12:08:38 -0700 Subject: remove extra storage from EnvMap on windows --- lib/std/child_process.zig | 14 +- lib/std/process.zig | 399 +++++++++++----------------------------------- 2 files changed, 100 insertions(+), 313 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/child_process.zig b/lib/std/child_process.zig index 48f0776465..0bb737decb 100644 --- a/lib/std/child_process.zig +++ b/lib/std/child_process.zig @@ -1245,7 +1245,7 @@ pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const EnvMap) ! while (it.next()) |pair| { // +1 for '=' // +1 for null byte - max_chars_needed += pair.name.len + pair.value.len + 2; + max_chars_needed += pair.key_ptr.len + pair.value_ptr.len + 2; } break :x max_chars_needed; }; @@ -1255,10 +1255,10 @@ pub fn createWindowsEnvBlock(allocator: mem.Allocator, env_map: *const EnvMap) ! var it = env_map.iterator(); var i: usize = 0; while (it.next()) |pair| { - i += try unicode.utf8ToUtf16Le(result[i..], pair.name); + i += try unicode.utf8ToUtf16Le(result[i..], pair.key_ptr.*); result[i] = '='; i += 1; - i += try unicode.utf8ToUtf16Le(result[i..], pair.value); + i += try unicode.utf8ToUtf16Le(result[i..], pair.value_ptr.*); result[i] = 0; i += 1; } @@ -1280,10 +1280,10 @@ pub fn createNullDelimitedEnvMap(arena: mem.Allocator, env_map: *const EnvMap) ! var it = env_map.iterator(); var i: usize = 0; while (it.next()) |pair| : (i += 1) { - const env_buf = try arena.allocSentinel(u8, pair.name.len + pair.value.len + 1, 0); - mem.copy(u8, env_buf, pair.name); - env_buf[pair.name.len] = '='; - mem.copy(u8, env_buf[pair.name.len + 1 ..], pair.value); + const env_buf = try arena.allocSentinel(u8, pair.key_ptr.len + pair.value_ptr.len + 1, 0); + mem.copy(u8, env_buf, pair.key_ptr.*); + env_buf[pair.key_ptr.len] = '='; + mem.copy(u8, env_buf[pair.key_ptr.len + 1 ..], pair.value_ptr.*); envp_buf[i] = env_buf.ptr; } assert(i == envp_count); diff --git a/lib/std/process.zig b/lib/std/process.zig index 712c8e9317..0b891bbdf5 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -52,341 +52,128 @@ test "getCwdAlloc" { testing.allocator.free(cwd); } -/// EnvMap for Windows that handles Unicode-aware case insensitivity for lookups, while also -/// providing the canonical environment variable names when iterating. -/// -/// Allows for zero-allocation lookups (even though it needs to do UTF-8 -> UTF-16 -> uppercase -/// conversions) by allocating a buffer large enough to fit the largest environment variable -/// name, and using that when doing lookups (i.e. anything that overflows the buffer can be treated -/// as the environment variable not being found). -pub const EnvMapWindows = struct { - allocator: Allocator, - /// Keys are UTF-16le stored as []const u8 - uppercased_map: std.StringHashMapUnmanaged(EnvValue), - /// Buffer for converting to uppercased UTF-16 on key lookups - /// Must call `reallocUppercaseBuf` before doing any lookups after a `put` call. - uppercase_buf_utf16: []u16 = &[_]u16{}, - max_name_utf16_length: usize = 0, - - pub const EnvValue = struct { - value: []const u8, - canonical_name: []const u8, +pub const EnvMap = struct { + hash_map: HashMap, + + const HashMap = std.HashMap( + []const u8, + []const u8, + EnvNameHashContext, + std.hash_map.default_max_load_percentage, + ); + + pub const EnvNameHashContext = struct { + pub fn hash(self: @This(), s: []const u8) u64 { + _ = self; + if (builtin.os.tag == .windows) { + const h = std.hash.Wyhash.init(0); + // TODO: improve this, instead of iterating over ascii, + // iterate over with unicode + for (s) |c| { + var s_upper = [_]u8 { std.ascii.toLower(c) }; + h.update(s_upper); + } + return h.final(); + } + return std.hash_map.hashString(s); + } + pub fn eql(self: @This(), a: []const u8, b: []const u8) bool { + _ = self; + if (builtin.os.tag == .windows) { + // TODO: improve this, instead of comparing ascii + // compare with unicode + return std.ascii.eqlIgnoreCase(a, b); + } + return std.hash_map.eqlString(a, b); + } }; - const Self = @This(); - - /// Deinitialize with `deinit`. - pub fn init(allocator: Allocator) Self { - return .{ - .allocator = allocator, - .uppercased_map = std.StringHashMapUnmanaged(EnvValue){}, - }; + /// Create a EnvMap backed by a specific allocator. + /// That allocator will be used for both backing allocations + /// and string deduplication. + pub fn init(allocator: Allocator) EnvMap { + return EnvMap{ .hash_map = HashMap.init(allocator) }; } - pub fn deinit(self: *Self) void { - var it = self.uppercased_map.iterator(); + /// Free the backing storage of the map, as well as all + /// of the stored keys and values. + pub fn deinit(self: *EnvMap) void { + var it = self.hash_map.iterator(); while (it.next()) |entry| { - self.allocator.free(entry.key_ptr.*); - self.allocator.free(entry.value_ptr.value); - self.allocator.free(entry.value_ptr.canonical_name); + self.free(entry.key_ptr.*); + self.free(entry.value_ptr.*); } - self.uppercased_map.deinit(self.allocator); - self.allocator.free(self.uppercase_buf_utf16); - } - /// Increases the size of the uppercase buffer if the maximum name size has increased. - /// Must be called before any `get` calls after any number of `put` calls. - pub fn reallocUppercaseBuf(self: *Self) !void { - if (self.max_name_utf16_length > self.uppercase_buf_utf16.len) { - self.uppercase_buf_utf16 = try self.allocator.realloc(self.uppercase_buf_utf16, self.max_name_utf16_length); - } + self.hash_map.deinit(); } - /// Converts `src` to uppercase using `RtlUpcaseUnicodeString` and puts the result in `dest`. - /// Returns the length of the converted UTF-16 string. `dest.len` must be >= `src.len`. - /// - /// Note: As of now, RtlUpcaseUnicodeString does not seem to handle codepoints above 0x10000 - /// (i.e. those that require a surrogate pair), so this function will always return a length - /// equal to `src.len`. However, if RtlUpcaseUnicodeString is updated to handle codepoints above - /// 0x10000, this property would still hold unless there are lowercase <-> uppercase conversions - /// that cross over the boundary between codepoints >= 0x10000 and < 0x10000. - /// TODO: Is it feasible that Unicode lowercase <-> uppercase conversions could cross that boundary? - fn uppercaseName(dest: []u16, src: []const u16) u16 { - assert(dest.len >= src.len); - - const dest_bytes = @intCast(u16, dest.len * 2); - var dest_string = os.windows.UNICODE_STRING{ - .Length = dest_bytes, - .MaximumLength = dest_bytes, - .Buffer = @intToPtr([*]u16, @ptrToInt(dest.ptr)), - }; - const src_bytes = @intCast(u16, src.len * 2); - const src_string = os.windows.UNICODE_STRING{ - .Length = src_bytes, - .MaximumLength = src_bytes, - .Buffer = @intToPtr([*]u16, @ptrToInt(src.ptr)), - }; - const rc = os.windows.ntdll.RtlUpcaseUnicodeString(&dest_string, &src_string, os.windows.FALSE); - switch (rc) { - .SUCCESS => return dest_string.Length / 2, - else => unreachable, // we are not allocating, so no errors should be possible + /// Same as `put` but the key and value become owned by the EnvMap rather + /// than being copied. + /// If `putMove` fails, the ownership of key and value does not transfer. + pub fn putMove(self: *EnvMap, key: []u8, value: []u8) !void { + const get_or_put = try self.hash_map.getOrPut(key); + if (get_or_put.found_existing) { + self.free(get_or_put.key_ptr.*); + self.free(get_or_put.value_ptr.*); + get_or_put.key_ptr.* = key; } + get_or_put.value_ptr.* = value; } - /// Note: Does not realloc the uppercase buf to allow for calling put for many variables and - /// only allocating the uppercase buf afterwards. - pub fn putUtf8(self: *Self, name: []const u8, value: []const u8) !void { - const uppercased_len = len: { - const name_uppercased_utf16 = uppercased: { - var name_utf16_buf = try std.ArrayListAligned(u8, @alignOf(u16)).initCapacity(self.allocator, name.len); - errdefer name_utf16_buf.deinit(); - - const bytes_written = try std.unicode.utf8ToUtf16LeWriter(name_utf16_buf.writer(), name); - var name_utf16 = name_utf16_buf.items[0..bytes_written]; - - // uppercase in place - var name_uppercased_utf16 = std.mem.bytesAsSlice(u16, name_utf16); - const uppercased_len = uppercaseName(name_uppercased_utf16, name_uppercased_utf16); - assert(uppercased_len == name_uppercased_utf16.len); - - break :uppercased name_utf16_buf.toOwnedSlice(); - }; - errdefer self.allocator.free(name_uppercased_utf16); - - const name_canonical = try self.allocator.dupe(u8, name); - errdefer self.allocator.free(name_canonical); - - const value_dupe = try self.allocator.dupe(u8, value); - errdefer self.allocator.free(value_dupe); - - const get_or_put = try self.uppercased_map.getOrPut(self.allocator, name_uppercased_utf16); - if (get_or_put.found_existing) { - // note: this is only safe from UAF because the errdefer that frees this value above - // no longer has a possibility of being triggered after this point - self.allocator.free(name_uppercased_utf16); - self.allocator.free(get_or_put.value_ptr.value); - self.allocator.free(get_or_put.value_ptr.canonical_name); - } else { - get_or_put.key_ptr.* = name_uppercased_utf16; - } - get_or_put.value_ptr.value = value_dupe; - get_or_put.value_ptr.canonical_name = name_canonical; - - break :len name_uppercased_utf16.len; - }; - - // The buffer for case conversion for key lookups will need to be as big as the largest - // key stored in the hash map. - self.max_name_utf16_length = @maximum(self.max_name_utf16_length, uppercased_len); - } - - /// Asserts that the name does not already exist in the map. - /// Note: Does not realloc the uppercase buf to allow for calling put for many variables and - /// only allocating the uppercase buf afterwards. - pub fn putUtf16NoClobber(self: *Self, name_utf16: []const u16, value_utf16: []const u16) !void { - const uppercased_len = len: { - const name_canonical = try std.unicode.utf16leToUtf8Alloc(self.allocator, name_utf16); - errdefer self.allocator.free(name_canonical); - - const value = try std.unicode.utf16leToUtf8Alloc(self.allocator, value_utf16); - errdefer self.allocator.free(value); - - const name_uppercased_utf16 = try self.allocator.alloc(u16, name_utf16.len); - errdefer self.allocator.free(name_uppercased_utf16); - - const uppercased_len = uppercaseName(name_uppercased_utf16, name_utf16); - assert(uppercased_len == name_uppercased_utf16.len); - - try self.uppercased_map.putNoClobber(self.allocator, std.mem.sliceAsBytes(name_uppercased_utf16), EnvValue{ - .value = value, - .canonical_name = name_canonical, - }); - break :len name_uppercased_utf16.len; - }; - - // The buffer for case conversion for key lookups will need to be as big as the largest - // key stored in the hash map. - self.max_name_utf16_length = @maximum(self.max_name_utf16_length, uppercased_len); - } - - /// Attempts to convert a UTF-8 name into a uppercased UTF-16le name for a lookup. If the - /// name cannot be converted, this function will return `null`. - fn utf8ToUppercasedUtf16(self: Self, name: []const u8) ?[]u16 { - const name_utf16: []u16 = to_utf16: { - var utf16_buf_stream = std.io.fixedBufferStream(std.mem.sliceAsBytes(self.uppercase_buf_utf16)); - _ = std.unicode.utf8ToUtf16LeWriter(utf16_buf_stream.writer(), name) catch |err| switch (err) { - // If the buffer isn't large enough, we can treat that as 'env var not found', as we - // know anything too large for the buffer can't be found in the map. - error.NoSpaceLeft => return null, - // Anything with invalid UTF-8 will also not be found in the map, so treat that as - // 'env var not found' too - error.InvalidUtf8 => return null, + /// `key` and `value` are copied into the EnvMap. + pub fn put(self: *EnvMap, key: []const u8, value: []const u8) !void { + const value_copy = try self.copy(value); + errdefer self.free(value_copy); + const get_or_put = try self.hash_map.getOrPut(key); + if (get_or_put.found_existing) { + self.free(get_or_put.value_ptr.*); + } else { + get_or_put.key_ptr.* = self.copy(key) catch |err| { + _ = self.hash_map.remove(key); + return err; }; - break :to_utf16 std.mem.bytesAsSlice(u16, utf16_buf_stream.getWritten()); - }; - - // uppercase in place - const uppercased_len = uppercaseName(name_utf16, name_utf16); - assert(uppercased_len == name_utf16.len); - - return name_utf16; - } - - /// Returns true if an entry was found and deleted, false otherwise. - pub fn remove(self: *Self, name: []const u8) bool { - const name_utf16 = self.utf8ToUppercasedUtf16(name) orelse return false; - const kv = self.uppercased_map.fetchRemove(std.mem.sliceAsBytes(name_utf16)) orelse return false; - self.allocator.free(kv.key); - self.allocator.free(kv.value.value); - self.allocator.free(kv.value.canonical_name); - return true; - } - - pub fn get(self: Self, name: []const u8) ?EnvValue { - const name_utf16 = self.utf8ToUppercasedUtf16(name) orelse return null; - return self.uppercased_map.get(std.mem.sliceAsBytes(name_utf16)); - } - - pub fn count(self: Self) EnvMap.Size { - return self.uppercased_map.count(); - } - - pub fn iterator(self: *const Self) Iterator { - return .{ - .env_map = self, - .uppercased_map_iterator = self.uppercased_map.iterator(), - }; - } - - pub const Iterator = struct { - env_map: *const Self, - uppercased_map_iterator: std.StringHashMapUnmanaged(EnvValue).Iterator, - - pub fn next(it: *Iterator) ?EnvMap.Entry { - if (it.uppercased_map_iterator.next()) |uppercased_entry| { - return EnvMap.Entry{ - .name = uppercased_entry.value_ptr.canonical_name, - .value = uppercased_entry.value_ptr.value, - }; - } else { - return null; - } } - }; -}; - -test "EnvMapWindows" { - if (builtin.os.tag != .windows) return error.SkipZigTest; - - var env_map = EnvMapWindows.init(testing.allocator); - defer env_map.deinit(); - - // both put methods - try env_map.putUtf16NoClobber(std.unicode.utf8ToUtf16LeStringLiteral("Path"), std.unicode.utf8ToUtf16LeStringLiteral("something")); - try env_map.putUtf8("КИРиллИЦА", "something else"); - try env_map.reallocUppercaseBuf(); - - try testing.expectEqual(@as(EnvMap.Size, 2), env_map.count()); - - // unicode-aware case-insensitive lookups - try testing.expectEqualStrings("something", env_map.get("PATH").?.value); - try testing.expectEqualStrings("something else", env_map.get("кириллица").?.value); - try testing.expect(env_map.get("missing") == null); - - // canonical names when iterating - var it = env_map.iterator(); - var count: EnvMap.Size = 0; - while (it.next()) |entry| { - const is_an_expected_name = std.mem.eql(u8, "Path", entry.name) or std.mem.eql(u8, "КИРиллИЦА", entry.name); - try testing.expect(is_an_expected_name); - count += 1; + get_or_put.value_ptr.* = value_copy; } - try testing.expectEqual(@as(EnvMap.Size, 2), count); -} - -pub const EnvMap = struct { - storage: StorageType, - - pub const StorageType = switch (builtin.os.tag) { - .windows => EnvMapWindows, - else => std.BufMap, - }; - pub const Size = std.BufMap.BufMapHashMap.Size; - - const Self = @This(); - - /// Deinitialize with `deinit`. - pub fn init(allocator: Allocator) Self { - return Self{ .storage = StorageType.init(allocator) }; + /// Find the address of the value associated with a key. + /// The returned pointer is invalidated if the map resizes. + pub fn getPtr(self: EnvMap, key: []const u8) ?*[]const u8 { + return self.hash_map.getPtr(key); } - pub fn deinit(self: *Self) void { - self.storage.deinit(); + /// Return the map's copy of the value associated with + /// a key. The returned string is invalidated if this + /// key is removed from the map. + pub fn get(self: EnvMap, key: []const u8) ?[]const u8 { + return self.hash_map.get(key); } - pub fn get(self: Self, name: []const u8) ?[]const u8 { - switch (builtin.os.tag) { - .windows => { - if (self.storage.get(name)) |entry| { - return entry.value; - } else { - return null; - } - }, - else => return self.storage.get(name), - } + /// Removes the item from the map and frees its value. + /// This invalidates the value returned by get() for this key. + pub fn remove(self: *EnvMap, key: []const u8) void { + const kv = self.hash_map.fetchRemove(key) orelse return; + self.free(kv.key); + self.free(kv.value); } - pub fn count(self: Self) Size { - return self.storage.count(); + /// Returns the number of KV pairs stored in the map. + pub fn count(self: EnvMap) HashMap.Size { + return self.hash_map.count(); } - pub fn iterator(self: *const Self) Iterator { - return .{ .storage_iterator = self.storage.iterator() }; + /// Returns an iterator over entries in the map. + pub fn iterator(self: *const EnvMap) HashMap.Iterator { + return self.hash_map.iterator(); } - pub fn put(self: *Self, name: []const u8, value: []const u8) !void { - switch (builtin.os.tag) { - .windows => { - try self.storage.putUtf8(name, value); - try self.storage.reallocUppercaseBuf(); - }, - else => return self.storage.put(name, value), - } + fn free(self: EnvMap, value: []const u8) void { + self.hash_map.allocator.free(value); } - pub fn remove(self: *Self, name: []const u8) void { - _ = self.storage.remove(name); + fn copy(self: EnvMap, value: []const u8) ![]u8 { + return self.hash_map.allocator.dupe(u8, value); } - - pub const Entry = struct { - name: []const u8, - value: []const u8, - }; - - pub const Iterator = struct { - storage_iterator: switch (builtin.os.tag) { - .windows => EnvMapWindows.Iterator, - else => std.BufMap.BufMapHashMap.Iterator, - }, - - pub fn next(it: *Iterator) ?Entry { - switch (builtin.os.tag) { - .windows => return it.storage_iterator.next(), - else => { - if (it.storage_iterator.next()) |entry| { - return Entry{ - .name = entry.key_ptr.*, - .value = entry.value_ptr.*, - }; - } else { - return null; - } - }, - } - } - }; }; test "EnvMap" { -- cgit v1.2.3 From e65d8f82c5f98b20236f571fe4a4e40924ea8dc2 Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Fri, 4 Feb 2022 22:36:24 -0700 Subject: add unicode support --- lib/std/os/windows/ntdll.zig | 4 ++++ lib/std/process.zig | 32 ++++++++++++++++++++++++-------- 2 files changed, 28 insertions(+), 8 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/os/windows/ntdll.zig b/lib/std/os/windows/ntdll.zig index 2444d5f487..5817e482e7 100644 --- a/lib/std/os/windows/ntdll.zig +++ b/lib/std/os/windows/ntdll.zig @@ -235,6 +235,10 @@ pub extern "NtDll" fn RtlUpcaseUnicodeString( AllocateDestinationString: BOOLEAN, ) callconv(WINAPI) NTSTATUS; +pub extern "NtDll" fn RtlUpcaseUnicodeChar( + SourceCharacter: u16, +) callconv(WINAPI) u16; + pub extern "ntdll" fn NtLockFile( FileHandle: HANDLE, Event: ?HANDLE, diff --git a/lib/std/process.zig b/lib/std/process.zig index 0b891bbdf5..8892f1cc88 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -63,26 +63,42 @@ pub const EnvMap = struct { ); pub const EnvNameHashContext = struct { + fn upcase(c: u21) u21 { + if (c <= std.math.maxInt(u16)) + return std.os.windows.ntdll.RtlUpcaseUnicodeChar(c); + return c; + } + pub fn hash(self: @This(), s: []const u8) u64 { _ = self; if (builtin.os.tag == .windows) { const h = std.hash.Wyhash.init(0); - // TODO: improve this, instead of iterating over ascii, - // iterate over with unicode - for (s) |c| { - var s_upper = [_]u8 { std.ascii.toLower(c) }; - h.update(s_upper); + var it = std.unicode.Utf8View(s).iterator(); + while (it.nextCodepoint()) |cp| { + const cp_upper = upcase(cp); + h.update(&[_]u8{ + @intCast(u8, (cp_upper >> 16) & 0xff), + @intCast(u8, (cp_upper >> 8) & 0xff), + @intCast(u8, (cp_upper >> 0) & 0xff), + }); } return h.final(); } return std.hash_map.hashString(s); } + pub fn eql(self: @This(), a: []const u8, b: []const u8) bool { _ = self; if (builtin.os.tag == .windows) { - // TODO: improve this, instead of comparing ascii - // compare with unicode - return std.ascii.eqlIgnoreCase(a, b); + var it_a = std.unicode.Utf8View(a).iterator(); + var it_b = std.unicode.Utf8View(b).iterator(); + while (true) { + const c_a = it_a.nextCodepoint() orelse break; + const c_b = it_b.nextCodepoint() orelse return false; + if (upcase(c_a) != upcase(c_b)) + return false; + } + if (it_b.nextCodepoint()) return false; } return std.hash_map.eqlString(a, b); } -- cgit v1.2.3 From 1c874a871fe4795f71edf6e9739b72d071eb453a Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Fri, 4 Feb 2022 23:35:22 -0700 Subject: reverse some of the now unneeded changes from squeek --- lib/std/buf_map.zig | 2 +- lib/std/process.zig | 10 ++++++---- lib/std/unicode.zig | 23 ----------------------- 3 files changed, 7 insertions(+), 28 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/buf_map.zig b/lib/std/buf_map.zig index 5d155747d2..2a6239c490 100644 --- a/lib/std/buf_map.zig +++ b/lib/std/buf_map.zig @@ -9,7 +9,7 @@ const testing = std.testing; pub const BufMap = struct { hash_map: BufMapHashMap, - pub const BufMapHashMap = StringHashMap([]const u8); + const BufMapHashMap = StringHashMap([]const u8); /// Create a BufMap backed by a specific allocator. /// That allocator will be used for both backing allocations diff --git a/lib/std/process.zig b/lib/std/process.zig index 8892f1cc88..26cde9db51 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -255,19 +255,21 @@ pub fn getEnvMap(allocator: Allocator) !EnvMap { while (ptr[i] != 0 and ptr[i] != '=') : (i += 1) {} const key_w = ptr[key_start..i]; + const key = try std.unicode.utf16leToUtf8Alloc(allocator, key_w); + errdefer allocator.free(key); if (ptr[i] == '=') i += 1; const value_start = i; while (ptr[i] != 0) : (i += 1) {} const value_w = ptr[value_start..i]; - - try result.storage.putUtf16NoClobber(key_w, value_w); + const value = try std.unicode.utf16leToUtf8Alloc(allocator, value_w); + errdefer allocator.free(value); i += 1; // skip over null byte - } - try result.storage.reallocUppercaseBuf(); + try result.putMove(key, value); + } return result; } else if (builtin.os.tag == .wasi and !builtin.link_libc) { var environ_count: usize = undefined; diff --git a/lib/std/unicode.zig b/lib/std/unicode.zig index 706b12105a..81a7ed838f 100644 --- a/lib/std/unicode.zig +++ b/lib/std/unicode.zig @@ -710,29 +710,6 @@ pub fn utf8ToUtf16Le(utf16le: []u16, utf8: []const u8) !usize { return dest_i; } -pub fn utf8ToUtf16LeWriter(writer: anytype, utf8: []const u8) !usize { - var src_i: usize = 0; - var bytes_written: usize = 0; - while (src_i < utf8.len) { - const n = utf8ByteSequenceLength(utf8[src_i]) catch return error.InvalidUtf8; - const next_src_i = src_i + n; - const codepoint = utf8Decode(utf8[src_i..next_src_i]) catch return error.InvalidUtf8; - if (codepoint < 0x10000) { - const short = @intCast(u16, codepoint); - try writer.writeIntLittle(u16, short); - bytes_written += 2; - } else { - const high = @intCast(u16, (codepoint - 0x10000) >> 10) + 0xD800; - const low = @intCast(u16, codepoint & 0x3FF) + 0xDC00; - try writer.writeIntLittle(u16, high); - try writer.writeIntLittle(u16, low); - bytes_written += 4; - } - src_i = next_src_i; - } - return bytes_written; -} - test "utf8ToUtf16Le" { var utf16le: [2]u16 = [_]u16{0} ** 2; { -- cgit v1.2.3 From 71f69190ef7a79650647cf5422633813756fbe48 Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Fri, 4 Feb 2022 23:42:10 -0700 Subject: some fixes to the EnvMap HashContext --- lib/std/build/RunStep.zig | 2 +- lib/std/process.zig | 16 +++++++++------- 2 files changed, 10 insertions(+), 8 deletions(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/build/RunStep.zig b/lib/std/build/RunStep.zig index 1e4cca5167..e8cf87a441 100644 --- a/lib/std/build/RunStep.zig +++ b/lib/std/build/RunStep.zig @@ -99,7 +99,7 @@ pub fn clearEnvironment(self: *RunStep) void { pub fn addPathDir(self: *RunStep, search_path: []const u8) void { const env_map = self.getEnvMap(); - var key: []const u8 = "PATH"; + const key = "PATH"; var prev_path = env_map.get(key); if (prev_path) |pp| { diff --git a/lib/std/process.zig b/lib/std/process.zig index 26cde9db51..b460264a8c 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -62,18 +62,20 @@ pub const EnvMap = struct { std.hash_map.default_max_load_percentage, ); + pub const Size = HashMap.Size; + pub const EnvNameHashContext = struct { fn upcase(c: u21) u21 { if (c <= std.math.maxInt(u16)) - return std.os.windows.ntdll.RtlUpcaseUnicodeChar(c); + return std.os.windows.ntdll.RtlUpcaseUnicodeChar(@intCast(u16, c)); return c; } pub fn hash(self: @This(), s: []const u8) u64 { _ = self; if (builtin.os.tag == .windows) { - const h = std.hash.Wyhash.init(0); - var it = std.unicode.Utf8View(s).iterator(); + var h = std.hash.Wyhash.init(0); + var it = std.unicode.Utf8View.initUnchecked(s).iterator(); while (it.nextCodepoint()) |cp| { const cp_upper = upcase(cp); h.update(&[_]u8{ @@ -90,15 +92,15 @@ pub const EnvMap = struct { pub fn eql(self: @This(), a: []const u8, b: []const u8) bool { _ = self; if (builtin.os.tag == .windows) { - var it_a = std.unicode.Utf8View(a).iterator(); - var it_b = std.unicode.Utf8View(b).iterator(); + var it_a = std.unicode.Utf8View.initUnchecked(a).iterator(); + var it_b = std.unicode.Utf8View.initUnchecked(b).iterator(); while (true) { const c_a = it_a.nextCodepoint() orelse break; const c_b = it_b.nextCodepoint() orelse return false; if (upcase(c_a) != upcase(c_b)) return false; } - if (it_b.nextCodepoint()) return false; + if (it_b.nextCodepoint()) |_| return false; } return std.hash_map.eqlString(a, b); } @@ -220,7 +222,7 @@ test "EnvMap" { var it = env.iterator(); var count: EnvMap.Size = 0; while (it.next()) |entry| { - const is_an_expected_name = std.mem.eql(u8, "SOMETHING_NEW", entry.name) or std.mem.eql(u8, "SOMETHING_NEW_AND_LONGER", entry.name); + const is_an_expected_name = std.mem.eql(u8, "SOMETHING_NEW", entry.key_ptr.*) or std.mem.eql(u8, "SOMETHING_NEW_AND_LONGER", entry.key_ptr.*); try testing.expect(is_an_expected_name); count += 1; } -- cgit v1.2.3 From 8492ced0755924b3c4c0d67fb306d65bffd0933a Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Sun, 6 Feb 2022 23:30:06 -0700 Subject: incorporate review changes from squeek --- lib/std/process.zig | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) (limited to 'lib/std/process.zig') diff --git a/lib/std/process.zig b/lib/std/process.zig index b460264a8c..84d41972b5 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -100,7 +100,7 @@ pub const EnvMap = struct { if (upcase(c_a) != upcase(c_b)) return false; } - if (it_b.nextCodepoint()) |_| return false; + return if (it_b.nextCodepoint()) |_| false else true; } return std.hash_map.eqlString(a, b); } @@ -232,6 +232,12 @@ test "EnvMap" { try testing.expect(env.get("SOMETHING_NEW") == null); try testing.expectEqual(@as(EnvMap.Size, 1), env.count()); + + // test Unicode case-insensitivity on Windows + if (builtin.os.tag == .windows) { + try env.put("КИРиллИЦА", "something else"); + try testing.expectEqualStrings("something else", env.get("кириллица").?); + } } /// Returns a snapshot of the environment variables of the current process. -- cgit v1.2.3 From a38e6a64d32974bebb1c338dac5e56a35c8bcac4 Mon Sep 17 00:00:00 2001 From: Jonathan Marler Date: Fri, 18 Feb 2022 14:58:13 -0700 Subject: document that on Windows, all key arguments in EnvMap must be valid utf8 --- lib/std/process.zig | 5 +++++ 1 file changed, 5 insertions(+) (limited to 'lib/std/process.zig') diff --git a/lib/std/process.zig b/lib/std/process.zig index 84d41972b5..0b64b5910d 100644 --- a/lib/std/process.zig +++ b/lib/std/process.zig @@ -128,6 +128,7 @@ pub const EnvMap = struct { /// Same as `put` but the key and value become owned by the EnvMap rather /// than being copied. /// If `putMove` fails, the ownership of key and value does not transfer. + /// On Windows `key` must be a valid UTF-8 string. pub fn putMove(self: *EnvMap, key: []u8, value: []u8) !void { const get_or_put = try self.hash_map.getOrPut(key); if (get_or_put.found_existing) { @@ -139,6 +140,7 @@ pub const EnvMap = struct { } /// `key` and `value` are copied into the EnvMap. + /// On Windows `key` must be a valid UTF-8 string. pub fn put(self: *EnvMap, key: []const u8, value: []const u8) !void { const value_copy = try self.copy(value); errdefer self.free(value_copy); @@ -156,6 +158,7 @@ pub const EnvMap = struct { /// Find the address of the value associated with a key. /// The returned pointer is invalidated if the map resizes. + /// On Windows `key` must be a valid UTF-8 string. pub fn getPtr(self: EnvMap, key: []const u8) ?*[]const u8 { return self.hash_map.getPtr(key); } @@ -163,12 +166,14 @@ pub const EnvMap = struct { /// Return the map's copy of the value associated with /// a key. The returned string is invalidated if this /// key is removed from the map. + /// On Windows `key` must be a valid UTF-8 string. pub fn get(self: EnvMap, key: []const u8) ?[]const u8 { return self.hash_map.get(key); } /// Removes the item from the map and frees its value. /// This invalidates the value returned by get() for this key. + /// On Windows `key` must be a valid UTF-8 string. pub fn remove(self: *EnvMap, key: []const u8) void { const kv = self.hash_map.fetchRemove(key) orelse return; self.free(kv.key); -- cgit v1.2.3