diff --git a/docs/using.md b/docs/using.md
index 1bd16891c07..e4960e38a10 100644
--- a/docs/using.md
+++ b/docs/using.md
@@ -123,9 +123,14 @@ specific contract.
 #### Entrypoint
 
 When containers are run in a `Task`, the `entrypoint` of the container will be
-overwritten with a custom binary that redirects the logs to a separate location
-for aggregating the log output. As such, it is always recommended to explicitly
-specify a command.
+overwritten with a custom binary. The plan is to use this custom binary for
+controlling the execution of step containers ([#224](https://github.com/knative/build-pipeline/issues/224)) and log streaming
+[#107](https://github.com/knative/build-pipeline/issues/107), though currently
+it will write logs only to an [`emptyDir`](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir)
+(which cannot be read from after the pod has finished executing, so logs must be obtained
+[via k8s logs](https://kubernetes.io/docs/concepts/cluster-administration/logging/),
+using a tool such as [test/logs/README.md](../test/logs/README.md),
+or setting up an external system to consume logs).
 
 When `command` is not explicitly set, the controller will attempt to lookup the
 entrypoint from the remote registry.
diff --git a/pkg/reconciler/v1alpha1/taskrun/taskrun.go b/pkg/reconciler/v1alpha1/taskrun/taskrun.go
index 15c4a53f685..47121a2c8f2 100644
--- a/pkg/reconciler/v1alpha1/taskrun/taskrun.go
+++ b/pkg/reconciler/v1alpha1/taskrun/taskrun.go
@@ -39,11 +39,9 @@ import (
 	corev1 "k8s.io/api/core/v1"
 	"k8s.io/apimachinery/pkg/api/equality"
 	"k8s.io/apimachinery/pkg/api/errors"
-	"k8s.io/apimachinery/pkg/api/resource"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	"k8s.io/apimachinery/pkg/runtime/schema"
 	coreinformers "k8s.io/client-go/informers/core/v1"
-	"k8s.io/client-go/kubernetes"
 	"k8s.io/client-go/tools/cache"
 )
 
@@ -71,8 +69,6 @@ const (
 	taskRunAgentName = "taskrun-controller"
 	// taskRunControllerName defines name for TaskRun Controller
 	taskRunControllerName = "TaskRun"
-
-	pvcSizeBytes = 5 * 1024 * 1024 * 1024 // 5 GBs
 )
 
 var (
@@ -276,19 +272,8 @@ func (c *Reconciler) reconcile(ctx context.Context, tr *v1alpha1.TaskRun) error
 			return err
 		}
 	} else {
-		pvc, err := c.KubeClientSet.CoreV1().PersistentVolumeClaims(tr.Namespace).Get(tr.Name, metav1.GetOptions{})
-		if errors.IsNotFound(err) {
-			// Create a persistent volume claim to hold Build logs
-			pvc, err = createPVC(c.KubeClientSet, tr)
-			if err != nil {
-				return fmt.Errorf("Failed to create persistent volume claim %s for task %q: %v", tr.Name, err, tr.Name)
-			}
-		} else if err != nil {
-			c.Logger.Errorf("Failed to reconcile taskrun: %q, failed to get pvc %q: %v", tr.Name, tr.Name, err)
-			return err
-		}
 		// Build pod is not present, create build pod.
-		pod, err = c.createBuildPod(ctx, tr, rtr.TaskSpec, rtr.TaskName, pvc.Name)
+		pod, err = c.createBuildPod(ctx, tr, rtr.TaskSpec, rtr.TaskName)
 		if err != nil {
 			// This Run has failed, so we need to mark it as failed and stop reconciling it
 			var msg string
@@ -368,40 +353,9 @@ func (c *Reconciler) updateStatus(taskrun *v1alpha1.TaskRun) (*v1alpha1.TaskRun,
 	return newtaskrun, nil
 }
 
-// createPVC will create a persistent volume mount for tr which
-// will be used to gather logs using the entrypoint wrapper
-func createPVC(kc kubernetes.Interface, tr *v1alpha1.TaskRun) (*corev1.PersistentVolumeClaim, error) {
-	v, err := kc.CoreV1().PersistentVolumeClaims(tr.Namespace).Create(
-		&corev1.PersistentVolumeClaim{
-			ObjectMeta: metav1.ObjectMeta{
-				Namespace: tr.Namespace,
-				// This pvc is specific to this TaskRun, so we'll use the same name
-				Name: tr.Name,
-				OwnerReferences: []metav1.OwnerReference{
-					*metav1.NewControllerRef(tr, groupVersionKind),
-				},
-			},
-			Spec: corev1.PersistentVolumeClaimSpec{
-				AccessModes: []corev1.PersistentVolumeAccessMode{
-					corev1.ReadWriteOnce,
-				},
-				Resources: corev1.ResourceRequirements{
-					Requests: map[corev1.ResourceName]resource.Quantity{
-						corev1.ResourceStorage: *resource.NewQuantity(pvcSizeBytes, resource.BinarySI),
-					},
-				},
-			},
-		},
-	)
-	if err != nil {
-		return nil, fmt.Errorf("failed to claim Persistent Volume %q due to error: %s", tr.Name, err)
-	}
-	return v, nil
-}
-
 // createPod creates a Pod based on the Task's configuration, with pvcName as a
 // volumeMount
-func (c *Reconciler) createBuildPod(ctx context.Context, tr *v1alpha1.TaskRun, ts *v1alpha1.TaskSpec, taskName, pvcName string) (*corev1.Pod, error) {
+func (c *Reconciler) createBuildPod(ctx context.Context, tr *v1alpha1.TaskRun, ts *v1alpha1.TaskSpec, taskName string) (*corev1.Pod, error) {
 	// TODO: Preferably use Validate on task.spec to catch validation error
 	bs := ts.GetBuildSpec()
 	if bs == nil {
@@ -434,7 +388,7 @@ func (c *Reconciler) createBuildPod(ctx context.Context, tr *v1alpha1.TaskRun, t
 		}
 	}
 
-	build, err := createRedirectedBuild(ctx, bSpec, pvcName, tr)
+	build, err := createRedirectedBuild(ctx, bSpec, tr)
 	if err != nil {
 		return nil, fmt.Errorf("couldn't create redirected Build: %v", err)
 	}
@@ -488,7 +442,7 @@ func (c *Reconciler) createBuildPod(ctx context.Context, tr *v1alpha1.TaskRun, t
 // an entrypoint cache creates a build where all entrypoints are switched to
 // be the entrypoint redirector binary. This function assumes that it receives
 // its own copy of the BuildSpec and modifies it freely
-func createRedirectedBuild(ctx context.Context, bs *buildv1alpha1.BuildSpec, pvcName string, tr *v1alpha1.TaskRun) (*buildv1alpha1.Build, error) {
+func createRedirectedBuild(ctx context.Context, bs *buildv1alpha1.BuildSpec, tr *v1alpha1.TaskRun) (*buildv1alpha1.Build, error) {
 	// Pass service account name from taskrun to build
 	bs.ServiceAccountName = tr.Spec.ServiceAccount
 
@@ -519,9 +473,9 @@ func createRedirectedBuild(ctx context.Context, bs *buildv1alpha1.BuildSpec, pvc
 	b.Spec.Volumes = append(b.Spec.Volumes, corev1.Volume{
 		Name: entrypoint.MountName,
 		VolumeSource: corev1.VolumeSource{
-			PersistentVolumeClaim: &corev1.PersistentVolumeClaimVolumeSource{
-				ClaimName: pvcName,
-			},
+			// TODO(#107) we need to actually stream these logs somewhere, probably via sidecar.
+			// Currently these logs will be lost when the pod is unscheduled.
+			EmptyDir: &corev1.EmptyDirVolumeSource{},
 		},
 	})
 
diff --git a/pkg/reconciler/v1alpha1/taskrun/taskrun_test.go b/pkg/reconciler/v1alpha1/taskrun/taskrun_test.go
index 9f64e62b7d6..05ff838e77e 100644
--- a/pkg/reconciler/v1alpha1/taskrun/taskrun_test.go
+++ b/pkg/reconciler/v1alpha1/taskrun/taskrun_test.go
@@ -35,7 +35,6 @@ import (
 	"go.uber.org/zap"
 	"go.uber.org/zap/zaptest/observer"
 	corev1 "k8s.io/api/core/v1"
-	"k8s.io/apimachinery/pkg/api/resource"
 	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
 	"k8s.io/apimachinery/pkg/runtime"
 	fakekubeclientset "k8s.io/client-go/kubernetes/fake"
@@ -108,36 +107,11 @@ var (
 	))
 )
 
-func getExpectedPVC(tr *v1alpha1.TaskRun) *corev1.PersistentVolumeClaim {
-	return &corev1.PersistentVolumeClaim{
-		ObjectMeta: metav1.ObjectMeta{
-			Namespace: tr.Namespace,
-			// This pvc is specific to this TaskRun, so we'll use the same name
-			Name: tr.Name,
-			OwnerReferences: []metav1.OwnerReference{
-				*metav1.NewControllerRef(tr, groupVersionKind),
-			},
-		},
-		Spec: corev1.PersistentVolumeClaimSpec{
-			AccessModes: []corev1.PersistentVolumeAccessMode{
-				corev1.ReadWriteOnce,
-			},
-			Resources: corev1.ResourceRequirements{
-				Requests: map[corev1.ResourceName]resource.Quantity{
-					corev1.ResourceStorage: *resource.NewQuantity(pvcSizeBytes, resource.BinarySI),
-				},
-			},
-		},
-	}
-}
-
 func getToolsVolume(claimName string) corev1.Volume {
 	return corev1.Volume{
 		Name: toolsMountName,
 		VolumeSource: corev1.VolumeSource{
-			PersistentVolumeClaim: &corev1.PersistentVolumeClaimVolumeSource{
-				ClaimName: claimName,
-			},
+			EmptyDir: &corev1.EmptyDirVolumeSource{},
 		},
 	}
 }
@@ -422,27 +396,6 @@ func TestReconcile(t *testing.T) {
 			if len(clients.Kube.Actions()) == 0 {
 				t.Fatalf("Expected actions to be logged in the kubeclient, got none")
 			}
-
-			pvc, err := clients.Kube.CoreV1().PersistentVolumeClaims(namespace).Get(name, metav1.GetOptions{})
-			if err != nil {
-				t.Errorf("Failed to fetch build: %v", err)
-			}
-
-			expectedVolume := getExpectedPVC(tr)
-			if d := cmp.Diff(pvc.Name, expectedVolume.Name); d != "" {
-				t.Errorf("pvc doesn't match, diff: %s", d)
-			}
-			if d := cmp.Diff(pvc.OwnerReferences, expectedVolume.OwnerReferences); d != "" {
-				t.Errorf("pvc doesn't match, diff: %s", d)
-			}
-			if d := cmp.Diff(pvc.Spec.AccessModes, expectedVolume.Spec.AccessModes); d != "" {
-				t.Errorf("pvc doesn't match, diff: %s", d)
-			}
-			if pvc.Spec.Resources.Requests["storage"] != expectedVolume.Spec.Resources.Requests["storage"] {
-				t.Errorf("pvc doesn't match, got: %v, expected: %v",
-					pvc.Spec.Resources.Requests["storage"],
-					expectedVolume.Spec.Resources.Requests["storage"])
-			}
 		})
 	}
 }
@@ -787,7 +740,7 @@ func TestCreateRedirectedBuild(t *testing.T) {
 	expectedSteps := len(bs.Steps) + 1
 	expectedVolumes := len(bs.Volumes) + 1
 
-	b, err := createRedirectedBuild(ctx, &bs, "pvc", tr)
+	b, err := createRedirectedBuild(ctx, &bs, tr)
 	if err != nil {
 		t.Errorf("expected createRedirectedBuild to pass: %v", err)
 	}
diff --git a/test/cancel_test.go b/test/cancel_test.go
index d9af861f9fa..b0487da7b84 100644
--- a/test/cancel_test.go
+++ b/test/cancel_test.go
@@ -45,7 +45,7 @@ func TestTaskRunPipelineRunCancel(t *testing.T) {
 		tb.Step("foo", "ubuntu", tb.Command("/bin/bash"), tb.Args("-c", "sleep 500")),
 	))
 	if _, err := c.TaskClient.Create(task); err != nil {
-		t.Fatalf("Failed to create Task `%s`: %s", hwTaskName, err)
+		t.Fatalf("Failed to create Task `banana`: %s", err)
 	}
 
 	pipeline := tb.Pipeline("tomatoes", namespace,
@@ -64,7 +64,7 @@ func TestTaskRunPipelineRunCancel(t *testing.T) {
 		c := pr.Status.GetCondition(duckv1alpha1.ConditionSucceeded)
 		if c != nil {
 			if c.Status == corev1.ConditionTrue || c.Status == corev1.ConditionFalse {
-				return true, fmt.Errorf("pipelineRun %s already finished!", "pear")
+				return true, fmt.Errorf("pipelineRun %s already finished", "pear")
 			} else if c.Status == corev1.ConditionUnknown && (c.Reason == "Running" || c.Reason == "Pending") {
 				return true, nil
 			}
@@ -114,10 +114,10 @@ func TestTaskRunPipelineRunCancel(t *testing.T) {
 		}
 		return false, nil
 	}, "PipelineRunCancelled"); err != nil {
-		t.Errorf("Error waiting for TaskRun %s to finish: %s", hwTaskRunName, err)
+		t.Errorf("Error waiting for PipelineRun `pear` to finish: %s", err)
 	}
 
-	logger.Infof("Waiting for TaskRun %s in namespace %s to be cancelled", hwTaskRunName, namespace)
+	logger.Infof("Waiting for TaskRun `pear-foo` in namespace %s to be cancelled", namespace)
 	if err := WaitForTaskRunState(c, "pear-foo", func(tr *v1alpha1.TaskRun) (bool, error) {
 		c := tr.Status.GetCondition(duckv1alpha1.ConditionSucceeded)
 		if c != nil {
diff --git a/test/crd.go b/test/crd.go
deleted file mode 100644
index d07fb040fb0..00000000000
--- a/test/crd.go
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
-Copyright 2018 The Knative Authors
-
-Licensed 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.
-*/
-
-// crd contains defintions of resource instances which are useful across integration tests
-
-package test
-
-import (
-	"bufio"
-	"bytes"
-	"fmt"
-	"io"
-	"testing"
-
-	"github.com/knative/pkg/test/logging"
-	corev1 "k8s.io/api/core/v1"
-	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
-
-	"github.com/knative/build-pipeline/pkg/apis/pipeline/v1alpha1"
-	tb "github.com/knative/build-pipeline/test/builder"
-)
-
-const (
-	hwTaskName          = "helloworld"
-	hwTaskRunName       = "helloworld-run"
-	hwValidationPodName = "helloworld-validation-busybox"
-	hwPipelineName      = "helloworld-pipeline"
-	hwPipelineRunName   = "helloworld-pipelinerun"
-	hwPipelineTaskName1 = "helloworld-task-1"
-	hwPipelineTaskName2 = "helloworld-task-2"
-	hwSecret            = "helloworld-secret"
-	hwSA                = "helloworld-sa"
-
-	logPath = "/logs"
-	logFile = "process-log.txt"
-
-	hwContainerName = "helloworld-busybox"
-	taskOutput      = "do you want to build a snowman"
-	buildOutput     = "Build successful"
-)
-
-func getHelloWorldValidationPod(namespace, volumeClaimName string) *corev1.Pod {
-	return &corev1.Pod{
-		ObjectMeta: metav1.ObjectMeta{
-			Namespace: namespace,
-			Name:      hwValidationPodName,
-		},
-		Spec: corev1.PodSpec{
-			Containers: []corev1.Container{{
-				Name:  hwValidationPodName,
-				Image: "busybox",
-				Command: []string{
-					"cat",
-				},
-				Args: []string{fmt.Sprintf("%s/%s", logPath, logFile)},
-				VolumeMounts: []corev1.VolumeMount{
-					{
-						Name:      "scratch",
-						MountPath: logPath,
-					},
-				},
-			}},
-			Volumes: []corev1.Volume{{
-				Name: "scratch",
-				VolumeSource: corev1.VolumeSource{
-					PersistentVolumeClaim: &corev1.PersistentVolumeClaimVolumeSource{
-						ClaimName: volumeClaimName,
-					},
-				},
-			}},
-		},
-	}
-}
-
-func getHelloWorldTask(namespace string, args []string) *v1alpha1.Task {
-	return tb.Task(hwTaskName, namespace,
-		tb.TaskSpec(
-			tb.TaskInputs(tb.InputsResource("docs", v1alpha1.PipelineResourceTypeGit)),
-			tb.Step("read", "ubuntu",
-				tb.Command("/bin/bash"),
-				tb.Args("-c", "cat /workspace/docs/README.md"),
-			),
-			tb.Step(hwContainerName, "busybox", tb.Command(args...)),
-		))
-}
-
-func getHelloWorldTaskRun(namespace string) *v1alpha1.TaskRun {
-	testSpec := &v1alpha1.PipelineResourceSpec{
-		Type: v1alpha1.PipelineResourceTypeGit,
-		Params: []v1alpha1.Param{{
-			Name:  "URL",
-			Value: "http://github.com/knative/docs",
-		}},
-	}
-	return tb.TaskRun(hwTaskRunName, namespace,
-		tb.TaskRunSpec(
-			tb.TaskRunInputs(
-				tb.TaskRunInputsResource("docs", tb.TaskResourceBindingResourceSpec(testSpec)),
-			),
-			tb.TaskRunTaskRef(hwTaskName)))
-}
-
-func getBuildOutputFromVolume(t *testing.T, logger *logging.BaseLogger, c *clients, namespace, testStr string) string {
-	t.Helper()
-	// Create Validation Pod
-	pods := c.KubeClient.Kube.CoreV1().Pods(namespace)
-
-	// Volume created for Task should have the same name as the Task
-	if _, err := pods.Create(getHelloWorldValidationPod(namespace, hwTaskRunName)); err != nil {
-		t.Fatalf("failed to create Validation pod to mount volume `%s`: %s", hwTaskRunName, err)
-	}
-
-	logger.Infof("Waiting for pod with test volume %s to come up so we can read logs from it", hwTaskRunName)
-	if err := WaitForPodState(c, hwValidationPodName, namespace, func(p *corev1.Pod) (bool, error) {
-		// the "Running" status is used as "Succeeded" caused issues as the pod succeeds and restarts quickly
-		// there might be a race condition here and possibly a better way of handling this, perhaps using a Job or different state validation
-		if p.Status.Phase == corev1.PodRunning {
-			return true, nil
-		}
-		return false, nil
-	}, "ValidationPodCompleted"); err != nil {
-		t.Fatalf("error waiting for Pod %s to finish: %s", hwValidationPodName, err)
-	}
-
-	// Get validation pod logs and verify that the build executed a container w/ desired output
-	req := pods.GetLogs(hwValidationPodName, &corev1.PodLogOptions{})
-	readCloser, err := req.Stream()
-	if err != nil {
-		t.Fatalf("failed to open stream to read: %v", err)
-	}
-	defer readCloser.Close()
-	var buf bytes.Buffer
-	out := bufio.NewWriter(&buf)
-	_, err = io.Copy(out, readCloser)
-	return buf.String()
-}
diff --git a/test/embed_test.go b/test/embed_test.go
new file mode 100644
index 00000000000..2790f4cbd5f
--- /dev/null
+++ b/test/embed_test.go
@@ -0,0 +1,88 @@
+// +build e2e
+
+/*
+Copyright 2018 Knative Authors LLC
+Licensed 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 test
+
+import (
+	"fmt"
+	"testing"
+
+	"github.com/knative/build-pipeline/pkg/apis/pipeline/v1alpha1"
+	tb "github.com/knative/build-pipeline/test/builder"
+	knativetest "github.com/knative/pkg/test"
+	"github.com/knative/pkg/test/logging"
+)
+
+const (
+	embedTaskName    = "helloworld"
+	embedTaskRunName = "helloworld-run"
+
+	// TODO(#127) Currently not reliable to retrieve this output
+	taskOutput = "do you want to build a snowman"
+)
+
+func getEmbeddedTask(namespace string, args []string) *v1alpha1.Task {
+	return tb.Task(embedTaskName, namespace,
+		tb.TaskSpec(
+			tb.TaskInputs(tb.InputsResource("docs", v1alpha1.PipelineResourceTypeGit)),
+			tb.Step("read", "ubuntu",
+				tb.Command("/bin/bash"),
+				tb.Args("-c", "cat /workspace/docs/README.md"),
+			),
+			tb.Step("helloworld-busybox", "busybox", tb.Command(args...)),
+		))
+}
+
+func getEmbeddedTaskRun(namespace string) *v1alpha1.TaskRun {
+	testSpec := &v1alpha1.PipelineResourceSpec{
+		Type: v1alpha1.PipelineResourceTypeGit,
+		Params: []v1alpha1.Param{{
+			Name:  "URL",
+			Value: "http://github.com/knative/docs",
+		}},
+	}
+	return tb.TaskRun(embedTaskRunName, namespace,
+		tb.TaskRunSpec(
+			tb.TaskRunInputs(
+				tb.TaskRunInputsResource("docs", tb.TaskResourceBindingResourceSpec(testSpec)),
+			),
+			tb.TaskRunTaskRef(embedTaskName)))
+}
+
+// TestTaskRun_EmbeddedResource is an integration test that will verify a very simple "hello world" TaskRun can be
+// executed with an embedded resource spec.
+func TestTaskRun_EmbeddedResource(t *testing.T) {
+	logger := logging.GetContextLogger(t.Name())
+	c, namespace := setup(t, logger)
+
+	knativetest.CleanupOnInterrupt(func() { tearDown(t, logger, c, namespace) }, logger)
+	defer tearDown(t, logger, c, namespace)
+
+	logger.Infof("Creating Task and TaskRun in namespace %s", namespace)
+	if _, err := c.TaskClient.Create(getEmbeddedTask(namespace, []string{"/bin/sh", "-c", fmt.Sprintf("echo %s", taskOutput)})); err != nil {
+		t.Fatalf("Failed to create Task `%s`: %s", embedTaskName, err)
+	}
+	if _, err := c.TaskRunClient.Create(getEmbeddedTaskRun(namespace)); err != nil {
+		t.Fatalf("Failed to create TaskRun `%s`: %s", embedTaskRunName, err)
+	}
+
+	logger.Infof("Waiting for TaskRun %s in namespace %s to complete", embedTaskRunName, namespace)
+	if err := WaitForTaskRunState(c, embedTaskRunName, TaskRunSucceed(embedTaskRunName), "TaskRunSuccess"); err != nil {
+		t.Errorf("Error waiting for TaskRun %s to finish: %s", embedTaskRunName, err)
+	}
+
+	// TODO(#127) Currently we have no reliable access to logs from the TaskRun so we'll assume successful
+	// completion of the TaskRun means the TaskRun did what it was intended.
+}
diff --git a/test/logs/README.md b/test/logs/README.md
index a14e98de11f..482f3f4d692 100644
--- a/test/logs/README.md
+++ b/test/logs/README.md
@@ -7,7 +7,8 @@
 `Pipeline` definition. Each `TaskRun` object creates Kubernetes Pod and all the
 steps are translated into init containers.
 
-There is a "gotcha" about tailing logs for init container. Logs cannot be
+There is a "gotcha" about tailing logs for init containers (to be fixed in
+[#224](https://github.com/knative/build-pipeline/issues/224)). Logs cannot be
 retrieved from pod that has been shut down for a while. In this case tailing
 logs will return error `Unable to retrieve container logs`.
 
diff --git a/test/pipelinerun_test.go b/test/pipelinerun_test.go
index f83d7a4de5c..3740075dca1 100644
--- a/test/pipelinerun_test.go
+++ b/test/pipelinerun_test.go
@@ -36,6 +36,12 @@ import (
 )
 
 var (
+	pipelineName       = "pipeline"
+	pipelineRunName    = "pipelinerun"
+	secretName         = "secret"
+	saName             = "service-account"
+	taskName           = "task"
+	task1Name          = "task1"
 	pipelineRunTimeout = 10 * time.Minute
 )
 
@@ -43,7 +49,7 @@ func TestPipelineRun(t *testing.T) {
 	t.Parallel()
 	type tests struct {
 		name                   string
-		testSetup              func(c *clients, namespace string, index int)
+		testSetup              func(t *testing.T, c *clients, namespace string, index int)
 		expectedTaskRuns       []string
 		expectedNumberOfEvents int
 		pipelineRunFunc        func(int, string) *v1alpha1.PipelineRun
@@ -51,7 +57,7 @@ func TestPipelineRun(t *testing.T) {
 
 	tds := []tests{{
 		name: "fan-in and fan-out",
-		testSetup: func(c *clients, namespace string, index int) {
+		testSetup: func(t *testing.T, c *clients, namespace string, index int) {
 			t.Helper()
 			for _, task := range getFanInFanOutTasks(namespace) {
 				if _, err := c.TaskClient.Create(task); err != nil {
@@ -66,7 +72,7 @@ func TestPipelineRun(t *testing.T) {
 			}
 
 			if _, err := c.PipelineClient.Create(getFanInFanOutPipeline(index, namespace)); err != nil {
-				t.Fatalf("Failed to create Pipeline `%s`: %s", getName(hwPipelineName, index), err)
+				t.Fatalf("Failed to create Pipeline `%s`: %s", getName(pipelineName, index), err)
 			}
 		},
 		pipelineRunFunc:  getFanInFanOutPipelineRun,
@@ -77,17 +83,17 @@ func TestPipelineRun(t *testing.T) {
 	// TODO(#375): Reenable the 'fan-in and fan-out' test once it's fixed.
 	tds = []tests{{
 		name: "service account propagation",
-		testSetup: func(c *clients, namespace string, index int) {
+		testSetup: func(t *testing.T, c *clients, namespace string, index int) {
 			t.Helper()
 			if _, err := c.KubeClient.Kube.CoreV1().Secrets(namespace).Create(getPipelineRunSecret(index, namespace)); err != nil {
-				t.Fatalf("Failed to create secret `%s`: %s", getName(hwSecret, index), err)
+				t.Fatalf("Failed to create secret `%s`: %s", getName(secretName, index), err)
 			}
 
 			if _, err := c.KubeClient.Kube.CoreV1().ServiceAccounts(namespace).Create(getPipelineRunServiceAccount(index, namespace)); err != nil {
-				t.Fatalf("Failed to create SA `%s`: %s", getName(hwSA, index), err)
+				t.Fatalf("Failed to create SA `%s`: %s", getName(saName, index), err)
 			}
 
-			task := tb.Task(getName(hwTaskName, index), namespace, tb.TaskSpec(
+			task := tb.Task(getName(taskName, index), namespace, tb.TaskSpec(
 				// Reference build: https://github.com/knative/build/tree/master/test/docker-basic
 				tb.Step("config-docker", "gcr.io/cloud-builders/docker",
 					tb.Command("docker"),
@@ -102,14 +108,14 @@ func TestPipelineRun(t *testing.T) {
 				})),
 			))
 			if _, err := c.TaskClient.Create(task); err != nil {
-				t.Fatalf("Failed to create Task `%s`: %s", getName(hwTaskName, index), err)
+				t.Fatalf("Failed to create Task `%s`: %s", getName(taskName, index), err)
 			}
 
 			if _, err := c.PipelineClient.Create(getHelloWorldPipelineWithSingularTask(index, namespace)); err != nil {
-				t.Fatalf("Failed to create Pipeline `%s`: %s", getName(hwPipelineName, index), err)
+				t.Fatalf("Failed to create Pipeline `%s`: %s", getName(pipelineName, index), err)
 			}
 		},
-		expectedTaskRuns: []string{hwPipelineTaskName1},
+		expectedTaskRuns: []string{task1Name},
 		// 1 from PipelineRun and 1 from Tasks defined in pipelinerun
 		expectedNumberOfEvents: 2,
 		pipelineRunFunc:        getHelloWorldPipelineRun,
@@ -129,9 +135,9 @@ func TestPipelineRun(t *testing.T) {
 			defer tearDown(t, logger, c, namespace)
 
 			logger.Infof("Setting up test resources for %q test in namespace %s", td.name, namespace)
-			td.testSetup(c, namespace, i)
+			td.testSetup(t, c, namespace, i)
 
-			prName := fmt.Sprintf("%s%d", hwPipelineRunName, i)
+			prName := fmt.Sprintf("%s%d", pipelineRunName, i)
 			if _, err := c.PipelineRunClient.Create(td.pipelineRunFunc(i, namespace)); err != nil {
 				t.Fatalf("Failed to create PipelineRun `%s`: %s", prName, err)
 			}
@@ -155,8 +161,8 @@ func TestPipelineRun(t *testing.T) {
 					t.Fatalf("Expected TaskRun %s to have succeeded but Status is %v", taskRunName, r.Status)
 				}
 				for name, key := range map[string]string{
-					hwPipelineRunName: pipeline.PipelineRunLabelKey,
-					hwPipelineName:    pipeline.PipelineLabelKey,
+					pipelineRunName: pipeline.PipelineRunLabelKey,
+					pipelineName:    pipeline.PipelineLabelKey,
 				} {
 					expectedName := getName(name, i)
 					lbl := pipeline.GroupName + key
@@ -183,8 +189,8 @@ func TestPipelineRun(t *testing.T) {
 }
 
 func getHelloWorldPipelineWithSingularTask(suffix int, namespace string) *v1alpha1.Pipeline {
-	return tb.Pipeline(getName(hwPipelineName, suffix), namespace, tb.PipelineSpec(
-		tb.PipelineTask(hwPipelineTaskName1, getName(hwTaskName, suffix)),
+	return tb.Pipeline(getName(pipelineName, suffix), namespace, tb.PipelineSpec(
+		tb.PipelineTask(task1Name, getName(taskName, suffix)),
 	))
 }
 
@@ -239,7 +245,7 @@ func getFanInFanOutTasks(namespace string) []*v1alpha1.Task {
 }
 
 func getFanInFanOutPipeline(suffix int, namespace string) *v1alpha1.Pipeline {
-	return tb.Pipeline(getName(hwPipelineName, suffix), namespace, tb.PipelineSpec(
+	return tb.Pipeline(getName(pipelineName, suffix), namespace, tb.PipelineSpec(
 		tb.PipelineDeclaredResource("git-repo", "git"),
 		tb.PipelineTask("create-file-kritis", "create-file",
 			tb.PipelineTaskOutputResource("workspace", "git-repo"),
@@ -272,16 +278,16 @@ func getPipelineRunServiceAccount(suffix int, namespace string) *corev1.ServiceA
 	return &corev1.ServiceAccount{
 		ObjectMeta: metav1.ObjectMeta{
 			Namespace: namespace,
-			Name:      getName(hwSA, suffix),
+			Name:      getName(saName, suffix),
 		},
 		Secrets: []corev1.ObjectReference{{
-			Name: getName(hwSecret, suffix),
+			Name: getName(secretName, suffix),
 		}},
 	}
 }
 func getFanInFanOutPipelineRun(suffix int, namespace string) *v1alpha1.PipelineRun {
-	return tb.PipelineRun(getName(hwPipelineRunName, suffix), namespace, tb.PipelineRunSpec(
-		getName(hwPipelineName, suffix),
+	return tb.PipelineRun(getName(pipelineRunName, suffix), namespace, tb.PipelineRunSpec(
+		getName(pipelineName, suffix),
 		tb.PipelineRunResourceBinding("git-repo", tb.PipelineResourceBindingRef("kritis-resource-git")),
 	))
 }
@@ -299,7 +305,7 @@ func getPipelineRunSecret(suffix int, namespace string) *corev1.Secret {
 	return &corev1.Secret{
 		ObjectMeta: metav1.ObjectMeta{
 			Namespace: namespace,
-			Name:      getName(hwSecret, suffix),
+			Name:      getName(secretName, suffix),
 			Annotations: map[string]string{
 				"build.knative.dev/docker-0": "https://us.gcr.io",
 				"build.knative.dev/docker-1": "https://eu.gcr.io",
@@ -316,9 +322,9 @@ func getPipelineRunSecret(suffix int, namespace string) *corev1.Secret {
 }
 
 func getHelloWorldPipelineRun(suffix int, namespace string) *v1alpha1.PipelineRun {
-	return tb.PipelineRun(getName(hwPipelineRunName, suffix), namespace, tb.PipelineRunSpec(
-		getName(hwPipelineName, suffix),
-		tb.PipelineRunServiceAccount(fmt.Sprintf("%s%d", hwSA, suffix)),
+	return tb.PipelineRun(getName(pipelineRunName, suffix), namespace, tb.PipelineRunSpec(
+		getName(pipelineName, suffix),
+		tb.PipelineRunServiceAccount(fmt.Sprintf("%s%d", saName, suffix)),
 	))
 }
 
diff --git a/test/status_test.go b/test/status_test.go
index a7215da5251..e68a1dce358 100644
--- a/test/status_test.go
+++ b/test/status_test.go
@@ -40,18 +40,18 @@ func TestTaskRunPipelineRunStatus(t *testing.T) {
 		tb.Step("foo", "busybox", tb.Command("ls", "-la")),
 	))
 	if _, err := c.TaskClient.Create(task); err != nil {
-		t.Fatalf("Failed to create Task `%s`: %s", hwTaskName, err)
+		t.Fatalf("Failed to create Task: %s", err)
 	}
 	taskRun := tb.TaskRun("apple", namespace, tb.TaskRunSpec(
 		tb.TaskRunTaskRef("banana"), tb.TaskRunServiceAccount("inexistent"),
 	))
 	if _, err := c.TaskRunClient.Create(taskRun); err != nil {
-		t.Fatalf("Failed to create TaskRun `%s`: %s", hwTaskRunName, err)
+		t.Fatalf("Failed to create TaskRun: %s", err)
 	}
 
-	logger.Infof("Waiting for TaskRun %s in namespace %s to fail", hwTaskRunName, namespace)
+	logger.Infof("Waiting for TaskRun in namespace %s to fail", namespace)
 	if err := WaitForTaskRunState(c, "apple", TaskRunFailed("apple"), "BuildValidationFailed"); err != nil {
-		t.Errorf("Error waiting for TaskRun %s to finish: %s", hwTaskRunName, err)
+		t.Errorf("Error waiting for TaskRun to finish: %s", err)
 	}
 
 	pipeline := tb.Pipeline("tomatoes", namespace,
@@ -67,8 +67,8 @@ func TestTaskRunPipelineRunStatus(t *testing.T) {
 		t.Fatalf("Failed to create PipelineRun `%s`: %s", "pear", err)
 	}
 
-	logger.Infof("Waiting for PipelineRun %s in namespace %s to fail", hwTaskRunName, namespace)
+	logger.Infof("Waiting for PipelineRun in namespace %s to fail", namespace)
 	if err := WaitForPipelineRunState(c, "pear", pipelineRunTimeout, PipelineRunFailed("pear"), "BuildValidationFailed"); err != nil {
-		t.Errorf("Error waiting for TaskRun %s to finish: %s", hwTaskRunName, err)
+		t.Errorf("Error waiting for TaskRun to finish: %s", err)
 	}
 }
diff --git a/test/taskrun_test.go b/test/taskrun_test.go
deleted file mode 100644
index 73dea907f1b..00000000000
--- a/test/taskrun_test.go
+++ /dev/null
@@ -1,99 +0,0 @@
-// +build e2e
-
-/*
-Copyright 2018 Knative Authors LLC
-Licensed 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 test
-
-import (
-	"fmt"
-	"strings"
-	"testing"
-	"time"
-
-	tb "github.com/knative/build-pipeline/test/builder"
-	duckv1alpha1 "github.com/knative/pkg/apis/duck/v1alpha1"
-	"github.com/knative/build-pipeline/pkg/apis/pipeline/v1alpha1"
-	knativetest "github.com/knative/pkg/test"
-	"github.com/knative/pkg/test/logging"
-	corev1 "k8s.io/api/core/v1"
-)
-
-// TestTaskRun is an integration test that will verify a very simple "hello world" TaskRun can be
-// executed.
-func TestTaskRun(t *testing.T) {
-	logger := logging.GetContextLogger(t.Name())
-	c, namespace := setup(t, logger)
-	t.Parallel()
-
-	knativetest.CleanupOnInterrupt(func() { tearDown(t, logger, c, namespace) }, logger)
-	defer tearDown(t, logger, c, namespace)
-
-	logger.Infof("Creating Task and TaskRun in namespace %s", namespace)
-	if _, err := c.TaskClient.Create(getHelloWorldTask(namespace, []string{"/bin/sh", "-c", fmt.Sprintf("echo %s", taskOutput)})); err != nil {
-		t.Fatalf("Failed to create Task `%s`: %s", hwTaskName, err)
-	}
-	if _, err := c.TaskRunClient.Create(getHelloWorldTaskRun(namespace)); err != nil {
-		t.Fatalf("Failed to create TaskRun `%s`: %s", hwTaskRunName, err)
-	}
-
-	logger.Infof("Waiting for TaskRun %s in namespace %s to complete", hwTaskRunName, namespace)
-	if err := WaitForTaskRunState(c, hwTaskRunName, TaskRunSucceed(hwTaskRunName), "TaskRunSuccess"); err != nil {
-		t.Errorf("Error waiting for TaskRun %s to finish: %s", hwTaskRunName, err)
-	}
-
-	// The volume created with the results will have the same name as the TaskRun
-	logger.Infof("Verifying TaskRun %s output in volume %s", hwTaskRunName, hwTaskRunName)
-	output := getBuildOutputFromVolume(t, logger, c, namespace, taskOutput)
-	if !strings.Contains(output, taskOutput) {
-		t.Fatalf("Expected output %s from pod %s but got %s", buildOutput, hwValidationPodName, output)
-	}
-}
-
-// TestTaskRunTimeout is an integration test that will verify a TaskRun can be timed out.
-func TestTaskRunTimeout(t *testing.T) {
-	logger := logging.GetContextLogger(t.Name())
-	c, namespace := setup(t, logger)
-
-	knativetest.CleanupOnInterrupt(func() { tearDown(t, logger, c, namespace) }, logger)
-	defer tearDown(t, logger, c, namespace)
-
-	logger.Infof("Creating Task and TaskRun in namespace %s", namespace)
-	if _, err := c.TaskClient.Create(tb.Task(hwTaskName, namespace,
-		tb.TaskSpec(tb.Step(hwContainerName, "busybox", tb.Command("/bin/bash"), tb.Args("-c", "sleep 300")),
-			tb.TaskTimeout(10 * time.Second)))); err != nil {
-		t.Fatalf("Failed to create Task `%s`: %s", hwTaskName, err)
-	}
-	if _, err := c.TaskRunClient.Create(tb.TaskRun(hwTaskRunName, namespace, tb.TaskRunSpec(tb.TaskRunTaskRef(hwTaskName)))); err != nil {
-		t.Fatalf("Failed to create TaskRun `%s`: %s", hwTaskRunName, err)
-	}
-
-	logger.Infof("Waiting for TaskRun %s in namespace %s to complete", hwTaskRunName, namespace)
-	if err := WaitForTaskRunState(c, hwTaskRunName, func(tr *v1alpha1.TaskRun) (bool, error) {
-		cond := tr.Status.GetCondition(duckv1alpha1.ConditionSucceeded)
-		if cond != nil {
-			if cond.Status == corev1.ConditionFalse {
-				if cond.Reason == "TaskRunTimeout" {
-					return true, nil
-				}
-				return true, fmt.Errorf("taskRun %s completed with the wrong reason: %s", hwTaskRunName, cond.Reason)
-			} else if cond.Status == corev1.ConditionTrue {
-				return true, fmt.Errorf("taskRun %s completed successfully, should have been timed out", hwTaskRunName)
-			}
-		}
-
-		return false, nil
-	}, "TaskRunTimeout"); err != nil {
-		t.Errorf("Error waiting for TaskRun %s to finish: %s", hwTaskRunName, err)
-	}
-}
diff --git a/test/timeout_test.go b/test/timeout_test.go
index 28aa739284d..2310501384d 100644
--- a/test/timeout_test.go
+++ b/test/timeout_test.go
@@ -36,6 +36,7 @@ import (
 func TestPipelineRunTimeout(t *testing.T) {
 	logger := logging.GetContextLogger(t.Name())
 	c, namespace := setup(t, logger)
+	t.Parallel()
 
 	knativetest.CleanupOnInterrupt(func() { tearDown(t, logger, c, namespace) }, logger)
 	defer tearDown(t, logger, c, namespace)
@@ -44,7 +45,7 @@ func TestPipelineRunTimeout(t *testing.T) {
 	task := tb.Task("banana", namespace, tb.TaskSpec(
 		tb.Step("foo", "busybox", tb.Command("sleep"), tb.Args("10"))))
 	if _, err := c.TaskClient.Create(task); err != nil {
-		t.Fatalf("Failed to create Task `%s`: %s", hwTaskName, err)
+		t.Fatalf("Failed to create Task `%s`: %s", "banana", err)
 	}
 
 	pipeline := tb.Pipeline("tomatoes", namespace,
@@ -149,3 +150,42 @@ func TestPipelineRunTimeout(t *testing.T) {
 		t.Fatalf("Error waiting for PipelineRun %s to finish: %s", secondPipelineRun.Name, err)
 	}
 }
+
+// TestTaskRunTimeout is an integration test that will verify a TaskRun can be timed out.
+func TestTaskRunTimeout(t *testing.T) {
+	logger := logging.GetContextLogger(t.Name())
+	c, namespace := setup(t, logger)
+	t.Parallel()
+
+	knativetest.CleanupOnInterrupt(func() { tearDown(t, logger, c, namespace) }, logger)
+	defer tearDown(t, logger, c, namespace)
+
+	logger.Infof("Creating Task and TaskRun in namespace %s", namespace)
+	if _, err := c.TaskClient.Create(tb.Task("giraffe", namespace,
+		tb.TaskSpec(tb.Step("amazing-busybox", "busybox", tb.Command("/bin/bash"), tb.Args("-c", "sleep 300")),
+			tb.TaskTimeout(10*time.Second)))); err != nil {
+		t.Fatalf("Failed to create Task `%s`: %s", "giraffe", err)
+	}
+	if _, err := c.TaskRunClient.Create(tb.TaskRun("run-giraffe", namespace, tb.TaskRunSpec(tb.TaskRunTaskRef("giraffe")))); err != nil {
+		t.Fatalf("Failed to create TaskRun `%s`: %s", "run-giraffe", err)
+	}
+
+	logger.Infof("Waiting for TaskRun %s in namespace %s to complete", "run-giraffe", namespace)
+	if err := WaitForTaskRunState(c, "run-giraffe", func(tr *v1alpha1.TaskRun) (bool, error) {
+		cond := tr.Status.GetCondition(duckv1alpha1.ConditionSucceeded)
+		if cond != nil {
+			if cond.Status == corev1.ConditionFalse {
+				if cond.Reason == "TaskRunTimeout" {
+					return true, nil
+				}
+				return true, fmt.Errorf("taskRun %s completed with the wrong reason: %s", "run-giraffe", cond.Reason)
+			} else if cond.Status == corev1.ConditionTrue {
+				return true, fmt.Errorf("taskRun %s completed successfully, should have been timed out", "run-giraffe")
+			}
+		}
+
+		return false, nil
+	}, "TaskRunTimeout"); err != nil {
+		t.Errorf("Error waiting for TaskRun %s to finish: %s", "run-giraffe", err)
+	}
+}