Skip to content

Commit

Permalink
[#25582] Use a Pathing Jar instead of long command line class paths i…
Browse files Browse the repository at this point in the history
…n Java Boot loader. (#26087)
  • Loading branch information
lostluck committed Apr 11, 2023
1 parent 50c6d04 commit e0e1398
Show file tree
Hide file tree
Showing 5 changed files with 235 additions and 46 deletions.
4 changes: 2 additions & 2 deletions .github/workflows/go_tests.yml
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ on:
pull_request:
branches: ['master', 'release-*']
tags: ['v*']
paths: ['sdks/go/pkg/**', 'sdks/go.mod', 'sdks/go.sum']
paths: ['sdks/go/pkg/**', 'sdks/go.mod', 'sdks/go.sum', 'sdks/go/container/*', 'sdks/java/container/*', 'sdks/python/container/*', 'sdks/typescript/container/*']
# This allows a subsequently queued workflow run to interrupt previous runs
concurrency:
group: '${{ github.workflow }} @ ${{ github.event.pull_request.head.label || github.head_ref || github.ref }}'
Expand All @@ -48,7 +48,7 @@ jobs:
- name: Delete old coverage
run: "cd sdks/go/pkg && rm -rf .coverage || :"
- name: Run coverage
run: cd sdks/go/pkg && go test -coverprofile=coverage.txt -covermode=atomic ./...
run: cd sdks && go test -coverprofile=coverage.txt -covermode=atomic ./go/pkg/... ./go/container/... ./java/container/... ./python/container/... ./typescript/container/...
- uses: codecov/codecov-action@v3
with:
flags: go
Expand Down
10 changes: 7 additions & 3 deletions sdks/java/container/boot.go
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,6 @@ func main() {
"-XX:+UseParallelGC",
"-XX:+AlwaysActAsServerClassMachine",
"-XX:-OmitStackTraceInFastThrow",
"-cp", strings.Join(cp, ":"),
}

enableGoogleCloudProfiler := strings.Contains(options, enableGoogleCloudProfilerOption)
Expand Down Expand Up @@ -210,9 +209,14 @@ func main() {

// (2) Add classpath: "-cp foo.jar:bar.jar:.."
if len(javaOptions.Classpath) > 0 {
args = append(args, "-cp")
args = append(args, strings.Join(javaOptions.Classpath, ":"))
cp = append(cp, javaOptions.Classpath...)
}
pathingjar, err := makePathingJar(cp)
if err != nil {
logger.Fatalf(ctx, "makePathingJar failed: %v", err)
}
args = append(args, "-cp")
args = append(args, pathingjar)

// (3) Add (sorted) properties: "-Dbar=baz -Dfoo=bar .."
var properties []string
Expand Down
88 changes: 47 additions & 41 deletions sdks/java/container/boot_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,57 +18,63 @@
package main

import (
"reflect"
"testing"
"context"
"reflect"
"testing"

"github.com/apache/beam/sdks/v2/go/container/tools"
)

func TestBuildOptionsEmpty(t *testing.T) {
dir := "test/empty"
metaOptions, err := LoadMetaOptions(dir)
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}
if metaOptions != nil {
t.Fatalf("LoadMetaOptions(%v) = %v, want nil", dir, metaOptions)
}
ctx, logger := context.Background(), &tools.Logger{}
dir := "test/empty"
metaOptions, err := LoadMetaOptions(ctx, logger, dir)
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}
if metaOptions != nil {
t.Fatalf("LoadMetaOptions(%v) = %v, want nil", dir, metaOptions)
}

javaOptions := BuildOptions(metaOptions)
if len(javaOptions.JavaArguments) != 0 || len(javaOptions.Classpath) != 0 || len(javaOptions.Properties) != 0 {
t.Errorf("BuildOptions(%v) = %v, want nil", metaOptions, javaOptions)
}
javaOptions := BuildOptions(ctx, logger, metaOptions)
if len(javaOptions.JavaArguments) != 0 || len(javaOptions.Classpath) != 0 || len(javaOptions.Properties) != 0 {
t.Errorf("BuildOptions(%v) = %v, want nil", metaOptions, javaOptions)
}
}

func TestBuildOptionsDisabled(t *testing.T) {
metaOptions, err := LoadMetaOptions("test/disabled")
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}
ctx, logger := context.Background(), &tools.Logger{}
metaOptions, err := LoadMetaOptions(ctx, logger, "test/disabled")
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}

