提交 f93e354c 编写于 作者: D Derek Murray 提交者: TensorFlower Gardener

[tf.contrib.data] Switch backend Dataset representation to DT_VARIANT.

This change introduces a new `DatasetWrapper` type that wraps a
`DatasetBase*` and can be stored in a DT_VARIANT tensor. All Dataset
ops now consume and produce DT_VARIANT instead of DT_RESOURCE, and the
underlying implementation is simplified because the `DatasetWrapper`
can be passed directly by value without using the `ResourceMgr`.

PiperOrigin-RevId: 168240571
上级 a4042cd2
......@@ -445,12 +445,14 @@ class Dataset(object):
def __init__(self):
pass
# TODO(mrry): Rename this to `make_dataset_variant()`,
# `make_dataset_tensor()`, or something else more accurate.
@abc.abstractmethod
def make_dataset_resource(self):
"""Creates a `tf.Tensor` of `tf.resource` tensor representing this dataset.
"""Creates a scalar `tf.Tensor` of `tf.variant` representing this dataset.
Returns:
A scalar `tf.Tensor` of `tf.resource` type, which represents this dataset.
A scalar `tf.Tensor` of `tf.variant` type, which represents this dataset.
"""
raise NotImplementedError("Dataset.make_dataset_resource")
......@@ -1924,17 +1926,17 @@ def _should_unpack_args(args):
return type(args) is tuple # pylint: disable=unidiomatic-typecheck
class _ResourceDataset(Dataset):
"""A Dataset wrapper for a tf.resource-typed function argument."""
class _VariantDataset(Dataset):
"""A Dataset wrapper for a tf.variant-typed function argument."""
def __init__(self, dataset_resource, output_types, output_shapes):
super(_ResourceDataset, self).__init__()
self._dataset_resource = dataset_resource,
def __init__(self, dataset_variant, output_types, output_shapes):
super(_VariantDataset, self).__init__()
self._dataset_variant = dataset_variant
self._output_types = output_types
self._output_shapes = output_shapes
def make_dataset_resource(self):
return self._dataset_resource
return self._dataset_variant
@property
def output_shapes(self):
......@@ -2686,13 +2688,13 @@ class GroupByWindowDataset(Dataset):
def _make_reduce_func(self, reduce_func, input_dataset):
"""Make wrapping Defun for reduce_func."""
@function.Defun(dtypes.int64, dtypes.resource)
def tf_reduce_func(key, window_dataset_resource):
@function.Defun(dtypes.int64, dtypes.variant)
def tf_reduce_func(key, window_dataset_variant):
"""A wrapper for Defun that facilitates shape inference."""
key.set_shape([])
window_dataset = _ResourceDataset(window_dataset_resource,
input_dataset.output_types,
input_dataset.output_shapes)
window_dataset = _VariantDataset(window_dataset_variant,
input_dataset.output_types,
input_dataset.output_shapes)
output_dataset = reduce_func(key, window_dataset)
if not isinstance(output_dataset, Dataset):
raise TypeError("`reduce_func` must return a `Dataset` object.")
......
......@@ -17,38 +17,112 @@ limitations under the License.
namespace tensorflow {
namespace {
// A wrapper class for storing a `DatasetBase` instance in a DT_VARIANT tensor.
// Objects of the wrapper class own a reference on an instance of `DatasetBase`,
// and the wrapper's copy constructor and destructor take care of managing the
// reference count.
//
// NOTE(mrry): This is not a feature-complete implementation of the DT_VARIANT
// specification. In particular, we cannot currently serialize an arbitrary
// `DatasetBase` object, so the `Encode()` and `Decode()` methods are not
// implemented.
class DatasetVariantWrapper {
public:
DatasetVariantWrapper() : dataset_(nullptr) {}
// Transfers ownership of `dataset` to `*this`.
explicit DatasetVariantWrapper(DatasetBase* dataset) : dataset_(dataset) {}
DatasetVariantWrapper(const DatasetVariantWrapper& other)
: dataset_(other.dataset_) {
if (dataset_) dataset_->Ref();
}
~DatasetVariantWrapper() {
if (dataset_) dataset_->Unref();
}
DatasetBase* get() const { return dataset_; }
string TypeName() const { return "tensorflow::DatasetVariantWrapper"; }
string DebugString() const {
if (dataset_) {
return dataset_->DebugString();
} else {
return "<Uninitialized DatasetVariantWrapper>";
}
}
void Encode(VariantTensorData* data) const {
LOG(ERROR) << "The Encode() method is not implemented for "
"DatasetVariantWrapper objects.";
}
bool Decode(const VariantTensorData& data) {
LOG(ERROR) << "The Decode() method is not implemented for "
"DatasetVariantWrapper objects.";
return false;
}
private:
DatasetBase* const dataset_; // Owns one reference.
};
} // namespace
Status GetDatasetFromVariantTensor(const Tensor& tensor,
DatasetBase** out_dataset) {
if (!(tensor.dtype() == DT_VARIANT ||
TensorShapeUtils::IsScalar(tensor.shape()))) {
return errors::InvalidArgument(
"Dataset tensor must be a scalar of dtype DT_VARIANT.");
}
const Variant& variant = tensor.scalar<Variant>()();
const DatasetVariantWrapper* wrapper = variant.get<DatasetVariantWrapper>();
if (wrapper == nullptr) {
return errors::InvalidArgument("Tensor must be a Dataset object.");
}
*out_dataset = wrapper->get();
if (*out_dataset == nullptr) {
return errors::Internal("Read uninitialized Dataset variant.");
}
return Status::OK();
}
Status StoreDatasetInVariantTensor(DatasetBase* dataset, Tensor* tensor) {
if (!(tensor->dtype() == DT_VARIANT ||
TensorShapeUtils::IsScalar(tensor->shape()))) {
return errors::InvalidArgument(
"Dataset tensor must be a scalar of dtype DT_VARIANT.");
}
tensor->scalar<Variant>()() = DatasetVariantWrapper(dataset);
return Status::OK();
}
void DatasetOpKernel::Compute(OpKernelContext* ctx) {
DatasetBase* dataset = nullptr;
MakeDataset(ctx, &dataset);
if (ctx->status().ok()) {
Tensor* output = nullptr;
OP_REQUIRES_OK(ctx, ctx->allocate_output(0, TensorShape({}), &output));
ResourceHandle handle = MakeResourceHandle<DatasetBase>(
ctx, ctx->step_container()->name(), name());
OP_REQUIRES_OK(ctx, CreateResource(ctx, handle, dataset));
output->flat<ResourceHandle>()(0) = handle;
OP_REQUIRES_OK(ctx, StoreDatasetInVariantTensor(dataset, output));
}
}
void UnaryDatasetOpKernel::MakeDataset(OpKernelContext* ctx,
DatasetBase** output) {
DatasetBase* input;
OP_REQUIRES_OK(ctx, LookupResource(ctx, HandleFromInput(ctx, 0), &input));
core::ScopedUnref unref_input(input);
OP_REQUIRES_OK(ctx, GetDatasetFromVariantTensor(ctx->input(0), &input));
MakeDataset(ctx, input, output);
}
void BinaryDatasetOpKernel::MakeDataset(OpKernelContext* ctx,
DatasetBase** output) {
DatasetBase* input;
OP_REQUIRES_OK(ctx, GetDatasetFromVariantTensor(ctx->input(0), &input));
DatasetBase* another_input;
OP_REQUIRES_OK(ctx, LookupResource(ctx, HandleFromInput(ctx, 0), &input));
OP_REQUIRES_OK(ctx,
LookupResource(ctx, HandleFromInput(ctx, 1), &another_input));
core::ScopedUnref unref_input(input);
core::ScopedUnref unref_another_input(another_input);
GetDatasetFromVariantTensor(ctx->input(1), &another_input));
MakeDataset(ctx, input, another_input, output);
}
......
......@@ -19,6 +19,8 @@ limitations under the License.
#include "tensorflow/core/framework/register_types.h"
#include "tensorflow/core/framework/resource_mgr.h"
#include "tensorflow/core/framework/variant_encode_decode.h"
#include "tensorflow/core/framework/variant_tensor_data.h"
#include "tensorflow/core/lib/strings/str_util.h"
#include "tensorflow/core/lib/strings/strcat.h"
#include "tensorflow/core/platform/tracing.h"
......@@ -239,7 +241,7 @@ class IteratorBase {
// Represents a (potentially infinite) range of outputs, where each
// output is a tuple of tensors.
class DatasetBase : public ResourceBase {
class DatasetBase : public core::RefCounted {
public:
// Returns a new iterator for iterating over the range of elements in
// this dataset.
......@@ -265,6 +267,9 @@ class DatasetBase : public ResourceBase {
// (and possibly partially defined) shapes of each tuple component
// in the outputs of this dataset.
virtual const std::vector<PartialTensorShape>& output_shapes() const = 0;
// A human-readable debug string for this dataset.
virtual string DebugString() = 0;
};
// Represents an iterator that is associated with a particular parent dataset.
......@@ -373,6 +378,22 @@ class BinaryDatasetOpKernel : public DatasetOpKernel {
DatasetBase** output) = 0;
};
// Validates and extracts a `DatasetBase` object from `tensor`.
//
// `tensor` must have been written by a call to SetVariantTensorToDataset().
//
// The retrieved pointer is a borrowed reference to the dataset, which is owned
// by the tensor. The consumer must either acquire its own reference to the
// dataset by calling `(*out_dataset)->Ref()`, or ensure that `tensor` is not
// destroyed or mutated while the retrieved pointer is in use.
Status GetDatasetFromVariantTensor(const Tensor& tensor,
DatasetBase** out_dataset);
// Stores a `DatasetBase` object in `tensor`.
//
// The ownership of `dataset` is transferred to `tensor`.
Status StoreDatasetInVariantTensor(DatasetBase* dataset, Tensor* tensor);
} // namespace tensorflow
#endif // THIRD_PARTY_TENSORFLOW_CORE_KERNELS_DATASET_H_
......@@ -39,37 +39,20 @@ Status MakeIteratorFromInputElement(
std::vector<Tensor> return_values;
TF_RETURN_IF_ERROR(captured_func->Run(opts, input_element, &return_values));
if (!(return_values.size() == 1 && return_values[0].dtype() == DT_RESOURCE &&
if (!(return_values.size() == 1 && return_values[0].dtype() == DT_VARIANT &&
TensorShapeUtils::IsScalar(return_values[0].shape()))) {
return errors::InvalidArgument(
"Function must return a single scalar of dtype DT_RESOURCE.");
"Function must return a single scalar of dtype DT_VARIANT.");
}
// Retrieve the dataset that was created in `f`.
DatasetBase* returned_dataset;
const ResourceHandle& dataset_resource =
return_values[0].scalar<ResourceHandle>()();
TF_RETURN_IF_ERROR(
GetDatasetFromVariantTensor(return_values[0], &returned_dataset));
// NOTE(mrry): We cannot use the core `LookupResource()` or
// `DeleteResource()` functions, because we have an
// `IteratorContext*` and not an `OpKernelContext*`, so we
// replicate the necessary functionality here.
auto type_index = MakeTypeIndex<DatasetBase>();
if (type_index.hash_code() != dataset_resource.hash_code()) {
return errors::InvalidArgument("Function must return a Dataset resource.");
}
TF_RETURN_IF_ERROR(captured_func->resource_manager()->Lookup(
dataset_resource.container(), dataset_resource.name(),
&returned_dataset));
core::ScopedUnref unref_dataset(returned_dataset);
// Create an iterator for the dataset that was returned by
// `f`. This transfers ownership of the dataset to the
// iterator, so we can delete it from the resource manager.
// Create an iterator for the dataset that was returned by `f`.
*out_iterator = returned_dataset->MakeIterator(
strings::StrCat(prefix, "[", thread_index, "]"));
TF_RETURN_IF_ERROR(captured_func->resource_manager()->Delete<DatasetBase>(
dataset_resource.container(), dataset_resource.name()));
return Status::OK();
}
......
......@@ -274,30 +274,9 @@ class GroupByWindowDatasetOp : public UnaryDatasetOpKernel {
Tensor key_arg(DT_INT64, TensorShape({}));
key_arg.scalar<int64>()() = key;
Tensor group_dataset_arg(DT_RESOURCE, TensorShape({}));
// NOTE(mrry): We cannot use the core `MakeResourceHandle()`,
// `LookupResource()` or `DeleteResource()` functions, because
// we have an `IteratorContext*` and not an
// `OpKernelContext*`, so we replicate the necessary
// functionality here.
ResourceHandle group_dataset_handle;
group_dataset_handle.set_device(
dataset()->captured_reduce_func_->device()->attributes().name());
group_dataset_handle.set_container(step_container.name());
group_dataset_handle.set_name(kWindowResourceName);
auto type_index = MakeTypeIndex<DatasetBase>();
group_dataset_handle.set_hash_code(type_index.hash_code());
group_dataset_handle.set_maybe_type_name(type_index.name());
// NOTE(mrry): Ownership of `group_dataset` transfers to
// `step_container` here.
TF_RETURN_IF_ERROR(dataset()
->captured_reduce_func_->resource_manager()
->Create<DatasetBase>(
group_dataset_handle.container(),
group_dataset_handle.name(), group_dataset));
group_dataset_arg.scalar<ResourceHandle>()() = group_dataset_handle;
Tensor group_dataset_arg(DT_VARIANT, TensorShape({}));
TF_RETURN_IF_ERROR(
StoreDatasetInVariantTensor(group_dataset, &group_dataset_arg));
std::vector<Tensor> args(
{std::move(key_arg), std::move(group_dataset_arg)});
......@@ -307,30 +286,20 @@ class GroupByWindowDatasetOp : public UnaryDatasetOpKernel {
dataset()->captured_reduce_func_->Run(opts, args, &return_values));
if (!(return_values.size() == 1 &&
return_values[0].dtype() == DT_RESOURCE &&
return_values[0].dtype() == DT_VARIANT &&
TensorShapeUtils::IsScalar(return_values[0].shape()))) {
return errors::InvalidArgument(
"`reduce_func` must return a single scalar of dtype "
"DT_RESOURCE.");
"DT_VARIANT.");
}
// Retrieve the dataset that was created in `f`.
// `returned_dataset` is borrowed from the `return_values[0]`.
DatasetBase* returned_dataset;
const ResourceHandle& dataset_resource =
return_values[0].scalar<ResourceHandle>()();
if (type_index.hash_code() != dataset_resource.hash_code()) {
return errors::InvalidArgument(
"`reduce_func` must return a Dataset resource.");
}
TF_RETURN_IF_ERROR(
dataset()->captured_reduce_func_->resource_manager()->Lookup(
dataset_resource.container(), dataset_resource.name(),
&returned_dataset));
core::ScopedUnref unref_returned_dataset(returned_dataset);
// Create an iterator for the dataset that was returned by
// `f`. This transfers ownership of the dataset to the
// iterator.
GetDatasetFromVariantTensor(return_values[0], &returned_dataset));
// Create an iterator for the dataset that was returned by `f`.
current_group_iterator_ = returned_dataset->MakeIterator(prefix());
return Status::OK();
}
......
......@@ -30,20 +30,18 @@ namespace {
// See documentation in ../ops/dataset_ops.cc for a high-level
// description of the following op.
class InterleaveDatasetOp : public OpKernel {
class InterleaveDatasetOp : public UnaryDatasetOpKernel {
public:
explicit InterleaveDatasetOp(OpKernelConstruction* ctx)
: OpKernel(ctx), graph_def_version_(ctx->graph_def_version()) {
: UnaryDatasetOpKernel(ctx),
graph_def_version_(ctx->graph_def_version()) {
OP_REQUIRES_OK(ctx, ctx->GetAttr("f", &func_));
OP_REQUIRES_OK(ctx, ctx->GetAttr("output_types", &output_types_));
OP_REQUIRES_OK(ctx, ctx->GetAttr("output_shapes", &output_shapes_));
}
void Compute(OpKernelContext* ctx) override {
DatasetBase* input;
OP_REQUIRES_OK(ctx, LookupResource(ctx, HandleFromInput(ctx, 0), &input));
core::ScopedUnref unref_input(input);
void MakeDataset(OpKernelContext* ctx, DatasetBase* input,
DatasetBase** output) override {
OpInputList inputs;
OP_REQUIRES_OK(ctx, ctx->input_list("other_arguments", &inputs));
std::vector<Tensor> other_arguments;
......@@ -75,16 +73,8 @@ class InterleaveDatasetOp : public OpKernel {
std::move(other_arguments),
&captured_func));
DatasetBase* dataset =
new Dataset(input, std::move(captured_func), cycle_length, block_length,
output_types_, output_shapes_);
Tensor* output = nullptr;
OP_REQUIRES_OK(ctx, ctx->allocate_output(0, TensorShape({}), &output));
ResourceHandle handle = MakeResourceHandle<DatasetBase>(
ctx, ctx->step_container()->name(), name());
OP_REQUIRES_OK(ctx, CreateResource(ctx, handle, dataset));
output->flat<ResourceHandle>()(0) = handle;
*output = new Dataset(input, std::move(captured_func), cycle_length,
block_length, output_types_, output_shapes_);
}
private:
......
......@@ -175,8 +175,7 @@ class MakeIteratorOp : public OpKernel {
void Compute(OpKernelContext* ctx) override {
DatasetBase* dataset;
OP_REQUIRES_OK(ctx, LookupResource(ctx, HandleFromInput(ctx, 0), &dataset));
core::ScopedUnref unref_dataset(dataset);
OP_REQUIRES_OK(ctx, GetDatasetFromVariantTensor(ctx->input(0), &dataset));
IteratorResource* iterator_resource;
OP_REQUIRES_OK(
ctx, LookupResource(ctx, HandleFromInput(ctx, 1), &iterator_resource));
......@@ -345,26 +344,19 @@ class OneShotIteratorOp : public AsyncOpKernel {
});
n.WaitForNotification();
TF_RETURN_IF_ERROR(factory_status);
if (return_values.size() != 1 || return_values[0].dtype() != DT_RESOURCE ||
if (return_values.size() != 1 || return_values[0].dtype() != DT_VARIANT ||
!TensorShapeUtils::IsScalar(return_values[0].shape())) {
return errors::InvalidArgument(
"The `dataset_factory` function must return "
"a single scalar of dtype DT_RESOURCE.");
"a single scalar of dtype DT_VARIANT.");
}
// Retrieve the dataset that was created in the factory function.
DatasetBase* dataset;
const ResourceHandle& dataset_resource =
return_values[0].flat<ResourceHandle>()(0);
TF_RETURN_IF_ERROR(LookupResource(ctx, dataset_resource, &dataset));
core::ScopedUnref unref_dataset(dataset);
// Create an iterator for the dataset that was created in the
// factory function. This transfers ownership of the dataset to
// the iterator, so we can delete it from the resource manager.
// factory function.
DatasetBase* dataset;
TF_RETURN_IF_ERROR(GetDatasetFromVariantTensor(return_values[0], &dataset));
TF_RETURN_IF_ERROR(
(*iterator)->set_iterator(dataset->MakeIterator("Iterator")));
TF_RETURN_IF_ERROR(DeleteResource<DatasetBase>(ctx, dataset_resource));
(*iterator)->Ref();
return Status::OK();
......
......@@ -30,29 +30,12 @@ class ZipDatasetOp : public DatasetOpKernel {
void MakeDataset(OpKernelContext* ctx, DatasetBase** output) override {
std::vector<DatasetBase*> inputs;
Status s;
for (size_t i = 0; i < ctx->num_inputs(); ++i) {
// Create a new ZipDatasetOp::Dataset, insert it in the step-local
// container, and return it as the output.
DatasetBase* input;
s.Update(LookupResource(ctx, HandleFromInput(ctx, i), &input));
if (!s.ok()) {
break;
}
OP_REQUIRES_OK(ctx, GetDatasetFromVariantTensor(ctx->input(i), &input));
inputs.push_back(input);
}
if (s.ok()) {
*output = new Dataset(inputs);
}
// TODO(mrry): Implement a container that acts as a
// `std::vector<core::ScopedUnref>`, to avoid having to unref the
// inputs manually, and re-enable the use of `OP_REQUIRES_OK()`.
for (DatasetBase* input : inputs) {
input->Unref();
}
ctx->SetStatus(s);
*output = new Dataset(inputs);
}
private:
......
......@@ -3042,7 +3042,7 @@ op {
name: "BatchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -3050,7 +3050,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -4464,7 +4464,7 @@ op {
name: "CacheDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "filename"
......@@ -4472,7 +4472,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -4866,15 +4866,15 @@ op {
name: "ConcatenateDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "another_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -6849,7 +6849,7 @@ op {
name: "DenseToSparseBatchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -6861,7 +6861,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -9108,7 +9108,7 @@ op {
name: "FilterDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -9116,7 +9116,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "predicate"
......@@ -9165,7 +9165,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
is_stateful: true
}
......@@ -9606,7 +9606,7 @@ op {
name: "FlatMapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -9614,7 +9614,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -10499,7 +10499,7 @@ op {
name: "GroupByWindowDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "key_func_other_arguments"
......@@ -10515,7 +10515,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "key_func"
......@@ -10901,11 +10901,11 @@ op {
name: "IgnoreErrorsDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -11229,7 +11229,7 @@ op {
name: "InterleaveDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -11245,7 +11245,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -12539,7 +12539,7 @@ op {
name: "MakeIterator"
input_arg {
name: "dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "iterator"
......@@ -12589,7 +12589,7 @@ op {
name: "MapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -12597,7 +12597,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -16237,7 +16237,7 @@ op {
name: "PaddedBatchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -16254,7 +16254,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -16422,7 +16422,7 @@ op {
name: "ParallelMapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -16434,7 +16434,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -16983,7 +16983,7 @@ op {
name: "PrefetchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "buffer_size"
......@@ -16991,7 +16991,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -19719,7 +19719,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -20514,7 +20514,7 @@ op {
name: "RepeatDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -20522,7 +20522,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -24723,7 +24723,7 @@ op {
name: "ShuffleDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "buffer_size"
......@@ -24739,7 +24739,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -24913,7 +24913,7 @@ op {
name: "SkipDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -24921,7 +24921,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -25038,7 +25038,7 @@ op {
name: "SloppyInterleaveDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -25054,7 +25054,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -27776,7 +27776,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Tvalues"
......@@ -27978,7 +27978,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -29283,7 +29283,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
is_stateful: true
}
......@@ -29350,7 +29350,7 @@ op {
name: "TakeDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -29358,7 +29358,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -30405,7 +30405,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -30429,7 +30429,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -30521,7 +30521,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
is_stateful: true
}
......@@ -31528,12 +31528,12 @@ op {
name: "ZipDataset"
input_arg {
name: "input_datasets"
type: DT_RESOURCE
type: DT_VARIANT
number_attr: "N"
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......
......@@ -22,16 +22,23 @@ namespace tensorflow {
// --------------------------------------------------------------------------
// The ops in this section can be composed to define an input
// pipeline. Each op produces a (step-local) resource that represents
// pipeline. Each op produces a DT_VARIANT tensor that represents
// a DAG of "dataset" objects. An "dataset" object can be converted
// to a stateful "iterator" by passing the "dataset" to the
// "MakeIterator" op.
//
// TODO(b/65524810): DT_VARIANT tensors that represent "dataset" objects are
// not presently serializable. To avoid issues with constant folding, ensure
// that any "source dataset" ops (i.e. ops that output a dataset and do not
// take one as input) are marked "stateful".
REGISTER_OP("TensorDataset")
.Input("components: Toutput_types")
.Output("handle: resource")
.Output("handle: variant")
.Attr("Toutput_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape) // TODO(mrry): Validate that
// `components` have shapes
// compatible with
......@@ -42,9 +49,11 @@ Creates a dataset that emits `components` as a tuple of tensors once.
REGISTER_OP("TensorSliceDataset")
.Input("components: Toutput_types")
.Output("handle: resource")
.Output("handle: variant")
.Attr("Toutput_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape) // TODO(mrry): Validate that the
// dim-0 slices of `components`
// have shapes compatible with
......@@ -57,16 +66,18 @@ REGISTER_OP("SparseTensorSliceDataset")
.Input("indices: int64")
.Input("values: Tvalues")
.Input("dense_shape: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("Tvalues: type")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape)
.Doc(R"doc(
Creates a dataset that splits a SparseTensor into elements row-wise.
)doc");
REGISTER_OP("ZipDataset")
.Input("input_datasets: N * resource")
.Output("handle: resource")
.Input("input_datasets: N * variant")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.Attr("N: int >= 1")
......@@ -76,9 +87,9 @@ Creates a dataset that zips together `input_datasets`.
)doc");
REGISTER_OP("ConcatenateDataset")
.Input("input_dataset: resource")
.Input("another_dataset: resource")
.Output("handle: resource")
.Input("input_dataset: variant")
.Input("another_dataset: variant")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -87,9 +98,9 @@ Creates a dataset that concatenates `input_dataset` with `another_dataset`.
)doc");
REGISTER_OP("RepeatDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("count: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape) // TODO(mrry): Validate the shape
......@@ -102,9 +113,9 @@ count: A scalar representing the number of times that `input_dataset` should
)doc");
REGISTER_OP("TakeDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("count: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -117,9 +128,9 @@ count: A scalar representing the number of elements from the `input_dataset`
)doc");
REGISTER_OP("SkipDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("count: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -131,8 +142,8 @@ count: A scalar representing the number of elements from the `input_dataset`
)doc");
REGISTER_OP("IgnoreErrorsDataset")
.Input("input_dataset: resource")
.Output("handle: resource")
.Input("input_dataset: variant")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -141,9 +152,9 @@ Creates a dataset that contains the elements of `input_dataset` ignoring errors.
)doc");
REGISTER_OP("MapDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Output("handle: resource")
.Output("handle: variant")
.Attr("f: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -154,10 +165,10 @@ Creates a dataset that applies `f` to the outputs of `input_dataset`.
)doc");
REGISTER_OP("ParallelMapDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Input("num_parallel_calls: int32")
.Output("handle: resource")
.Output("handle: variant")
.Attr("f: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -174,9 +185,9 @@ num_parallel_calls: The number of concurrent invocations of `f` that process
)doc");
REGISTER_OP("PrefetchDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("buffer_size: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -188,9 +199,9 @@ buffer_size: The maximum number of elements to buffer in an iterator over
)doc");
REGISTER_OP("FlatMapDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Output("handle: resource")
.Output("handle: variant")
.Attr("f: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -200,20 +211,20 @@ REGISTER_OP("FlatMapDataset")
Creates a dataset that applies `f` to the outputs of `input_dataset`.
Unlike MapDataset, the `f` in FlatMapDataset is expected to return a
Dataset resource, and FlatMapDataset will flatten successive results
Dataset variant, and FlatMapDataset will flatten successive results
into a single Dataset.
f: A function mapping elements of `input_dataset`, concatenated with
`other_arguments`, to a Dataset resource that contains elements matching
`other_arguments`, to a Dataset variant that contains elements matching
`output_types` and `output_shapes`.
)doc");
REGISTER_OP("InterleaveDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Input("cycle_length: int64")
.Input("block_length: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("f: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -223,22 +234,22 @@ REGISTER_OP("InterleaveDataset")
Creates a dataset that applies `f` to the outputs of `input_dataset`.
Unlike MapDataset, the `f` in InterleaveDataset is expected to return
a Dataset resource, and InterleaveDataset will flatten successive
a Dataset variant, and InterleaveDataset will flatten successive
results into a single Dataset. Unlike FlatMapDataset,
InterleaveDataset will interleave sequences of up to `block_length`
consecutive elements from `cycle_length` input elements.
f: A function mapping elements of `input_dataset`, concatenated with
`other_arguments`, to a Dataset resource that contains elements matching
`other_arguments`, to a Dataset variant that contains elements matching
`output_types` and `output_shapes`.
)doc");
REGISTER_OP("SloppyInterleaveDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Input("cycle_length: int64")
.Input("block_length: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("f: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -256,17 +267,17 @@ allows the training step to proceed so long as some data is available.
!! WARNING !! This dataset is not deterministic!
f: A function mapping elements of `input_dataset`, concatenated with
`other_arguments`, to a Dataset resource that contains elements matching
`other_arguments`, to a Dataset variant that contains elements matching
`output_types` and `output_shapes`.
)doc");
REGISTER_OP("GroupByWindowDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("key_func_other_arguments: Tkey_func_other_arguments")
.Input("reduce_func_other_arguments: Treduce_func_other_arguments")
.Input(
"window_size_func_other_arguments: Twindow_size_func_other_arguments")
.Output("handle: resource")
.Output("handle: variant")
.Attr("key_func: func")
.Attr("reduce_func: func")
.Attr("window_size_func: func")
......@@ -286,9 +297,9 @@ key_func: A function mapping an element of `input_dataset`, concatenated
)doc");
REGISTER_OP("FilterDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("other_arguments: Targuments")
.Output("handle: resource")
.Output("handle: variant")
.Attr("predicate: func")
.Attr("Targuments: list(type) >= 0")
.Attr("output_types: list(type) >= 1")
......@@ -309,9 +320,9 @@ other_arguments: A list of tensors, typically values that were captured when
)doc");
REGISTER_OP("BatchDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("batch_size: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -323,11 +334,11 @@ batch_size: A scalar representing the number of elements to accumulate in a
)doc");
REGISTER_OP("PaddedBatchDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("batch_size: int64")
.Input("padded_shapes: N * int64")
.Input("padding_values: Toutput_types")
.Output("handle: resource")
.Output("handle: variant")
.Attr("Toutput_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.Attr("N: int >= 1")
......@@ -356,10 +367,10 @@ padding_values: A list of scalars containing the padding value to use for
)doc");
REGISTER_OP("DenseToSparseBatchDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("batch_size: int64")
.Input("row_shape: int64")
.Output("handle: resource")
.Output("handle: variant")
// NOTE(mrry): the 0th and 2nd elements will be DT_INT64.
.Attr("output_types: list(type) >= 1")
// NOTE(mrry): the 1st and 2nd elements will be vectors.
......@@ -379,9 +390,11 @@ REGISTER_OP("RangeDataset")
.Input("start: int64")
.Input("stop: int64")
.Input("step: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape)
.Doc(R"doc(
Creates a dataset with a range of values. Corresponds to python's xrange.
......@@ -392,11 +405,11 @@ step: corresponds to step in python's xrange().
)doc");
REGISTER_OP("ShuffleDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("buffer_size: int64")
.Input("seed: int64")
.Input("seed2: int64")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -413,9 +426,9 @@ seed2: A second scalar seed to avoid seed collision.
)doc");
REGISTER_OP("CacheDataset")
.Input("input_dataset: resource")
.Input("input_dataset: variant")
.Input("filename: string")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetShapeFn(shape_inference::ScalarShape)
......@@ -435,7 +448,9 @@ REGISTER_OP("TextLineDataset")
.Input("filenames: string")
.Input("compression_type: string")
.Input("buffer_size: int64")
.Output("handle: resource")
.Output("handle: variant")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape) // TODO(mrry): validate
// that `filenames` is
// a scalar or a
......@@ -454,9 +469,11 @@ REGISTER_OP("SqlDataset")
.Input("driver_name: string")
.Input("data_source_name: string")
.Input("query: string")
.Output("handle: resource")
.Output("handle: variant")
.Attr("output_types: list(type) >= 1")
.Attr("output_shapes: list(shape) >= 1")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape)
.Doc(R"doc(
Creates a dataset that executes a SQL query and emits rows of the result set.
......@@ -472,7 +489,9 @@ REGISTER_OP("FixedLengthRecordDataset")
.Input("record_bytes: int64")
.Input("footer_bytes: int64")
.Input("buffer_size: int64")
.Output("handle: resource")
.Output("handle: variant")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape)
.Doc(R"doc(
Creates a dataset that emits the records from one or more binary files.
......@@ -491,7 +510,9 @@ REGISTER_OP("TFRecordDataset")
.Input("filenames: string")
.Input("compression_type: string")
.Input("buffer_size: int64")
.Output("handle: resource")
.Output("handle: variant")
.SetIsStateful() // TODO(b/65524810): Source dataset ops must be marked
// stateful to inhibit constant folding.
.SetShapeFn(shape_inference::ScalarShape)
.Doc(R"doc(
Creates a dataset that emits the records from one or more TFRecord files.
......@@ -519,7 +540,7 @@ handle: A handle to the iterator that can be passed to a "MakeIterator"
)doc");
REGISTER_OP("MakeIterator")
.Input("dataset: resource")
.Input("dataset: variant")
.Input("iterator: resource")
.SetShapeFn(shape_inference::NoOutputs)
.Doc(R"doc(
......@@ -579,8 +600,8 @@ times by rerunning "MakeIterator".
handle: A handle to the iterator that can be passed to an "IteratorGetNext"
op.
dataset_factory: A function of type `() -> DT_RESOURCE`, where the returned
DT_RESOURCE is a handle to a dataset.
dataset_factory: A function of type `() -> DT_VARIANT`, where the returned
DT_VARIANT is a dataset.
)doc");
REGISTER_OP("IteratorGetNext")
......
......@@ -2793,7 +2793,7 @@ op {
name: "BatchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -2802,7 +2802,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -4229,7 +4229,7 @@ op {
name: "CacheDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "filename"
......@@ -4238,7 +4238,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -4660,15 +4660,15 @@ op {
name: "ConcatenateDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "another_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -6366,7 +6366,7 @@ op {
input_arg {
name: "input_dataset"
description: "A handle to an input dataset. Must have a single component."
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -6380,7 +6380,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -8296,7 +8296,7 @@ op {
name: "FilterDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -8305,7 +8305,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "predicate"
......@@ -8362,7 +8362,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
summary: "Creates a dataset that emits the records from one or more binary files."
is_stateful: true
......@@ -8610,7 +8610,7 @@ op {
name: "FlatMapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -8618,7 +8618,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -9601,7 +9601,7 @@ op {
name: "GroupByWindowDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "key_func_other_arguments"
......@@ -9617,7 +9617,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "key_func"
......@@ -10075,11 +10075,11 @@ op {
name: "IgnoreErrorsDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -10454,7 +10454,7 @@ op {
name: "InterleaveDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -10470,7 +10470,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -11794,7 +11794,7 @@ op {
name: "MakeIterator"
input_arg {
name: "dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "iterator"
......@@ -11847,7 +11847,7 @@ op {
name: "MapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -11855,7 +11855,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -14652,7 +14652,7 @@ op {
attr {
name: "dataset_factory"
type: "func"
description: "A function of type `() -> DT_RESOURCE`, where the returned\nDT_RESOURCE is a handle to a dataset."
description: "A function of type `() -> DT_VARIANT`, where the returned\nDT_VARIANT is a handle to a dataset."
}
attr {
name: "output_types"
......@@ -15172,7 +15172,7 @@ op {
name: "PaddedBatchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "batch_size"
......@@ -15192,7 +15192,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -15384,7 +15384,7 @@ op {
name: "ParallelMapDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -15397,7 +15397,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -15972,7 +15972,7 @@ op {
name: "PrefetchDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "buffer_size"
......@@ -15981,7 +15981,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -18931,7 +18931,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -19859,7 +19859,7 @@ op {
name: "RepeatDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -19868,7 +19868,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -24195,7 +24195,7 @@ op {
name: "ShuffleDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "buffer_size"
......@@ -24214,7 +24214,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -24399,7 +24399,7 @@ op {
name: "SkipDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -24408,7 +24408,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -24544,7 +24544,7 @@ op {
name: "SloppyInterleaveDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "other_arguments"
......@@ -24560,7 +24560,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "f"
......@@ -27625,7 +27625,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Tvalues"
......@@ -27860,7 +27860,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -29207,7 +29207,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
summary: "Creates a dataset that emits the records from one or more TFRecord files."
is_stateful: true
......@@ -29283,7 +29283,7 @@ op {
name: "TakeDataset"
input_arg {
name: "input_dataset"
type: DT_RESOURCE
type: DT_VARIANT
}
input_arg {
name: "count"
......@@ -29292,7 +29292,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......@@ -30444,7 +30444,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -30469,7 +30469,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "Toutput_types"
......@@ -30575,7 +30575,7 @@ op {
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
summary: "Creates a dataset that emits the lines of one or more text files."
is_stateful: true
......@@ -31569,12 +31569,12 @@ op {
name: "ZipDataset"
input_arg {
name: "input_datasets"
type: DT_RESOURCE
type: DT_VARIANT
number_attr: "N"
}
output_arg {
name: "handle"
type: DT_RESOURCE
type: DT_VARIANT
}
attr {
name: "output_types"
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册