You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@yunikorn.apache.org by pb...@apache.org on 2024/01/10 20:55:14 UTC

(yunikorn-k8shim) branch master updated: [YUNIKORN-2305] E2E test: Upload stdout logs to Github Action artifact (#758)

This is an automated email from the ASF dual-hosted git repository.

pbacsko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/yunikorn-k8shim.git


The following commit(s) were added to refs/heads/master by this push:
     new ce8ac51d [YUNIKORN-2305] E2E test: Upload stdout logs to Github Action artifact (#758)
ce8ac51d is described below

commit ce8ac51de190ac90b3e1dbcd71db4015c7f6e29c
Author: Yu-Lin Chen <kh...@gmail.com>
AuthorDate: Wed Jan 10 21:54:56 2024 +0100

    [YUNIKORN-2305] E2E test: Upload stdout logs to Github Action artifact (#758)
    
    Closes: #758
    
    Signed-off-by: Peter Bacsko <pb...@cloudera.com>
---
 .github/workflows/pre-commit.yml                   |   6 +
 .../admission_controller_suite_test.go             |   4 +
 .../admission_controller_test.go                   |   3 +
 test/e2e/basic_scheduling/basic_scheduling_test.go |  10 +-
 test/e2e/bin_packing/bin_packing_suite_test.go     |   4 +
 test/e2e/bin_packing/bin_packing_test.go           |   7 +-
 test/e2e/framework/configmanager/constants.go      |  10 +-
 test/e2e/framework/helpers/common/utils.go         |  64 ++++++++---
 test/e2e/framework/helpers/k8s/k8s_utils.go        |  47 ++++++--
 .../framework/helpers/yunikorn/rest_api_utils.go   |  23 ++++
 .../gang_scheduling/gang_scheduling_suite_test.go  |   4 +
 test/e2e/gang_scheduling/gang_scheduling_test.go   |   6 +-
 .../node_resources/node_resources_suite_test.go    |   4 +
 test/e2e/node_resources/node_resources_test.go     |   7 +-
 .../persistent_volume/persistent_volume_test.go    |   4 +
 test/e2e/predicates/predicates_suite_test.go       |   4 +
 test/e2e/predicates/predicates_test.go             |   7 +-
 test/e2e/preemption/preemption_test.go             |  10 +-
 .../priority_scheduling_suite_test.go              |   4 +
 .../priority_scheduling_test.go                    |   6 +-
 .../queue_quota_mgmt_suite_test.go                 |   4 +
 test/e2e/queue_quota_mgmt/queue_quota_mgmt_test.go |   8 +-
 .../recovery_and_restart_test.go                   |   4 +
 .../resource_fairness_suite_test.go                |   4 +
 .../resource_fairness/resource_fairness_test.go    |   7 +-
 test/e2e/simple_preemptor/simple_preemptor_test.go |  19 ++--
 .../spark_jobs_scheduling_suite_test.go            |   4 +
 .../spark_jobs_scheduling_test.go                  |   7 +-
 .../drip_feed_schedule_test.go                     |   8 +-
 .../state_aware_app_scheduling/fallback_test.go    |   8 +-
 .../state_aware_app_scheduling_suite_test.go       |   4 +
 test/e2e/user_group_limit/user_group_limit_test.go |  11 +-
 test/e2e/wrappers.go                               | 126 ++++++++++++---------
 33 files changed, 287 insertions(+), 161 deletions(-)

diff --git a/.github/workflows/pre-commit.yml b/.github/workflows/pre-commit.yml
index bfeb1a88..4c896332 100644
--- a/.github/workflows/pre-commit.yml
+++ b/.github/workflows/pre-commit.yml
@@ -68,3 +68,9 @@ jobs:
         env:
           KIND_NODE_IMAGE: ${{ matrix.k8s }}
           KIND_EXTRA_ARGS: ${{ matrix.plugin }}
+      - name: Upload artifacts
+        uses: actions/upload-artifact@v4
+        if: ${{ failure() }}
+        with:
+          name: ${{ github.job }} stdout (${{ matrix.k8s }}${{ matrix.plugin == '--plugin' && format(', {0}', matrix.plugin) || matrix.plugin }}) 
+          path: build/e2e
diff --git a/test/e2e/admission_controller/admission_controller_suite_test.go b/test/e2e/admission_controller/admission_controller_suite_test.go
index 7f874c96..047ac12c 100644
--- a/test/e2e/admission_controller/admission_controller_suite_test.go
+++ b/test/e2e/admission_controller/admission_controller_suite_test.go
@@ -21,6 +21,7 @@ package admission_controller_test
 import (
 	"fmt"
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	. "github.com/onsi/ginkgo/v2"
@@ -45,6 +46,7 @@ func init() {
 
 const appName = "sleep"
 
+var suiteName string
 var kubeClient k8s.KubeCtl
 var ns string
 var bypassNs = "kube-system"
@@ -188,6 +190,8 @@ func TestAdmissionController(t *testing.T) {
 }
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	restClient = yunikorn.RClient{}
 
 	kubeClient = k8s.KubeCtl{}
diff --git a/test/e2e/admission_controller/admission_controller_test.go b/test/e2e/admission_controller/admission_controller_test.go
index 27b3bb56..82097f6a 100644
--- a/test/e2e/admission_controller/admission_controller_test.go
+++ b/test/e2e/admission_controller/admission_controller_test.go
@@ -30,6 +30,7 @@ import (
 
 	amConf "github.com/apache/yunikorn-k8shim/pkg/admission/conf"
 	"github.com/apache/yunikorn-k8shim/pkg/common/constants"
+	tests "github.com/apache/yunikorn-k8shim/test/e2e"
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/configmanager"
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/common"
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/k8s"
@@ -448,6 +449,8 @@ var _ = ginkgo.Describe("AdmissionController", func() {
 	})
 
 	ginkgo.AfterEach(func() {
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
+
 		ginkgo.By("Tear down namespace: " + ns)
 		err := kubeClient.TearDownNamespace(ns)
 		gomega.Ω(err).NotTo(gomega.HaveOccurred())
diff --git a/test/e2e/basic_scheduling/basic_scheduling_test.go b/test/e2e/basic_scheduling/basic_scheduling_test.go
index 67765360..cca1674b 100644
--- a/test/e2e/basic_scheduling/basic_scheduling_test.go
+++ b/test/e2e/basic_scheduling/basic_scheduling_test.go
@@ -19,6 +19,7 @@
 package basicscheduling_test
 
 import (
+	"runtime"
 	"time"
 
 	"github.com/onsi/ginkgo/v2"
@@ -32,6 +33,7 @@ import (
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
 )
 
+var suiteName string
 var kClient k8s.KubeCtl
 var restClient yunikorn.RClient
 var sleepRespPod *v1.Pod
@@ -44,6 +46,8 @@ var oldConfigMap = new(v1.ConfigMap)
 var sleepPodConfigs = k8s.SleepPodConfig{Name: "sleepjob", NS: dev}
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	gomega.Ω(kClient.SetClient()).To(gomega.BeNil())
@@ -119,11 +123,7 @@ var _ = ginkgo.Describe("", func() {
 	})
 
 	ginkgo.AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{dev})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{dev})
 		// call the healthCheck api to check scheduler health
 		ginkgo.By("Check Yunikorn's health")
 		checks, err := yunikorn.GetFailedHealthChecks()
diff --git a/test/e2e/bin_packing/bin_packing_suite_test.go b/test/e2e/bin_packing/bin_packing_suite_test.go
index 8bd63525..9bc30b87 100644
--- a/test/e2e/bin_packing/bin_packing_suite_test.go
+++ b/test/e2e/bin_packing/bin_packing_suite_test.go
@@ -20,6 +20,7 @@ package bin_packing
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -53,11 +54,14 @@ func TestBinPacking(t *testing.T) {
 	ginkgo.RunSpecs(t, "TestBinPacking", ginkgo.Label("TestBinPacking"))
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 var kClient = k8s.KubeCtl{} //nolint
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 
 	Ω(kClient.SetClient()).To(BeNil())
 	/* Sample configMap. Post-update, Yunikorn will use binpacking node sort and fair app sort
diff --git a/test/e2e/bin_packing/bin_packing_test.go b/test/e2e/bin_packing/bin_packing_test.go
index fa9fa3af..43b8ec6c 100644
--- a/test/e2e/bin_packing/bin_packing_test.go
+++ b/test/e2e/bin_packing/bin_packing_test.go
@@ -23,7 +23,6 @@ import (
 	"sort"
 	"time"
 
-	"github.com/onsi/ginkgo/v2"
 	v1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/api/resource"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
@@ -193,11 +192,7 @@ var _ = Describe("", func() {
 	})
 
 	AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
 		By("Tear down namespace: " + ns)
 		err := kClient.TearDownNamespace(ns)
 		Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/framework/configmanager/constants.go b/test/e2e/framework/configmanager/constants.go
index e66fb0a1..0eeeda08 100644
--- a/test/e2e/framework/configmanager/constants.go
+++ b/test/e2e/framework/configmanager/constants.go
@@ -18,16 +18,11 @@
 
 package configmanager
 
-import (
-	"os"
-)
-
 const (
 	TestResultsPath = "test_results/"
 
-	// LogPerm is the permission for files that are created by this framework
-	// that contain logs, outputs etc
-	LogPerm = os.FileMode(0666)
+	// LogPath is the path to store the dumped log files and should be equal to the artifact path set in pre-commit.yaml.
+	LogPath = "build/e2e/"
 
 	YKScheduler          = "yunikorn-scheduler"
 	YKSchedulerContainer = "yunikorn-scheduler-k8s"
@@ -50,6 +45,7 @@ const (
 	GroupUsagePath    = "ws/v1/partition/%s/usage/group/%s"
 	HealthCheckPath   = "ws/v1/scheduler/healthcheck"
 	ValidateConfPath  = "ws/v1/validate-conf"
+	FullStateDumpPath = "ws/v1/fullstatedump"
 
 	// YuniKorn Service Details
 	DefaultYuniKornHost   = "localhost"
diff --git a/test/e2e/framework/helpers/common/utils.go b/test/e2e/framework/helpers/common/utils.go
index 789bddde..7d72c46f 100644
--- a/test/e2e/framework/helpers/common/utils.go
+++ b/test/e2e/framework/helpers/common/utils.go
@@ -89,20 +89,6 @@ func CreateJUnitReportDir() error {
 	return err
 }
 
-// CreateLogFile creates the ReportDirectory if it is not present, writes the
-// given testdata to the given filename.
-func CreateLogFile(filename string, data []byte) error {
-	path, err := CreateReportDirectory()
-	if err != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "ReportDirectory cannot be created: %v\n", err)
-		return err
-	}
-
-	finalPath := filepath.Join(path, filename)
-	err = os.WriteFile(finalPath, data, configmanager.LogPerm)
-	return err
-}
-
 func GetFileContents(filename string) ([]byte, error) {
 	data, err := os.ReadFile(filename)
 	return data, err
@@ -254,3 +240,53 @@ func RunShellCmdForeground(cmdStr string) (string, error) {
 
 	return stdOutStream.String(), nil
 }
+
+func CreateLogFile(suiteName string, specName string, logType string, extension string) (*os.File, error) {
+	filePath, err := getLogFilePath(suiteName, specName, logType, extension)
+	if err != nil {
+		return nil, err
+	}
+
+	dir := filepath.Dir(filePath)
+	if _, err = os.Stat(dir); os.IsNotExist(err) {
+		err = os.MkdirAll(dir, 0755)
+		if err != nil {
+			return nil, err
+		}
+	}
+
+	file, err := os.Create(filePath)
+	if err != nil {
+		return nil, err
+	}
+	fmt.Fprintf(ginkgo.GinkgoWriter, "Created log file: %s\n", filePath)
+	return file, nil
+}
+
+func getLogFilePath(suiteName string, specName string, logType string, extension string) (string, error) {
+	gitRoot, runErr := RunShellCmdForeground("git rev-parse --show-toplevel")
+	if runErr != nil {
+		return "", runErr
+	}
+	gitRoot = strings.TrimSpace(gitRoot)
+	suiteName = replaceInvalidFileChars(strings.TrimSpace(suiteName))
+	specName = replaceInvalidFileChars(strings.TrimSpace(specName))
+
+	dumpLogFilePath := filepath.Join(gitRoot, configmanager.LogPath, suiteName, fmt.Sprintf("%s_%s.%s", specName, logType, extension))
+	return dumpLogFilePath, nil
+}
+
+func replaceInvalidFileChars(str string) string {
+	// some charaters are not allowed in upload-artifact : https://github.com/actions/upload-artifact/issues/333
+	invalidChars := []string{"\"", ":", "<", ">", "|", "*", "?", "\r", "\n"}
+	for _, char := range invalidChars {
+		str = strings.ReplaceAll(str, char, "_")
+	}
+	return str
+}
+
+func GetSuiteName(testFilePath string) string {
+	dir := filepath.Dir(testFilePath)
+	suiteName := filepath.Base(dir)
+	return suiteName
+}
diff --git a/test/e2e/framework/helpers/k8s/k8s_utils.go b/test/e2e/framework/helpers/k8s/k8s_utils.go
index 6d3f9fc2..749831d1 100644
--- a/test/e2e/framework/helpers/k8s/k8s_utils.go
+++ b/test/e2e/framework/helpers/k8s/k8s_utils.go
@@ -631,14 +631,47 @@ func GetConfigMapObj(yamlPath string) (*v1.ConfigMap, error) {
 	return c.(*v1.ConfigMap), err
 }
 
-func LogNamespaceInfo(ns string) error {
-	fmt.Fprintf(ginkgo.GinkgoWriter, "Log namespace info from %s\n", ns)
+func (k *KubeCtl) LogNamespaceInfo(file *os.File, ns string) error {
+	fmt.Fprintf(file, "Log namespace info, ns: %s\n", ns)
 	cmd := fmt.Sprintf("kubectl cluster-info dump --namespaces=%s", ns)
 	out, runErr := common.RunShellCmdForeground(cmd)
 	if runErr != nil {
 		return runErr
 	}
-	ginkgo.By("Cluster dump output:\n" + out)
+	_, err := fmt.Fprintln(file, out)
+	return err
+}
+
+func (k *KubeCtl) LogPodsInfo(file *os.File) error {
+	fmt.Fprintln(file, "Log pods info:")
+	pods, err := k.GetPodsByOptions(metav1.ListOptions{})
+	if err != nil {
+		return err
+	} else {
+		fmt.Fprintf(file, "Pod count is %d\n", len(pods.Items))
+		for _, pod := range pods.Items {
+			fmt.Fprintf(file, "Pod name is %s\n", pod.Name)
+			fmt.Fprintf(file, "Pod details: %s\n", pod.String())
+		}
+	}
+	return nil
+}
+
+func (k *KubeCtl) LogNodesInfo(file *os.File) error {
+	fmt.Fprintln(file, "Log nodes info:")
+	nodes, err := k.GetNodes()
+	if err != nil {
+		return err
+	}
+	fmt.Fprintf(file, "Node count is %d\n", len(nodes.Items))
+	for _, node := range nodes.Items {
+		fmt.Fprintf(file, "Node: %s\n", node.Name)
+		nodeInfo, err := k.DescribeNode(node)
+		if err != nil {
+			fmt.Fprintf(file, "Failed to describe node: %s, err: %v\n", node.Name, err)
+		}
+		fmt.Fprintln(file, nodeInfo)
+	}
 	return nil
 }
 
@@ -1443,15 +1476,13 @@ func (k *KubeCtl) GetNodesAvailRes(nodes v1.NodeList) map[string]v1.ResourceList
 	return nodeAvailRes
 }
 
-// DescribeNode Describe Node
-func (k *KubeCtl) DescribeNode(node v1.Node) error {
+func (k *KubeCtl) DescribeNode(node v1.Node) (string, error) {
 	cmd := "kubectl describe node " + node.Name
 	out, runErr := common.RunShellCmdForeground(cmd)
 	if runErr != nil {
-		return runErr
+		return "", runErr
 	}
-	ginkgo.By("describe output for node is:\n" + out)
-	return nil
+	return out, nil
 }
 
 func (k *KubeCtl) SetNodeLabel(name, key, value string) error {
diff --git a/test/e2e/framework/helpers/yunikorn/rest_api_utils.go b/test/e2e/framework/helpers/yunikorn/rest_api_utils.go
index a2b0c54b..732807d2 100644
--- a/test/e2e/framework/helpers/yunikorn/rest_api_utils.go
+++ b/test/e2e/framework/helpers/yunikorn/rest_api_utils.go
@@ -87,6 +87,19 @@ func (c *RClient) do(req *http.Request, v interface{}) (*http.Response, error) {
 	return resp, err
 }
 
+func (c *RClient) getBody(req *http.Request) (string, error) {
+	resp, err := c.httpClient.Do(req)
+	if err != nil {
+		return "", err
+	}
+	defer resp.Body.Close()
+	body, err := io.ReadAll(resp.Body)
+	if err != nil {
+		return "", err
+	}
+	return string(body), nil
+}
+
 func (c *RClient) GetQueues(partition string) (*dao.PartitionQueueDAOInfo, error) {
 	req, err := c.newRequest("GET", fmt.Sprintf(configmanager.QueuesPath, partition), nil)
 	if err != nil {
@@ -214,6 +227,16 @@ func (c *RClient) GetAllocationLog(partition string, queueName string, appID str
 	return nil, errors.New("allocation is empty")
 }
 
+func (c *RClient) GetFullStateDump() (string, error) {
+	req, err := c.newRequest("GET", configmanager.FullStateDumpPath, nil)
+	if err != nil {
+		return "", err
+	}
+
+	fullStateDump, err := c.getBody(req)
+	return fullStateDump, err
+}
+
 func (c *RClient) isAllocLogPresent(partition string, queueName string, appID string, podName string) wait.ConditionFunc {
 	return func() (bool, error) {
 		log, err := c.GetAllocationLog(partition, queueName, appID, podName)
diff --git a/test/e2e/gang_scheduling/gang_scheduling_suite_test.go b/test/e2e/gang_scheduling/gang_scheduling_suite_test.go
index cbe85fa2..1a73ac06 100644
--- a/test/e2e/gang_scheduling/gang_scheduling_suite_test.go
+++ b/test/e2e/gang_scheduling/gang_scheduling_suite_test.go
@@ -20,6 +20,7 @@ package gangscheduling_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -52,11 +53,14 @@ func TestGangScheduling(t *testing.T) {
 	ginkgo.RunSpecs(t, "TestGangScheduling", ginkgo.Label("TestGangScheduling"))
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 var kClient = k8s.KubeCtl{} //nolint
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "fifo", annotation)
diff --git a/test/e2e/gang_scheduling/gang_scheduling_test.go b/test/e2e/gang_scheduling/gang_scheduling_test.go
index 46d722df..8fd4ecec 100644
--- a/test/e2e/gang_scheduling/gang_scheduling_test.go
+++ b/test/e2e/gang_scheduling/gang_scheduling_test.go
@@ -571,11 +571,7 @@ var _ = Describe("", func() {
 	})
 
 	AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
 
 		By(fmt.Sprintf("Cleanup jobs: %v", jobNames))
 		for _, jobName := range jobNames {
diff --git a/test/e2e/node_resources/node_resources_suite_test.go b/test/e2e/node_resources/node_resources_suite_test.go
index abef9c43..de1b95ef 100644
--- a/test/e2e/node_resources/node_resources_suite_test.go
+++ b/test/e2e/node_resources/node_resources_suite_test.go
@@ -20,6 +20,7 @@ package node_resources_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -51,10 +52,13 @@ func TestNodeResources(t *testing.T) {
 	ginkgo.RunSpecs(t, "TestNodeResources", ginkgo.Label("TestNodeResources"))
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "", annotation)
diff --git a/test/e2e/node_resources/node_resources_test.go b/test/e2e/node_resources/node_resources_test.go
index 9087dfd6..61c7689f 100644
--- a/test/e2e/node_resources/node_resources_test.go
+++ b/test/e2e/node_resources/node_resources_test.go
@@ -21,7 +21,6 @@ package node_resources_test
 import (
 	"fmt"
 
-	"github.com/onsi/ginkgo/v2"
 	v1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/api/resource"
 
@@ -48,11 +47,7 @@ var _ = Describe("", func() {
 	})
 
 	AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
 
 		By("Tear down namespace: " + ns)
 		err := kClient.TearDownNamespace(ns)
diff --git a/test/e2e/persistent_volume/persistent_volume_test.go b/test/e2e/persistent_volume/persistent_volume_test.go
index e641f793..f131dc89 100644
--- a/test/e2e/persistent_volume/persistent_volume_test.go
+++ b/test/e2e/persistent_volume/persistent_volume_test.go
@@ -19,6 +19,7 @@
 package persistent_volume
 
 import (
+	"runtime"
 	"time"
 
 	"github.com/onsi/gomega"
@@ -38,6 +39,7 @@ import (
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
 )
 
+var suiteName string
 var kClient k8s.KubeCtl
 var restClient yunikorn.RClient
 var dev = "dev-" + common.RandSeq(5)
@@ -48,6 +50,8 @@ const (
 )
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	Ω(kClient.SetClient()).To(gomega.BeNil())
diff --git a/test/e2e/predicates/predicates_suite_test.go b/test/e2e/predicates/predicates_suite_test.go
index 6a4c1f55..36d543f0 100644
--- a/test/e2e/predicates/predicates_suite_test.go
+++ b/test/e2e/predicates/predicates_suite_test.go
@@ -20,6 +20,7 @@ package predicates_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -36,10 +37,13 @@ func init() {
 	configmanager.YuniKornTestConfig.ParseFlags()
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation string
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "fifo", annotation)
diff --git a/test/e2e/predicates/predicates_test.go b/test/e2e/predicates/predicates_test.go
index 53c1b05d..6ffb1b00 100644
--- a/test/e2e/predicates/predicates_test.go
+++ b/test/e2e/predicates/predicates_test.go
@@ -108,13 +108,10 @@ var _ = Describe("Predicates", func() {
 	})
 
 	AfterEach(func() {
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns, anotherNS})
+
 		By("Cleanup")
 		for _, n := range []string{ns, anotherNS} {
-			testDescription := ginkgo.CurrentSpecReport()
-			if testDescription.Failed() {
-				tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{n})
-				tests.LogYunikornContainer(testDescription.FailureMessage())
-			}
 			ginkgo.By("Tear down namespace: " + n)
 			err = kClient.TearDownNamespace(n)
 			Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/preemption/preemption_test.go b/test/e2e/preemption/preemption_test.go
index 7140b965..bf3cb4ce 100644
--- a/test/e2e/preemption/preemption_test.go
+++ b/test/e2e/preemption/preemption_test.go
@@ -20,6 +20,7 @@ package preemption_test
 
 import (
 	"fmt"
+	"runtime"
 	"strings"
 	"time"
 
@@ -37,6 +38,7 @@ import (
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
 )
 
+var suiteName string
 var kClient k8s.KubeCtl
 var restClient yunikorn.RClient
 var ns *v1.Namespace
@@ -53,6 +55,8 @@ var taintKey = "e2e_test_preemption"
 var nodesToTaint []string
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	Ω(kClient.SetClient()).To(gomega.BeNil())
@@ -548,11 +552,7 @@ var _ = ginkgo.Describe("Preemption", func() {
 	})
 
 	ginkgo.AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns.Name})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{dev})
 
 		ginkgo.By("Tear down namespace: " + dev)
 		err := kClient.TearDownNamespace(dev)
diff --git a/test/e2e/priority_scheduling/priority_scheduling_suite_test.go b/test/e2e/priority_scheduling/priority_scheduling_suite_test.go
index b648474d..b5f40f77 100644
--- a/test/e2e/priority_scheduling/priority_scheduling_suite_test.go
+++ b/test/e2e/priority_scheduling/priority_scheduling_suite_test.go
@@ -21,6 +21,7 @@ package priority_test
 import (
 	"fmt"
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -55,6 +56,7 @@ func TestPriorityScheduling(t *testing.T) {
 	ginkgo.RunSpecs(t, "TestPriorityScheduling", ginkgo.Label("TestPriorityScheduling"))
 }
 
+var suiteName string
 var kubeClient k8s.KubeCtl
 
 var preemptPolicyNever = v1.PreemptNever
@@ -87,6 +89,8 @@ var annotation = "ann-" + common.RandSeq(10)
 var oldConfigMap = new(v1.ConfigMap)
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	var err error
 	kubeClient = k8s.KubeCtl{}
 	Expect(kubeClient.SetClient()).To(BeNil())
diff --git a/test/e2e/priority_scheduling/priority_scheduling_test.go b/test/e2e/priority_scheduling/priority_scheduling_test.go
index ef3c68d5..dc3540de 100644
--- a/test/e2e/priority_scheduling/priority_scheduling_test.go
+++ b/test/e2e/priority_scheduling/priority_scheduling_test.go
@@ -391,11 +391,7 @@ var _ = ginkgo.Describe("PriorityScheduling", func() {
 	})
 
 	ginkgo.AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
 
 		// If there is any error test case, we need to delete all pods to make sure it doesn't influence other cases.
 		ginkgo.By("Delete all sleep pods")
diff --git a/test/e2e/queue_quota_mgmt/queue_quota_mgmt_suite_test.go b/test/e2e/queue_quota_mgmt/queue_quota_mgmt_suite_test.go
index 89ab6757..e1422913 100644
--- a/test/e2e/queue_quota_mgmt/queue_quota_mgmt_suite_test.go
+++ b/test/e2e/queue_quota_mgmt/queue_quota_mgmt_suite_test.go
@@ -20,6 +20,7 @@ package queuequotamgmt_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	v1 "k8s.io/api/core/v1"
@@ -38,10 +39,13 @@ func init() {
 	configmanager.YuniKornTestConfig.ParseFlags()
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "", annotation)
 })
diff --git a/test/e2e/queue_quota_mgmt/queue_quota_mgmt_test.go b/test/e2e/queue_quota_mgmt/queue_quota_mgmt_test.go
index d7d3b736..97ecda45 100644
--- a/test/e2e/queue_quota_mgmt/queue_quota_mgmt_test.go
+++ b/test/e2e/queue_quota_mgmt/queue_quota_mgmt_test.go
@@ -23,7 +23,6 @@ import (
 	"math"
 	"time"
 
-	"github.com/onsi/ginkgo/v2"
 	v1 "k8s.io/api/core/v1"
 
 	"github.com/apache/yunikorn-core/pkg/webservice/dao"
@@ -247,16 +246,13 @@ var _ = Describe("", func() {
 	})
 
 	AfterEach(func() {
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
+
 		By("Check Yunikorn's health")
 		checks, err := yunikorn.GetFailedHealthChecks()
 		Ω(err).NotTo(HaveOccurred())
 		Ω(checks).To(Equal(""), checks)
 
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
 		By("Tearing down namespace: " + ns)
 		err = kClient.TearDownNamespace(ns)
 		Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/recovery_and_restart/recovery_and_restart_test.go b/test/e2e/recovery_and_restart/recovery_and_restart_test.go
index 12da4cb7..ad46323b 100644
--- a/test/e2e/recovery_and_restart/recovery_and_restart_test.go
+++ b/test/e2e/recovery_and_restart/recovery_and_restart_test.go
@@ -20,6 +20,7 @@ package recoveryandrestart_test
 
 import (
 	"fmt"
+	"runtime"
 	"strings"
 	"time"
 
@@ -45,6 +46,7 @@ const (
 	taintKey             = "e2e_test"
 )
 
+var suiteName string
 var kClient k8s.KubeCtl
 var restClient yunikorn.RClient
 var oldConfigMap = new(v1.ConfigMap)
@@ -57,6 +59,8 @@ var sleepPodConfigs = k8s.SleepPodConfig{Name: "sleepjob", NS: dev}
 var sleepPod2Configs = k8s.SleepPodConfig{Name: "sleepjob2", NS: dev}
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	Ω(kClient.SetClient()).To(gomega.BeNil())
diff --git a/test/e2e/resource_fairness/resource_fairness_suite_test.go b/test/e2e/resource_fairness/resource_fairness_suite_test.go
index 9e8b5dde..7c517978 100644
--- a/test/e2e/resource_fairness/resource_fairness_suite_test.go
+++ b/test/e2e/resource_fairness/resource_fairness_suite_test.go
@@ -20,6 +20,7 @@ package resourcefairness_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -37,10 +38,13 @@ func init() {
 	configmanager.YuniKornTestConfig.ParseFlags()
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 var kClient = k8s.KubeCtl{} //nolint
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	Ω(kClient.SetClient()).To(BeNil())
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
diff --git a/test/e2e/resource_fairness/resource_fairness_test.go b/test/e2e/resource_fairness/resource_fairness_test.go
index b846dd25..f003bc4f 100644
--- a/test/e2e/resource_fairness/resource_fairness_test.go
+++ b/test/e2e/resource_fairness/resource_fairness_test.go
@@ -193,11 +193,8 @@ var _ = Describe("FairScheduling:", func() {
 	})
 
 	AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
+
 		By("Tear down namespace: " + ns)
 		err := kClient.TearDownNamespace(ns)
 		Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/simple_preemptor/simple_preemptor_test.go b/test/e2e/simple_preemptor/simple_preemptor_test.go
index 6de50c87..8f551f68 100644
--- a/test/e2e/simple_preemptor/simple_preemptor_test.go
+++ b/test/e2e/simple_preemptor/simple_preemptor_test.go
@@ -20,6 +20,7 @@ package simple_preemptor_test
 
 import (
 	"fmt"
+	"runtime"
 	"strings"
 	"time"
 
@@ -36,6 +37,7 @@ import (
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/yunikorn"
 )
 
+var suiteName string
 var kClient k8s.KubeCtl
 var restClient yunikorn.RClient
 var ns *v1.Namespace
@@ -54,6 +56,8 @@ var taintKey = "e2e_test_simple_preemptor"
 var nodesToTaint []string
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	Ω(kClient.SetClient()).To(gomega.BeNil())
@@ -142,8 +146,8 @@ var _ = ginkgo.AfterSuite(func() {
 })
 
 var _ = ginkgo.Describe("SimplePreemptor", func() {
-	ginkgo.It("Verify_basic_simple_preemption. Use case: Only one pod is running and same pod has been selected as victim", func() {
-
+	ginkgo.It("Verify_basic_simple_preemption", func() {
+		// Use case: Only one pod is running and same pod has been selected as victim
 		// Define sleepPod
 		sleepPodConfigs := k8s.SleepPodConfig{Name: "sleepjob", NS: dev, Mem: sleepPodMemLimit1 * 2, Time: 600, RequiredNode: Worker1}
 		sleepPod2Configs := k8s.SleepPodConfig{Name: "sleepjob2", NS: dev, Mem: sleepPodMemLimit2 * 2, Time: 600}
@@ -167,8 +171,8 @@ var _ = ginkgo.Describe("SimplePreemptor", func() {
 		gomega.Ω(err).NotTo(gomega.HaveOccurred())
 	})
 
-	ginkgo.It("Verify_simple_preemption. Use case: When 3 sleep pods (2 opted out, regular) are running, regular pod should be victim to free up resources for 4th sleep pod", func() {
-
+	ginkgo.It("Verify_simple_preemption", func() {
+		// Use case: When 3 sleep pods (2 opted out, regular) are running, regular pod should be victim to free up resources for 4th sleep pod
 		// Define sleepPod
 		sleepPodConfigs := k8s.SleepPodConfig{Name: "sleepjob", NS: dev, Mem: sleepPodMemLimit1, Time: 600, RequiredNode: Worker1}
 		sleepPod2Configs := k8s.SleepPodConfig{Name: "sleepjob2", NS: dev, Mem: sleepPodMemLimit1, Time: 600, RequiredNode: Worker1}
@@ -215,11 +219,8 @@ var _ = ginkgo.Describe("SimplePreemptor", func() {
 	})
 
 	ginkgo.AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns.Name})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns.Name})
+
 		// Delete all sleep pods
 		ginkgo.By("Delete all sleep pods")
 		pods, err := kClient.GetPodNamesFromNS(ns.Name)
diff --git a/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_suite_test.go b/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_suite_test.go
index bb3e2dd8..a1968ffb 100644
--- a/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_suite_test.go
+++ b/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_suite_test.go
@@ -20,6 +20,7 @@ package spark_jobs_scheduling
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/onsi/ginkgo/v2"
@@ -36,9 +37,12 @@ func init() {
 	configmanager.YuniKornTestConfig.ParseFlags()
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation = "ann-" + common.RandSeq(10)
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "stateaware", annotation)
diff --git a/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_test.go b/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_test.go
index a28e14db..8dc4821a 100644
--- a/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_test.go
+++ b/test/e2e/spark_jobs_scheduling/spark_jobs_scheduling_test.go
@@ -27,7 +27,6 @@ import (
 	"sort"
 	"time"
 
-	"github.com/onsi/ginkgo/v2"
 	v1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/util/wait"
 	"k8s.io/client-go/rest"
@@ -155,11 +154,7 @@ var _ = Describe("", func() {
 	})
 
 	AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{sparkNS})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{sparkNS})
 
 		By("Killing all spark jobs")
 		// delete the Spark pods one by one
diff --git a/test/e2e/state_aware_app_scheduling/drip_feed_schedule_test.go b/test/e2e/state_aware_app_scheduling/drip_feed_schedule_test.go
index 49d5719d..4016b0eb 100644
--- a/test/e2e/state_aware_app_scheduling/drip_feed_schedule_test.go
+++ b/test/e2e/state_aware_app_scheduling/drip_feed_schedule_test.go
@@ -23,7 +23,6 @@ import (
 	"fmt"
 	"time"
 
-	"github.com/onsi/ginkgo/v2"
 	v1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/util/wait"
 
@@ -140,16 +139,13 @@ var _ = Describe("DripFeedSchedule:", func() {
 	}, testTimeout)
 
 	AfterEach(func() {
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
+
 		By("Check Yunikorn's health")
 		checks, err := yunikorn.GetFailedHealthChecks()
 		Ω(err).NotTo(HaveOccurred())
 		Ω(checks).To(Equal(""), checks)
 
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
 		By("Tearing down namespace: " + ns)
 		err = kClient.TearDownNamespace(ns)
 		Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/state_aware_app_scheduling/fallback_test.go b/test/e2e/state_aware_app_scheduling/fallback_test.go
index d8cd3216..bfa9a8f2 100644
--- a/test/e2e/state_aware_app_scheduling/fallback_test.go
+++ b/test/e2e/state_aware_app_scheduling/fallback_test.go
@@ -21,7 +21,6 @@ package stateawareappscheduling_test
 import (
 	"fmt"
 
-	"github.com/onsi/ginkgo/v2"
 	"github.com/onsi/gomega"
 	v1 "k8s.io/api/core/v1"
 
@@ -105,16 +104,13 @@ var _ = Describe("FallbackTest:", func() {
 	})
 
 	AfterEach(func() {
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns})
+
 		By("Check Yunikorn's health")
 		checks, err := yunikorn.GetFailedHealthChecks()
 		Ω(err).NotTo(HaveOccurred())
 		Ω(checks).To(Equal(""), checks)
 
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
 		By("Tearing down namespace: " + ns)
 		err = kClient.TearDownNamespace(ns)
 		Ω(err).NotTo(HaveOccurred())
diff --git a/test/e2e/state_aware_app_scheduling/state_aware_app_scheduling_suite_test.go b/test/e2e/state_aware_app_scheduling/state_aware_app_scheduling_suite_test.go
index 3952f0bc..c6d12a58 100644
--- a/test/e2e/state_aware_app_scheduling/state_aware_app_scheduling_suite_test.go
+++ b/test/e2e/state_aware_app_scheduling/state_aware_app_scheduling_suite_test.go
@@ -20,6 +20,7 @@ package stateawareappscheduling_test
 
 import (
 	"path/filepath"
+	"runtime"
 	"testing"
 
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/helpers/common"
@@ -39,10 +40,13 @@ func init() {
 	configmanager.YuniKornTestConfig.ParseFlags()
 }
 
+var suiteName string
 var oldConfigMap = new(v1.ConfigMap)
 var annotation string
 
 var _ = BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	annotation = "ann-" + common.RandSeq(10)
 	yunikorn.EnsureYuniKornConfigsPresent()
 	yunikorn.UpdateConfigMapWrapper(oldConfigMap, "stateaware", annotation)
diff --git a/test/e2e/user_group_limit/user_group_limit_test.go b/test/e2e/user_group_limit/user_group_limit_test.go
index a6f4dbb2..452cbc30 100644
--- a/test/e2e/user_group_limit/user_group_limit_test.go
+++ b/test/e2e/user_group_limit/user_group_limit_test.go
@@ -21,6 +21,7 @@ package user_group_limit_test
 import (
 	"encoding/json"
 	"fmt"
+	"runtime"
 	"time"
 
 	"github.com/onsi/ginkgo/v2"
@@ -60,6 +61,7 @@ const (
 )
 
 var (
+	suiteName             string
 	kClient               k8s.KubeCtl
 	restClient            yunikorn.RClient
 	ns                    *v1.Namespace
@@ -73,6 +75,8 @@ var (
 )
 
 var _ = ginkgo.BeforeSuite(func() {
+	_, filename, _, _ := runtime.Caller(0)
+	suiteName = common.GetSuiteName(filename)
 	// Initializing kubectl client
 	kClient = k8s.KubeCtl{}
 	Ω(kClient.SetClient()).To(gomega.BeNil())
@@ -574,11 +578,8 @@ var _ = ginkgo.Describe("UserGroupLimit", func() {
 	})
 
 	ginkgo.AfterEach(func() {
-		testDescription := ginkgo.CurrentSpecReport()
-		if testDescription.Failed() {
-			tests.LogTestClusterInfoWrapper(testDescription.FailureMessage(), []string{ns.Name})
-			tests.LogYunikornContainer(testDescription.FailureMessage())
-		}
+		tests.DumpClusterInfoIfSpecFailed(suiteName, []string{ns.Name})
+
 		// Delete all sleep pods
 		ginkgo.By("Delete all sleep pods")
 		err := kClient.DeletePods(ns.Name)
diff --git a/test/e2e/wrappers.go b/test/e2e/wrappers.go
index 419aa23f..c9d484d6 100644
--- a/test/e2e/wrappers.go
+++ b/test/e2e/wrappers.go
@@ -26,7 +26,6 @@ import (
 	"github.com/onsi/gomega"
 	"gopkg.in/yaml.v3"
 	v1 "k8s.io/api/core/v1"
-	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 
 	"github.com/apache/yunikorn-core/pkg/common/configs"
 	"github.com/apache/yunikorn-k8shim/test/e2e/framework/configmanager"
@@ -92,82 +91,105 @@ func RestoreConfigMapWrapper(oldConfigMap *v1.ConfigMap, annotation string) {
 	Ω(err).NotTo(HaveOccurred())
 }
 
-func LogTestClusterInfoWrapper(testName string, namespaces []string) {
-	fmt.Fprintf(ginkgo.GinkgoWriter, "%s Log test cluster info\n", testName)
-	var restClient yunikorn.RClient
-	err := k.SetClient()
-	if err != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Error setting k8s client: %v\n", err)
-		return
-	}
-
-	for _, ns := range namespaces {
-		logErr := k8s.LogNamespaceInfo(ns)
-		if logErr != nil {
-			fmt.Fprintf(ginkgo.GinkgoWriter, "Error logging namespace info: %v\n", logErr)
-			continue
+func DumpClusterInfoIfSpecFailed(suiteName string, namespaces []string) {
+	// should call this function in ginkgo.AfterEach
+	// write cluster info to files by log type (ykFullStateDump, k8sClusterInfo, ykContainerLog)
+	testDescription := ginkgo.CurrentSpecReport()
+	if testDescription.Failed() {
+		specName := testDescription.LeafNodeText
+		fmt.Fprintf(ginkgo.GinkgoWriter, "Logging yk fullstatedump, spec: %s\n", specName)
+		err := dumpYKFullStateDump(suiteName, specName)
+		if err != nil {
+			fmt.Fprintf(ginkgo.GinkgoWriter, "Fail to log yk fullstatedump, spec: %s, err: %v\n", specName, err)
 		}
 
-		pods, err := k.GetPodsByOptions(metav1.ListOptions{})
+		fmt.Fprintf(ginkgo.GinkgoWriter, "Logging k8s cluster info, spec: %s\n", specName)
+		err = dumpKubernetesClusterInfo(suiteName, specName, namespaces)
 		if err != nil {
-			fmt.Fprintf(ginkgo.GinkgoWriter, "Error getting pods: %v\n", err)
-		} else {
-			fmt.Fprintf(ginkgo.GinkgoWriter, "Pod count is %d\n", len(pods.Items))
-			for _, pod := range pods.Items {
-				fmt.Fprintf(ginkgo.GinkgoWriter, "Pod name is %s\n", pod.Name)
-				fmt.Fprintf(ginkgo.GinkgoWriter, "Pod details: %s\n", pod.String())
-			}
+			fmt.Fprintf(ginkgo.GinkgoWriter, "Fail to log k8s cluster info, spec: %s, err: %v\n", specName, err)
 		}
 
-		logErr = restClient.LogAppsInfo(ns)
-		if logErr != nil {
-			fmt.Fprintf(ginkgo.GinkgoWriter, "Error logging apps info: %v\n", logErr)
+		fmt.Fprintf(ginkgo.GinkgoWriter, "Logging yk container logs, spec: %s\n", specName)
+		err = dumpYunikornContainer(suiteName, specName)
+		if err != nil {
+			fmt.Fprintf(ginkgo.GinkgoWriter, "Fail to log yk container logs, spec: %s, err: %v\n", specName, err)
 		}
 	}
-	logErr := restClient.LogQueuesInfo()
-	if logErr != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Error logging queues info: %v\n", logErr)
+}
+
+func dumpYKFullStateDump(suiteName string, specName string) error {
+	file, err := common.CreateLogFile(suiteName, specName, "ykFullStateDump", "json")
+	if err != nil {
+		return err
+	}
+	defer file.Close()
+
+	var restClient yunikorn.RClient
+	fullStateDumpJson, err := restClient.GetFullStateDump()
+	if err != nil {
+		return err
 	}
 
-	logErr = restClient.LogNodesInfo()
-	if logErr != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Error logging nodes info: %v\n", logErr)
+	_, err = fmt.Fprintln(file, fullStateDumpJson)
+	return err
+}
+
+func dumpKubernetesClusterInfo(suiteName string, specName string, namespaces []string) error {
+	file, err := common.CreateLogFile(suiteName, specName, "k8sClusterInfo", "txt")
+	if err != nil {
+		return err
 	}
+	defer file.Close()
 
-	nodes, err := k.GetNodes()
+	err = k.SetClient()
 	if err != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Error getting nodes: %v\n", err)
-	} else {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Node count is %d\n", len(nodes.Items))
-		for _, node := range nodes.Items {
-			fmt.Fprintf(ginkgo.GinkgoWriter, "Running describe node command for %s..\n", node.Name)
-			err = k.DescribeNode(node)
-			if err != nil {
-				fmt.Fprintf(ginkgo.GinkgoWriter, "Error describing node: %v\n", err)
-			}
+		return err
+	}
+
+	for _, ns := range namespaces {
+		err = k.LogNamespaceInfo(file, ns)
+		if err != nil {
+			fmt.Fprintf(ginkgo.GinkgoWriter, "Failed to log namespace info, ns:%s, err: %v\n", ns, err)
 		}
 	}
+
+	err = k.LogPodsInfo(file)
+	if err != nil {
+		fmt.Fprintf(ginkgo.GinkgoWriter, "Failed to log pods info, err: %v\n", err)
+	}
+
+	err = k.LogNodesInfo(file)
+	if err != nil {
+		fmt.Fprintf(ginkgo.GinkgoWriter, "Failed to log nodes info, err: %v\n", err)
+	}
+
+	return nil
 }
 
-func LogYunikornContainer(testName string) {
-	fmt.Fprintf(ginkgo.GinkgoWriter, "%s Log yk logs info from\n", testName)
-	err := k.SetClient()
+func dumpYunikornContainer(suiteName string, specName string) error {
+	file, err := common.CreateLogFile(suiteName, specName, "ykContainerLog", "txt")
+	if err != nil {
+		return err
+	}
+	defer file.Close()
+
+	err = k.SetClient()
 	if err != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Error setting k8s client: %v\n", err)
-		return
+		return err
 	}
+
 	ykSchedName, schedErr := yunikorn.GetSchedulerPodName(k)
 	if schedErr != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Failed to get the scheduler pod name: %v\n", schedErr)
-		return
+		return schedErr
 	}
 
 	logBytes, getErr := k.GetPodLogs(ykSchedName, configmanager.YuniKornTestConfig.YkNamespace, configmanager.YKSchedulerContainer)
 	if getErr != nil {
-		fmt.Fprintf(ginkgo.GinkgoWriter, "Failed to get scheduler pod logs: %v\n", getErr)
-		return
+		return getErr
 	}
-	fmt.Fprintf(ginkgo.GinkgoWriter, "Yunikorn Logs:%s\n", string(logBytes))
+
+	_, err = fmt.Fprintf(file, "Yunikorn Logs:\n%s\n", string(logBytes))
+	return err
 }
 
 var Describe = ginkgo.Describe


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@yunikorn.apache.org
For additional commands, e-mail: issues-help@yunikorn.apache.org