javaOptions := BuildOptions(metaOptions)
if len(javaOptions.JavaArguments) != 0 || len(javaOptions.Classpath) != 0 || len(javaOptions.Properties) != 0 {
t.Errorf("BuildOptions(%v) = %v, want nil", metaOptions, javaOptions)
}
javaOptions := BuildOptions(ctx, logger, metaOptions)
if len(javaOptions.JavaArguments) != 0 || len(javaOptions.Classpath) != 0 || len(javaOptions.Properties) != 0 {
t.Errorf("BuildOptions(%v) = %v, want nil", metaOptions, javaOptions)
}
}

func TestBuildOptions(t *testing.T) {
metaOptions, err := LoadMetaOptions("test/priority")
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}
ctx, logger := context.Background(), &tools.Logger{}
metaOptions, err := LoadMetaOptions(ctx, logger, "test/priority")
if err != nil {
t.Fatalf("Got error %v running LoadMetaOptions", err)
}

javaOptions := BuildOptions(metaOptions)
wantJavaArguments := []string{"java_args=low", "java_args=high"}
wantClasspath := []string{"classpath_high", "classpath_low"}
wantProperties := map[string]string{
"priority":"high",
}
if !reflect.DeepEqual(javaOptions.JavaArguments, wantJavaArguments) {
t.Errorf("BuildOptions(%v).JavaArguments = %v, want %v", metaOptions, javaOptions.JavaArguments, wantJavaArguments)
}
if !reflect.DeepEqual(javaOptions.Classpath, wantClasspath) {
t.Errorf("BuildOptions(%v).Classpath = %v, want %v", metaOptions, javaOptions.Classpath, wantClasspath)
}
if !reflect.DeepEqual(javaOptions.Properties, wantProperties) {
t.Errorf("BuildOptions(%v).JavaProperties = %v, want %v", metaOptions, javaOptions.Properties, wantProperties)
}
javaOptions := BuildOptions(ctx, logger, metaOptions)
wantJavaArguments := []string{"java_args=low", "java_args=high"}
wantClasspath := []string{"classpath_high", "classpath_low"}
wantProperties := map[string]string{
"priority": "high",
}
if !reflect.DeepEqual(javaOptions.JavaArguments, wantJavaArguments) {
t.Errorf("BuildOptions(%v).JavaArguments = %v, want %v", metaOptions, javaOptions.JavaArguments, wantJavaArguments)
}
if !reflect.DeepEqual(javaOptions.Classpath, wantClasspath) {
t.Errorf("BuildOptions(%v).Classpath = %v, want %v", metaOptions, javaOptions.Classpath, wantClasspath)
}
if !reflect.DeepEqual(javaOptions.Properties, wantProperties) {
t.Errorf("BuildOptions(%v).JavaProperties = %v, want %v", metaOptions, javaOptions.Properties, wantProperties)
}
}
91 changes: 91 additions & 0 deletions sdks/java/container/pathingjar.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,91 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package main

import (
"archive/zip"
"fmt"
"io"
"os"
"strings"
)

// makePathingJar produces a context or 'pathing' jar which only contains the relative
// classpaths in its META-INF/MANIFEST.MF.
//
// Since we build with Java 8 as a minimum, this is the only supported way of reducing
// command line length, since argsfile support wasn't added until Java 9.
//
// https://docs.oracle.com/javase/8/docs/technotes/guides/jar/jar.html is the spec.
//
// In particular, we only need to populate the Jar with a Manifest-Version and Class-Path
// attributes.
// Per https://docs.oracle.com/javase/8/docs/technotes/guides/jar/jar.html#classpath
// the classpath URLs must be relative for security reasons.
func makePathingJar(classpaths []string) (string, error) {
f, err := os.Create("pathing.jar")
if err != nil {
return "", fmt.Errorf("unable to create pathing.jar: %w", err)
}
defer f.Close()
if err := writePathingJar(classpaths, f); err != nil {
return "", fmt.Errorf("unable to write pathing.jar: %w", err)
}
return f.Name(), nil
}

