[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
This commit is contained in:
parent
a4042cd2a4
commit
f93e354cb2
tensorflow
@ -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);
|
||||
GetDatasetFromVariantTensor(return_values[0], &returned_dataset));
|
||||
|
||||
// Create an iterator for the dataset that was returned by
|
||||
// `f`. This transfers ownership of the dataset to the
|
||||
// iterator.
|
||||
// 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"
|
||||
|
Loading…
Reference in New Issue
Block a user