var lineBreak = []byte{'\r', '\n'}
var continuation = []byte{' '}

func writePathingJar(classpaths []string, w io.Writer) error {
jar := zip.NewWriter(w)
defer jar.Close()

if _, err := jar.Create("META-INF/"); err != nil {
return fmt.Errorf("unable to create META-INF/ directory: %w", err)
}

zf, err := jar.Create("META-INF/MANIFEST.MF")
if err != nil {
return fmt.Errorf("unable to create META-INF/MANIFEST.MF: %w", err)
}

zf.Write([]byte("Manifest-Version: 1.0"))
zf.Write(lineBreak)
zf.Write([]byte("Created-By: sdks/java/container/pathingjar.go"))
zf.Write(lineBreak)
// Class-Path: must have a sequence of relative URIs for the paths
// which we assume outright in this case.

// We could do this memory efficiently, but it's not worthwhile compared to the complexity
// at this stage.
allCPs := "Class-Path: file:" + strings.Join(classpaths, " file:")

const lineLenMax = 71 // it's actually 72, but we remove 1 to account for the continuation line prefix.
buf := make([]byte, lineLenMax)
cur := 0
for cur+lineLenMax < len(allCPs) {
next := cur + lineLenMax
copy(buf, allCPs[cur:next])
zf.Write(buf)
zf.Write(lineBreak)
zf.Write(continuation)
cur = next
}
zf.Write([]byte(allCPs[cur:]))
zf.Write(lineBreak)
return nil
}
88 changes: 88 additions & 0 deletions sdks/java/container/pathingjar_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Licensed to the Apache Software Foundation (ASF) under one or more
// contributor license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright ownership.
// The ASF licenses this file to You under the Apache License, Version 2.0
// (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package main

import (
"archive/zip"
"bufio"
"bytes"
"io"
"strings"
"testing"
)

func TestWritePathingJar(t *testing.T) {
var buf bytes.Buffer
input := []string{"a.jar", "b.jar", "c.jar", "thisVeryLongJarNameIsOverSeventyTwoCharactersLongAndNeedsToBeSplitCorrectly.jar"}
err := writePathingJar(input, &buf)
if err != nil {
t.Errorf("writePathingJar(%v) = %v, want nil", input, err)
}

zr, err := zip.NewReader(bytes.NewReader(buf.Bytes()), int64(buf.Len()))
if err != nil {
t.Errorf("zip.NewReader() = %v, want nil", err)
}

f := getManifest(zr)
if f == nil {
t.Fatalf("Jar didn't contain manifest")
}

{
fr, err := f.Open()
if err != nil {
t.Errorf("(%v).Open() = %v, want nil", f.Name, err)
}
all, err := io.ReadAll(fr)
if err != nil {
t.Errorf("(%v).Open() = %v, want nil", f.Name, err)
}
fr.Close()
want := "\nClass-Path: file:a.jar file:b.jar file:c.jar"
if !strings.Contains(string(all), want) {
t.Errorf("%v = %v, want nil", f.Name, err)
}
}

{
fr, err := f.Open()
if err != nil {
t.Errorf("(%v).Open() = %v, want nil", f.Name, err)
}
defer fr.Close()
fs := bufio.NewScanner(fr)
fs.Split(bufio.ScanLines)

for fs.Scan() {
if got, want := len(fs.Bytes()), 72; got > want {
t.Errorf("Manifest line exceeds limit got %v:, want %v line: %q", got, want, fs.Text())
}
}
}

}

// getManifest extracts the java manifest from the zip file.
func getManifest(zr *zip.Reader) *zip.File {
for _, f := range zr.File {
if f.Name != "META-INF/MANIFEST.MF" {
continue
}
return f
}
return nil
}

0 comments on commit e0e1398

Please sign in to comment.